Electra: field renames (#14091)

* renaming functions and fields based on consensus changes

* execution api rename

* fixing test

* reverting spectests changes, it should be changed with new version

* reverting temporarily

* revert exclusions
This commit is contained in:
james-prysm
2024-06-12 10:16:31 -05:00
committed by GitHub
parent 070a765d24
commit 3413d05b34
62 changed files with 708 additions and 707 deletions

View File

@@ -77,11 +77,11 @@ func ProcessPendingBalanceDeposits(ctx context.Context, st state.BeaconState, ac
}
}
// ProcessDepositReceipts is a function as part of electra to process execution layer deposits
func ProcessDepositReceipts(ctx context.Context, beaconState state.BeaconState, receipts []*enginev1.DepositReceipt) (state.BeaconState, error) {
_, span := trace.StartSpan(ctx, "electra.ProcessDepositReceipts")
// ProcessDepositRequests is a function as part of electra to process execution layer deposits
func ProcessDepositRequests(ctx context.Context, beaconState state.BeaconState, requests []*enginev1.DepositRequest) (state.BeaconState, error) {
_, span := trace.StartSpan(ctx, "electra.ProcessDepositRequests")
defer span.End()
// TODO: replace with 6110 logic
// return b.ProcessDepositReceipts(beaconState, receipts)
// return b.ProcessDepositRequests(beaconState, requests)
return beaconState, nil
}

View File

@@ -38,7 +38,7 @@ import (
// withdrawals_root=pre.latest_execution_payload_header.withdrawals_root,
// blob_gas_used=pre.latest_execution_payload_header.blob_gas_used,
// excess_blob_gas=pre.latest_execution_payload_header.excess_blob_gas,
// deposit_receipts_root=Root(), # [New in Electra:EIP6110]
// deposit_requests_root=Root(), # [New in Electra:EIP6110]
// withdrawal_requests_root=Root(), # [New in Electra:EIP7002],
// )
//
@@ -94,7 +94,7 @@ import (
// # Deep history valid from Capella onwards
// historical_summaries=pre.historical_summaries,
// # [New in Electra:EIP6110]
// deposit_receipts_start_index=UNSET_DEPOSIT_RECEIPTS_START_INDEX,
// deposit_requests_start_index=UNSET_DEPOSIT_REQUESTS_START_INDEX,
// # [New in Electra:EIP7251]
// deposit_balance_to_consume=0,
// exit_balance_to_consume=0,
@@ -261,14 +261,14 @@ func UpgradeToElectra(beaconState state.BeaconState) (state.BeaconState, error)
WithdrawalsRoot: wdRoot,
ExcessBlobGas: excessBlobGas,
BlobGasUsed: blobGasUsed,
DepositReceiptsRoot: bytesutil.Bytes32(0), // [New in Electra:EIP6110]
DepositRequestsRoot: bytesutil.Bytes32(0), // [New in Electra:EIP6110]
WithdrawalRequestsRoot: bytesutil.Bytes32(0), // [New in Electra:EIP7002]
},
NextWithdrawalIndex: wi,
NextWithdrawalValidatorIndex: vi,
HistoricalSummaries: summaries,
DepositReceiptsStartIndex: params.BeaconConfig().UnsetDepositReceiptsStartIndex,
DepositRequestsStartIndex: params.BeaconConfig().UnsetDepositRequestsStartIndex,
DepositBalanceToConsume: 0,
ExitBalanceToConsume: helpers.ActivationExitChurnLimit(primitives.Gwei(tab)),
EarliestExitEpoch: earliestExitEpoch,

View File

@@ -128,7 +128,7 @@ func TestUpgradeToElectra(t *testing.T) {
BlockHash: prevHeader.BlockHash(),
TransactionsRoot: txRoot,
WithdrawalsRoot: wdRoot,
DepositReceiptsRoot: bytesutil.Bytes32(0),
DepositRequestsRoot: bytesutil.Bytes32(0),
WithdrawalRequestsRoot: bytesutil.Bytes32(0),
}
require.DeepEqual(t, wanted, protoHeader)
@@ -145,9 +145,9 @@ func TestUpgradeToElectra(t *testing.T) {
require.NoError(t, err)
require.Equal(t, 0, len(summaries))
startIndex, err := mSt.DepositReceiptsStartIndex()
startIndex, err := mSt.DepositRequestsStartIndex()
require.NoError(t, err)
require.Equal(t, params.BeaconConfig().UnsetDepositReceiptsStartIndex, startIndex)
require.Equal(t, params.BeaconConfig().UnsetDepositRequestsStartIndex, startIndex)
balance, err := mSt.DepositBalanceToConsume()
require.NoError(t, err)

View File

@@ -19,15 +19,15 @@ import (
"go.opencensus.io/trace"
)
// ProcessExecutionLayerWithdrawalRequests processes the validator withdrawals from the provided execution payload
// ProcessWithdrawalRequests processes the validator withdrawals from the provided execution payload
// into the beacon state triggered by the execution layer.
//
// Spec pseudocode definition:
//
// def process_execution_layer_withdrawal_request(
// def process_withdrawal_request(
//
// state: BeaconState,
// execution_layer_withdrawal_request: ExecutionLayerWithdrawalRequest
// withdrawal_request: WithdrawalRequest
//
// ) -> None:
// amount = execution_layer_withdrawal_request.amount
@@ -86,8 +86,8 @@ import (
// amount=to_withdraw,
// withdrawable_epoch=withdrawable_epoch,
// ))
func ProcessExecutionLayerWithdrawalRequests(ctx context.Context, st state.BeaconState, wrs []*enginev1.ExecutionLayerWithdrawalRequest) (state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "electra.ProcessExecutionLayerWithdrawalRequests")
func ProcessWithdrawalRequests(ctx context.Context, st state.BeaconState, wrs []*enginev1.WithdrawalRequest) (state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "electra.ProcessWithdrawalRequests")
defer span.End()
currentEpoch := slots.ToEpoch(st.Slot())
for _, wr := range wrs {

View File

@@ -19,7 +19,7 @@ import (
"github.com/sirupsen/logrus/hooks/test"
)
func TestProcessExecutionLayerWithdrawRequests(t *testing.T) {
func TestProcessWithdrawRequests(t *testing.T) {
logHook := test.NewGlobal()
source, err := hexutil.Decode("0xb20a608c624Ca5003905aA834De7156C68b2E1d0")
require.NoError(t, err)
@@ -30,7 +30,7 @@ func TestProcessExecutionLayerWithdrawRequests(t *testing.T) {
require.NoError(t, err)
type args struct {
st state.BeaconState
wrs []*enginev1.ExecutionLayerWithdrawalRequest
wrs []*enginev1.WithdrawalRequest
}
tests := []struct {
name string
@@ -56,7 +56,7 @@ func TestProcessExecutionLayerWithdrawRequests(t *testing.T) {
require.NoError(t, preSt.SetValidators([]*eth.Validator{v}))
return preSt
}(),
wrs: []*enginev1.ExecutionLayerWithdrawalRequest{
wrs: []*enginev1.WithdrawalRequest{
{
SourceAddress: source,
ValidatorPubkey: bytesutil.SafeCopyBytes(val.PublicKey),
@@ -121,7 +121,7 @@ func TestProcessExecutionLayerWithdrawRequests(t *testing.T) {
}))
return preSt
}(),
wrs: []*enginev1.ExecutionLayerWithdrawalRequest{
wrs: []*enginev1.WithdrawalRequest{
{
SourceAddress: source,
ValidatorPubkey: bytesutil.SafeCopyBytes(val.PublicKey),
@@ -190,7 +190,7 @@ func TestProcessExecutionLayerWithdrawRequests(t *testing.T) {
require.NoError(t, preSt.SetValidators([]*eth.Validator{v}))
return preSt
}(),
wrs: []*enginev1.ExecutionLayerWithdrawalRequest{
wrs: []*enginev1.WithdrawalRequest{
{
SourceAddress: source,
ValidatorPubkey: bytesutil.SafeCopyBytes(val.PublicKey),
@@ -227,7 +227,7 @@ func TestProcessExecutionLayerWithdrawRequests(t *testing.T) {
require.NoError(t, preSt.SetValidators([]*eth.Validator{v}))
return preSt
}(),
wrs: []*enginev1.ExecutionLayerWithdrawalRequest{
wrs: []*enginev1.WithdrawalRequest{
{
SourceAddress: source,
ValidatorPubkey: bytesutil.SafeCopyBytes(val.PublicKey),
@@ -266,7 +266,7 @@ func TestProcessExecutionLayerWithdrawRequests(t *testing.T) {
}))
return preSt
}(),
wrs: []*enginev1.ExecutionLayerWithdrawalRequest{
wrs: []*enginev1.WithdrawalRequest{
{
SourceAddress: source,
ValidatorPubkey: bytesutil.SafeCopyBytes(val.PublicKey),
@@ -284,9 +284,9 @@ func TestProcessExecutionLayerWithdrawRequests(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := electra.ProcessExecutionLayerWithdrawalRequests(context.Background(), tt.args.st, tt.args.wrs)
got, err := electra.ProcessWithdrawalRequests(context.Background(), tt.args.st, tt.args.wrs)
if (err != nil) != tt.wantErr {
t.Errorf("ProcessExecutionLayerWithdrawalRequests() error = %v, wantErr %v", err, tt.wantErr)
t.Errorf("ProcessWithdrawalRequests() error = %v, wantErr %v", err, tt.wantErr)
return
}
tt.wantFn(t, got)

View File

@@ -227,7 +227,7 @@ func ProcessBlockNoVerifyAnySig(
// def process_operations(state: BeaconState, body: BeaconBlockBody) -> None:
// # [Modified in Electra:EIP6110]
// # Disable former deposit mechanism once all prior deposits are processed
// eth1_deposit_index_limit = min(state.eth1_data.deposit_count, state.deposit_receipts_start_index)
// eth1_deposit_index_limit = min(state.eth1_data.deposit_count, state.deposit_requests_start_index)
// if state.eth1_deposit_index < eth1_deposit_index_limit:
// assert len(body.deposits) == min(MAX_DEPOSITS, eth1_deposit_index_limit - state.eth1_deposit_index)
// else:
@@ -245,7 +245,7 @@ func ProcessBlockNoVerifyAnySig(
// for_ops(body.bls_to_execution_changes, process_bls_to_execution_change)
// # [New in Electra:EIP7002:EIP7251]
// for_ops(body.execution_payload.withdrawal_requests, process_execution_layer_withdrawal_request)
// for_ops(body.execution_payload.deposit_receipts, process_deposit_receipt) # [New in Electra:EIP6110]
// for_ops(body.execution_payload.deposit_requests, process_deposit_requests) # [New in Electra:EIP6110]
// for_ops(body.consolidations, process_consolidation) # [New in Electra:EIP7251]
func ProcessOperationsNoVerifyAttsSigs(
ctx context.Context,
@@ -401,7 +401,7 @@ func VerifyBlobCommitmentCount(blk interfaces.ReadOnlyBeaconBlock) error {
// def process_operations(state: BeaconState, body: BeaconBlockBody) -> None:
// # [Modified in Electra:EIP6110]
// # Disable former deposit mechanism once all prior deposits are processed
// eth1_deposit_index_limit = min(state.eth1_data.deposit_count, state.deposit_receipts_start_index)
// eth1_deposit_index_limit = min(state.eth1_data.deposit_count, state.deposit_requests_start_index)
// if state.eth1_deposit_index < eth1_deposit_index_limit:
// assert len(body.deposits) == min(MAX_DEPOSITS, eth1_deposit_index_limit - state.eth1_deposit_index)
// else:
@@ -419,7 +419,7 @@ func VerifyBlobCommitmentCount(blk interfaces.ReadOnlyBeaconBlock) error {
// for_ops(body.bls_to_execution_changes, process_bls_to_execution_change)
// # [New in Electra:EIP7002:EIP7251]
// for_ops(body.execution_payload.withdrawal_requests, process_execution_layer_withdrawal_request)
// for_ops(body.execution_payload.deposit_receipts, process_deposit_receipt) # [New in Electra:EIP6110]
// for_ops(body.execution_payload.deposit_requests, process_deposit_requests) # [New in Electra:EIP6110]
// for_ops(body.consolidations, process_consolidation) # [New in Electra:EIP7251]
func electraOperations(
ctx context.Context,
@@ -445,12 +445,12 @@ func electraOperations(
if !ok {
return nil, errors.New("could not cast execution data to electra execution data")
}
st, err = electra.ProcessExecutionLayerWithdrawalRequests(ctx, st, exe.WithdrawalRequests())
st, err = electra.ProcessWithdrawalRequests(ctx, st, exe.WithdrawalRequests())
if err != nil {
return nil, errors.Wrap(err, "could not process execution layer withdrawal requests")
}
st, err = electra.ProcessDepositReceipts(ctx, st, exe.DepositReceipts()) // TODO: EIP-6110 deposit changes.
st, err = electra.ProcessDepositRequests(ctx, st, exe.DepositRequests()) // TODO: EIP-6110 deposit changes.
if err != nil {
return nil, errors.Wrap(err, "could not process deposit receipts")
}

View File

@@ -149,7 +149,7 @@ func TestState_CanSaveRetrieve(t *testing.T) {
BlockHash: make([]byte, 32),
TransactionsRoot: make([]byte, 32),
WithdrawalsRoot: make([]byte, 32),
DepositReceiptsRoot: make([]byte, 32),
DepositRequestsRoot: make([]byte, 32),
WithdrawalRequestsRoot: make([]byte, 32),
})
require.NoError(t, err)

View File

@@ -631,7 +631,7 @@ func fullPayloadFromPayloadBody(
Withdrawals: body.Withdrawals,
ExcessBlobGas: ebg,
BlobGasUsed: bgu,
DepositReceipts: dr,
DepositRequests: dr,
WithdrawalRequests: wr,
}) // We can't get the block value and don't care about the block value for this instance
default:
@@ -780,8 +780,8 @@ func buildEmptyExecutionPayload(v int) (proto.Message, error) {
BlockHash: make([]byte, fieldparams.RootLength),
Transactions: make([][]byte, 0),
Withdrawals: make([]*pb.Withdrawal, 0),
WithdrawalRequests: make([]*pb.ExecutionLayerWithdrawalRequest, 0),
DepositReceipts: make([]*pb.DepositReceipt, 0),
WithdrawalRequests: make([]*pb.WithdrawalRequest, 0),
DepositRequests: make([]*pb.DepositRequest, 0),
}, nil
default:
return nil, errors.Wrapf(ErrUnsupportedVersion, "version=%s", version.String(v))

View File

@@ -1559,7 +1559,7 @@ func fixturesStruct() *payloadFixtures {
Withdrawals: []*pb.Withdrawal{},
BlobGasUsed: 2,
ExcessBlobGas: 3,
DepositReceipts: dr,
DepositRequests: dr,
WithdrawalRequests: wr,
}
hexUint := hexutil.Uint64(1)

View File

@@ -66,7 +66,7 @@ func payloadToBody(t *testing.T, ed interfaces.ExecutionData) *pb.ExecutionPaylo
}
eed, isElectra := ed.(interfaces.ExecutionDataElectra)
if isElectra {
body.DepositRequests = pb.ProtoDepositRequestsToJson(eed.DepositReceipts())
body.DepositRequests = pb.ProtoDepositRequestsToJson(eed.DepositRequests())
body.WithdrawalRequests = pb.ProtoWithdrawalRequestsToJson(eed.WithdrawalRequests())
}
return body

View File

@@ -149,7 +149,8 @@ func TestGetSpec(t *testing.T) {
config.MaxAttestationsElectra = 89
config.MaxWithdrawalRequestsPerPayload = 90
config.MaxCellsInExtendedMatrix = 91
config.UnsetDepositReceiptsStartIndex = 92
config.UnsetDepositRequestsStartIndex = 92
config.MaxDepositRequestsPerPayload = 93
var dbp [4]byte
copy(dbp[:], []byte{'0', '0', '0', '1'})
@@ -192,7 +193,7 @@ func TestGetSpec(t *testing.T) {
data, ok := resp.Data.(map[string]interface{})
require.Equal(t, true, ok)
assert.Equal(t, 154, len(data))
assert.Equal(t, 155, len(data))
for k, v := range data {
t.Run(k, func(t *testing.T) {
switch k {
@@ -525,8 +526,10 @@ func TestGetSpec(t *testing.T) {
assert.Equal(t, "90", v)
case "MAX_CELLS_IN_EXTENDED_MATRIX":
assert.Equal(t, "91", v)
case "UNSET_DEPOSIT_RECEIPTS_START_INDEX":
case "UNSET_DEPOSIT_REQUESTS_START_INDEX":
assert.Equal(t, "92", v)
case "MAX_DEPOSIT_REQUESTS_PER_PAYLOAD":
assert.Equal(t, "93", v)
default:
t.Errorf("Incorrect key: %s", k)
}

View File

@@ -630,14 +630,14 @@ func TestServer_GetBeaconBlock_Electra(t *testing.T) {
require.NoError(t, err)
timeStamp, err := slots.ToTime(beaconState.GenesisTime(), electraSlot+1)
require.NoError(t, err)
dr := []*enginev1.DepositReceipt{{
dr := []*enginev1.DepositRequest{{
Pubkey: bytesutil.PadTo(privKeys[0].PublicKey().Marshal(), 48),
WithdrawalCredentials: bytesutil.PadTo([]byte("wc"), 32),
Amount: 123,
Signature: bytesutil.PadTo([]byte("sig"), 96),
Index: 456,
}}
wr := []*enginev1.ExecutionLayerWithdrawalRequest{
wr := []*enginev1.WithdrawalRequest{
{
SourceAddress: bytesutil.PadTo([]byte("sa"), 20),
ValidatorPubkey: bytesutil.PadTo(privKeys[1].PublicKey().Marshal(), 48),
@@ -654,7 +654,7 @@ func TestServer_GetBeaconBlock_Electra(t *testing.T) {
PrevRandao: random,
BaseFeePerGas: make([]byte, fieldparams.RootLength),
BlockHash: make([]byte, fieldparams.RootLength),
DepositReceipts: dr,
DepositRequests: dr,
WithdrawalRequests: wr,
}
@@ -680,7 +680,7 @@ func TestServer_GetBeaconBlock_Electra(t *testing.T) {
got, err := proposerServer.GetBeaconBlock(ctx, req)
require.NoError(t, err)
p := got.GetElectra().Block.Body.ExecutionPayload
require.DeepEqual(t, dr, p.DepositReceipts)
require.DeepEqual(t, dr, p.DepositRequests)
require.DeepEqual(t, wr, p.WithdrawalRequests)
}

View File

@@ -219,7 +219,7 @@ type ReadOnlySyncCommittee interface {
type ReadOnlyDeposits interface {
DepositBalanceToConsume() (primitives.Gwei, error)
DepositReceiptsStartIndex() (uint64, error)
DepositRequestsStartIndex() (uint64, error)
PendingBalanceDeposits() ([]*ethpb.PendingBalanceDeposit, error)
}
@@ -327,7 +327,7 @@ type WriteOnlyConsolidations interface {
type WriteOnlyDeposits interface {
AppendPendingBalanceDeposit(index primitives.ValidatorIndex, amount uint64) error
SetDepositReceiptsStartIndex(index uint64) error
SetDepositRequestsStartIndex(index uint64) error
SetPendingBalanceDeposits(val []*ethpb.PendingBalanceDeposit) error
SetDepositBalanceToConsume(primitives.Gwei) error
}

View File

@@ -12,7 +12,7 @@ go_library(
"getters_block.go",
"getters_checkpoint.go",
"getters_consolidation.go",
"getters_deposit_receipts.go",
"getters_deposit_requests.go",
"getters_eth1.go",
"getters_exit.go",
"getters_misc.go",
@@ -33,7 +33,7 @@ go_library(
"setters_checkpoint.go",
"setters_churn.go",
"setters_consolidation.go",
"setters_deposit_receipts.go",
"setters_deposit_requests.go",
"setters_eth1.go",
"setters_misc.go",
"setters_participation.go",
@@ -94,7 +94,7 @@ go_test(
"getters_block_test.go",
"getters_checkpoint_test.go",
"getters_consolidation_test.go",
"getters_deposit_receipts_test.go",
"getters_deposit_requests_test.go",
"getters_exit_test.go",
"getters_participation_test.go",
"getters_test.go",
@@ -109,7 +109,7 @@ go_test(
"setters_balance_deposits_test.go",
"setters_churn_test.go",
"setters_consolidation_test.go",
"setters_deposit_receipts_test.go",
"setters_deposit_requests_test.go",
"setters_eth1_test.go",
"setters_misc_test.go",
"setters_participation_test.go",

View File

@@ -62,7 +62,7 @@ type BeaconState struct {
nextWithdrawalValidatorIndex primitives.ValidatorIndex
// Electra fields
depositReceiptsStartIndex uint64
depositRequestsStartIndex uint64
depositBalanceToConsume primitives.Gwei
exitBalanceToConsume primitives.Gwei
earliestExitEpoch primitives.Epoch
@@ -119,7 +119,7 @@ type beaconStateMarshalable struct {
LatestExecutionPayloadHeaderElectra *enginev1.ExecutionPayloadHeaderElectra `json:"latest_execution_payload_header_electra" yaml:"latest_execution_payload_header_electra"`
NextWithdrawalIndex uint64 `json:"next_withdrawal_index" yaml:"next_withdrawal_index"`
NextWithdrawalValidatorIndex primitives.ValidatorIndex `json:"next_withdrawal_validator_index" yaml:"next_withdrawal_validator_index"`
DepositReceiptsStartIndex uint64 `json:"deposit_receipts_start_index" yaml:"deposit_receipts_start_index"`
DepositRequestsStartIndex uint64 `json:"deposit_requests_start_index" yaml:"deposit_requests_start_index"`
DepositBalanceToConsume primitives.Gwei `json:"deposit_balance_to_consume" yaml:"deposit_balance_to_consume"`
ExitBalanceToConsume primitives.Gwei `json:"exit_balance_to_consume" yaml:"exit_balance_to_consume"`
EarliestExitEpoch primitives.Epoch `json:"earliest_exit_epoch" yaml:"earliest_exit_epoch"`
@@ -189,7 +189,7 @@ func (b *BeaconState) MarshalJSON() ([]byte, error) {
LatestExecutionPayloadHeaderElectra: b.latestExecutionPayloadHeaderElectra,
NextWithdrawalIndex: b.nextWithdrawalIndex,
NextWithdrawalValidatorIndex: b.nextWithdrawalValidatorIndex,
DepositReceiptsStartIndex: b.depositReceiptsStartIndex,
DepositRequestsStartIndex: b.depositRequestsStartIndex,
DepositBalanceToConsume: b.depositBalanceToConsume,
ExitBalanceToConsume: b.exitBalanceToConsume,
EarliestExitEpoch: b.earliestExitEpoch,

View File

@@ -62,7 +62,7 @@ type BeaconState struct {
nextWithdrawalValidatorIndex primitives.ValidatorIndex
// Electra fields
depositReceiptsStartIndex uint64
depositRequestsStartIndex uint64
depositBalanceToConsume primitives.Gwei
exitBalanceToConsume primitives.Gwei
earliestExitEpoch primitives.Epoch
@@ -119,7 +119,7 @@ type beaconStateMarshalable struct {
LatestExecutionPayloadHeaderElectra *enginev1.ExecutionPayloadHeaderElectra `json:"latest_execution_payload_header_electra" yaml:"latest_execution_payload_header_electra"`
NextWithdrawalIndex uint64 `json:"next_withdrawal_index" yaml:"next_withdrawal_index"`
NextWithdrawalValidatorIndex primitives.ValidatorIndex `json:"next_withdrawal_validator_index" yaml:"next_withdrawal_validator_index"`
DepositReceiptsStartIndex uint64 `json:"deposit_receipts_start_index" yaml:"deposit_receipts_start_index"`
DepositRequestsStartIndex uint64 `json:"deposit_requests_start_index" yaml:"deposit_requests_start_index"`
DepositBalanceToConsume primitives.Gwei `json:"deposit_balance_to_consume" yaml:"deposit_balance_to_consume"`
ExitBalanceToConsume primitives.Gwei `json:"exit_balance_to_consume" yaml:"exit_balance_to_consume"`
EarliestExitEpoch primitives.Epoch `json:"earliest_exit_epoch" yaml:"earliest_exit_epoch"`
@@ -189,7 +189,7 @@ func (b *BeaconState) MarshalJSON() ([]byte, error) {
LatestExecutionPayloadHeaderElectra: b.latestExecutionPayloadHeaderElectra,
NextWithdrawalIndex: b.nextWithdrawalIndex,
NextWithdrawalValidatorIndex: b.nextWithdrawalValidatorIndex,
DepositReceiptsStartIndex: b.depositReceiptsStartIndex,
DepositRequestsStartIndex: b.depositRequestsStartIndex,
DepositBalanceToConsume: b.depositBalanceToConsume,
ExitBalanceToConsume: b.exitBalanceToConsume,
EarliestExitEpoch: b.earliestExitEpoch,

View File

@@ -1,16 +0,0 @@
package state_native
import (
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
// DepositReceiptsStartIndex is used for returning the deposit receipts start index which is used for eip6110
func (b *BeaconState) DepositReceiptsStartIndex() (uint64, error) {
if b.version < version.Electra {
return 0, errNotSupported("DepositReceiptsStartIndex", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
return b.depositReceiptsStartIndex, nil
}

View File

@@ -0,0 +1,16 @@
package state_native
import (
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
// DepositRequestsStartIndex is used for returning the deposit receipts start index which is used for eip6110
func (b *BeaconState) DepositRequestsStartIndex() (uint64, error) {
if b.version < version.Electra {
return 0, errNotSupported("DepositRequestsStartIndex", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
return b.depositRequestsStartIndex, nil
}

View File

@@ -9,17 +9,17 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/util"
)
func TestDepositReceiptsStartIndex(t *testing.T) {
func TestDepositRequestsStartIndex(t *testing.T) {
t.Run("previous fork returns expected error", func(t *testing.T) {
dState, _ := util.DeterministicGenesisState(t, 1)
_, err := dState.DepositReceiptsStartIndex()
_, err := dState.DepositRequestsStartIndex()
require.ErrorContains(t, "is not supported", err)
})
t.Run("electra returns expected value", func(t *testing.T) {
want := uint64(2)
dState, err := state_native.InitializeFromProtoElectra(&ethpb.BeaconStateElectra{DepositReceiptsStartIndex: want})
dState, err := state_native.InitializeFromProtoElectra(&ethpb.BeaconStateElectra{DepositRequestsStartIndex: want})
require.NoError(t, err)
got, err := dState.DepositReceiptsStartIndex()
got, err := dState.DepositRequestsStartIndex()
require.NoError(t, err)
require.Equal(t, want, got)
})

View File

@@ -202,7 +202,7 @@ func (b *BeaconState) ToProtoUnsafe() interface{} {
NextWithdrawalIndex: b.nextWithdrawalIndex,
NextWithdrawalValidatorIndex: b.nextWithdrawalValidatorIndex,
HistoricalSummaries: b.historicalSummaries,
DepositReceiptsStartIndex: b.depositReceiptsStartIndex,
DepositRequestsStartIndex: b.depositRequestsStartIndex,
DepositBalanceToConsume: b.depositBalanceToConsume,
ExitBalanceToConsume: b.exitBalanceToConsume,
EarliestExitEpoch: b.earliestExitEpoch,
@@ -408,7 +408,7 @@ func (b *BeaconState) ToProto() interface{} {
NextWithdrawalIndex: b.nextWithdrawalIndex,
NextWithdrawalValidatorIndex: b.nextWithdrawalValidatorIndex,
HistoricalSummaries: b.historicalSummariesVal(),
DepositReceiptsStartIndex: b.depositReceiptsStartIndex,
DepositRequestsStartIndex: b.depositRequestsStartIndex,
DepositBalanceToConsume: b.depositBalanceToConsume,
ExitBalanceToConsume: b.exitBalanceToConsume,
EarliestExitEpoch: b.earliestExitEpoch,

View File

@@ -281,9 +281,9 @@ func ComputeFieldRootsWithHasher(ctx context.Context, state *BeaconState) ([][]b
}
if state.version >= version.Electra {
// DepositReceiptsStartIndex root.
drsiRoot := ssz.Uint64Root(state.depositReceiptsStartIndex)
fieldRoots[types.DepositReceiptsStartIndex.RealPosition()] = drsiRoot[:]
// DepositRequestsStartIndex root.
drsiRoot := ssz.Uint64Root(state.depositRequestsStartIndex)
fieldRoots[types.DepositRequestsStartIndex.RealPosition()] = drsiRoot[:]
// DepositBalanceToConsume root.
dbtcRoot := ssz.Uint64Root(uint64(state.depositBalanceToConsume))

View File

@@ -1,21 +0,0 @@
package state_native
import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native/types"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
// SetDepositReceiptsStartIndex for the beacon state. Updates the DepositReceiptsStartIndex
func (b *BeaconState) SetDepositReceiptsStartIndex(index uint64) error {
if b.version < version.Electra {
return errNotSupported("SetDepositReceiptsStartIndex", b.version)
}
b.lock.Lock()
defer b.lock.Unlock()
b.depositReceiptsStartIndex = index
b.markFieldAsDirty(types.DepositReceiptsStartIndex)
b.rebuildTrie[types.DepositReceiptsStartIndex] = true
return nil
}

View File

@@ -0,0 +1,21 @@
package state_native
import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native/types"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
// SetDepositRequestsStartIndex for the beacon state. Updates the DepositRequestsStartIndex
func (b *BeaconState) SetDepositRequestsStartIndex(index uint64) error {
if b.version < version.Electra {
return errNotSupported("SetDepositRequestsStartIndex", b.version)
}
b.lock.Lock()
defer b.lock.Unlock()
b.depositRequestsStartIndex = index
b.markFieldAsDirty(types.DepositRequestsStartIndex)
b.rebuildTrie[types.DepositRequestsStartIndex] = true
return nil
}

View File

@@ -9,18 +9,18 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/util"
)
func TestSetDepositReceiptsStartIndex(t *testing.T) {
func TestSetDepositRequestsStartIndex(t *testing.T) {
t.Run("previous fork returns expected error", func(t *testing.T) {
dState, _ := util.DeterministicGenesisState(t, 1)
require.ErrorContains(t, "is not supported", dState.SetDepositReceiptsStartIndex(1))
require.ErrorContains(t, "is not supported", dState.SetDepositRequestsStartIndex(1))
})
t.Run("electra sets expected value", func(t *testing.T) {
old := uint64(2)
dState, err := state_native.InitializeFromProtoElectra(&ethpb.BeaconStateElectra{DepositReceiptsStartIndex: old})
dState, err := state_native.InitializeFromProtoElectra(&ethpb.BeaconStateElectra{DepositRequestsStartIndex: old})
require.NoError(t, err)
want := uint64(3)
require.NoError(t, dState.SetDepositReceiptsStartIndex(want))
got, err := dState.DepositReceiptsStartIndex()
require.NoError(t, dState.SetDepositRequestsStartIndex(want))
got, err := dState.DepositRequestsStartIndex()
require.NoError(t, err)
require.Equal(t, want, got)
})

View File

@@ -100,7 +100,7 @@ var electraFields = append(
types.NextWithdrawalValidatorIndex,
types.HistoricalSummaries,
types.LatestExecutionPayloadHeaderElectra,
types.DepositReceiptsStartIndex,
types.DepositRequestsStartIndex,
types.DepositBalanceToConsume,
types.ExitBalanceToConsume,
types.EarliestExitEpoch,
@@ -744,7 +744,7 @@ func InitializeFromProtoUnsafeElectra(st *ethpb.BeaconStateElectra) (state.Beaco
nextWithdrawalIndex: st.NextWithdrawalIndex,
nextWithdrawalValidatorIndex: st.NextWithdrawalValidatorIndex,
historicalSummaries: st.HistoricalSummaries,
depositReceiptsStartIndex: st.DepositReceiptsStartIndex,
depositRequestsStartIndex: st.DepositRequestsStartIndex,
depositBalanceToConsume: st.DepositBalanceToConsume,
exitBalanceToConsume: st.ExitBalanceToConsume,
earliestExitEpoch: st.EarliestExitEpoch,
@@ -862,7 +862,7 @@ func (b *BeaconState) Copy() state.BeaconState {
eth1DepositIndex: b.eth1DepositIndex,
nextWithdrawalIndex: b.nextWithdrawalIndex,
nextWithdrawalValidatorIndex: b.nextWithdrawalValidatorIndex,
depositReceiptsStartIndex: b.depositReceiptsStartIndex,
depositRequestsStartIndex: b.depositRequestsStartIndex,
depositBalanceToConsume: b.depositBalanceToConsume,
exitBalanceToConsume: b.exitBalanceToConsume,
earliestExitEpoch: b.earliestExitEpoch,
@@ -1286,8 +1286,8 @@ func (b *BeaconState) rootSelector(ctx context.Context, field types.FieldIndex)
return ssz.Uint64Root(uint64(b.nextWithdrawalValidatorIndex)), nil
case types.HistoricalSummaries:
return stateutil.HistoricalSummariesRoot(b.historicalSummaries)
case types.DepositReceiptsStartIndex:
return ssz.Uint64Root(b.depositReceiptsStartIndex), nil
case types.DepositRequestsStartIndex:
return ssz.Uint64Root(b.depositRequestsStartIndex), nil
case types.DepositBalanceToConsume:
return ssz.Uint64Root(uint64(b.depositBalanceToConsume)), nil
case types.ExitBalanceToConsume:

View File

@@ -96,8 +96,8 @@ func (f FieldIndex) String() string {
return "nextWithdrawalValidatorIndex"
case HistoricalSummaries:
return "historicalSummaries"
case DepositReceiptsStartIndex:
return "depositReceiptsStartIndex"
case DepositRequestsStartIndex:
return "depositRequestsStartIndex"
case DepositBalanceToConsume:
return "depositBalanceToConsume"
case ExitBalanceToConsume:
@@ -179,7 +179,7 @@ func (f FieldIndex) RealPosition() int {
return 26
case HistoricalSummaries:
return 27
case DepositReceiptsStartIndex:
case DepositRequestsStartIndex:
return 28
case DepositBalanceToConsume:
return 29
@@ -253,7 +253,7 @@ const (
NextWithdrawalIndex
NextWithdrawalValidatorIndex
HistoricalSummaries
DepositReceiptsStartIndex // Electra: EIP-6110
DepositRequestsStartIndex // Electra: EIP-6110
DepositBalanceToConsume // Electra: EIP-7251
ExitBalanceToConsume // Electra: EIP-7251
EarliestExitEpoch // Electra: EIP-7251

0
cmd/prysmctl/tosaccepted Normal file
View File

View File

@@ -36,6 +36,6 @@ const (
PendingBalanceDepositsLimit = 134217728 // Maximum number of pending balance deposits in the beacon state.
PendingPartialWithdrawalsLimit = 134217728 // Maximum number of pending partial withdrawals in the beacon state.
PendingConsolidationsLimit = 262144 // Maximum number of pending consolidations in the beacon state.
MaxDepositReceiptsPerPayload = 8192 // Maximum number of deposit receipts in an execution payload.
MaxDepositRequestsPerPayload = 8192 // Maximum number of deposit requests in an execution payload.
MaxWithdrawalRequestsPerPayload = 16 // Maximum number of execution layer withdrawal requests in an execution payload.
)

View File

@@ -36,6 +36,6 @@ const (
PendingBalanceDepositsLimit = 134217728 // Maximum number of pending balance deposits in the beacon state.
PendingPartialWithdrawalsLimit = 64 // Maximum number of pending partial withdrawals in the beacon state.
PendingConsolidationsLimit = 64 // Maximum number of pending consolidations in the beacon state.
MaxDepositReceiptsPerPayload = 4 // Maximum number of deposit receipts in an execution payload.
MaxDepositRequestsPerPayload = 4 // Maximum number of deposit requests in an execution payload.
MaxWithdrawalRequestsPerPayload = 2 // Maximum number of execution layer withdrawal requests in an execution payload.
)

View File

@@ -250,7 +250,8 @@ type BeaconChainConfig struct {
MaxPendingPartialsPerWithdrawalsSweep uint64 `yaml:"MAX_PENDING_PARTIALS_PER_WITHDRAWALS_SWEEP" spec:"true"` // MaxPendingPartialsPerWithdrawalsSweep is the maximum number of pending partial withdrawals to process per payload.
FullExitRequestAmount uint64 `yaml:"FULL_EXIT_REQUEST_AMOUNT" spec:"true"` // FullExitRequestAmount is the amount of Gwei required to request a full exit.
MaxWithdrawalRequestsPerPayload uint64 `yaml:"MAX_WITHDRAWAL_REQUESTS_PER_PAYLOAD" spec:"true"` // MaxWithdrawalRequestsPerPayload is the maximum number of execution layer withdrawal requests in each payload.
UnsetDepositReceiptsStartIndex uint64 `yaml:"UNSET_DEPOSIT_RECEIPTS_START_INDEX" spec:"true"` // UnsetDepositReceiptsStartIndex is used to check the start index for eip6110
MaxDepositRequestsPerPayload uint64 `yaml:"MAX_DEPOSIT_REQUESTS_PER_PAYLOAD" spec:"true"` // MaxDepositRequestsPerPayload is the maximum number of execution layer deposits in each payload
UnsetDepositRequestsStartIndex uint64 `yaml:"UNSET_DEPOSIT_REQUESTS_START_INDEX" spec:"true"` // UnsetDepositRequestsStartIndex is used to check the start index for eip6110
// Networking Specific Parameters
GossipMaxSize uint64 `yaml:"GOSSIP_MAX_SIZE" spec:"true"` // GossipMaxSize is the maximum allowed size of uncompressed gossip messages.

View File

@@ -36,7 +36,7 @@ var placeholderFields = []string{
"MAX_BLOBS_PER_BLOCK",
"MAX_BLOB_COMMITMENTS_PER_BLOCK", // Compile time constant on BeaconBlockBodyDeneb.blob_kzg_commitments.
"MAX_BYTES_PER_TRANSACTION", // Used for ssz of EL transactions. Unused in Prysm.
"MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD", // Compile time constant on ExecutionPayload.deposit_receipts.
"MAX_DEPOSIT_RECEIPTS_PER_PAYLOAD", // Compile time constant on ExecutionPayload.deposit_receipts. TODO: rename when updating spec configs
"MAX_EXTRA_DATA_BYTES", // Compile time constant on ExecutionPayload.extra_data.
"MAX_TRANSACTIONS_PER_PAYLOAD", // Compile time constant on ExecutionPayload.transactions.
"REORG_HEAD_WEIGHT_THRESHOLD",

View File

@@ -290,7 +290,8 @@ var mainnetBeaconConfig = &BeaconChainConfig{
MaxPendingPartialsPerWithdrawalsSweep: 8,
FullExitRequestAmount: 0,
MaxWithdrawalRequestsPerPayload: 16,
UnsetDepositReceiptsStartIndex: math.MaxUint64,
MaxDepositRequestsPerPayload: 8192, // 2**13 (= 8192)
UnsetDepositRequestsStartIndex: math.MaxUint64,
// Values related to networking parameters.
GossipMaxSize: 10 * 1 << 20, // 10 MiB

View File

@@ -107,6 +107,7 @@ func MinimalSpecConfig() *BeaconChainConfig {
minimalConfig.PendingConsolidationsLimit = 64
minimalConfig.MaxPartialWithdrawalsPerPayload = 1
minimalConfig.MaxWithdrawalRequestsPerPayload = 2
minimalConfig.MaxDepositRequestsPerPayload = 4
minimalConfig.PendingPartialWithdrawalsLimit = 64
minimalConfig.MaxPendingPartialsPerWithdrawalsSweep = 1

View File

@@ -812,14 +812,14 @@ func PayloadToHeaderElectra(payload interfaces.ExecutionDataElectra) (*enginev1.
return nil, err
}
depositReceipts := payload.DepositReceipts()
depositReceiptsRoot, err := ssz.DepositReceiptSliceRoot(depositReceipts, fieldparams.MaxDepositReceiptsPerPayload)
depositRequests := payload.DepositRequests()
depositRequestsRoot, err := ssz.DepositRequestsSliceRoot(depositRequests, fieldparams.MaxDepositRequestsPerPayload)
if err != nil {
return nil, err
}
withdrawalRequests := payload.WithdrawalRequests()
withdrawalRequestsRoot, err := ssz.WithdrawalRequestSliceRoot(withdrawalRequests, fieldparams.MaxWithdrawalRequestsPerPayload)
withdrawalRequestsRoot, err := ssz.WithdrawalRequestsSliceRoot(withdrawalRequests, fieldparams.MaxWithdrawalRequestsPerPayload)
if err != nil {
return nil, err
}
@@ -842,7 +842,7 @@ func PayloadToHeaderElectra(payload interfaces.ExecutionDataElectra) (*enginev1.
WithdrawalsRoot: withdrawalsRoot[:],
BlobGasUsed: blobGasUsed,
ExcessBlobGas: excessBlobGas,
DepositReceiptsRoot: depositReceiptsRoot[:],
DepositRequestsRoot: depositRequestsRoot[:],
WithdrawalRequestsRoot: withdrawalRequestsRoot[:],
}, nil
}
@@ -907,7 +907,7 @@ func IsEmptyExecutionData(data interfaces.ExecutionData) (bool, error) {
epe, postElectra := data.(interfaces.ExecutionDataElectra)
if postElectra {
drs := epe.DepositReceipts()
drs := epe.DepositRequests()
if len(drs) != 0 {
return false, nil
}
@@ -1389,13 +1389,13 @@ func (e executionPayloadHeaderElectra) ExcessBlobGas() (uint64, error) {
return e.p.ExcessBlobGas, nil
}
// DepositReceipts --
func (e executionPayloadHeaderElectra) DepositReceipts() ([]*enginev1.DepositReceipt, error) {
// DepositRequests --
func (e executionPayloadHeaderElectra) DepositRequests() ([]*enginev1.DepositRequest, error) {
return nil, consensus_types.ErrUnsupportedField
}
// WithdrawalRequests --
func (e executionPayloadHeaderElectra) WithdrawalRequests() ([]*enginev1.ExecutionLayerWithdrawalRequest, error) {
func (e executionPayloadHeaderElectra) WithdrawalRequests() ([]*enginev1.WithdrawalRequest, error) {
return nil, consensus_types.ErrUnsupportedField
}
@@ -1556,13 +1556,13 @@ func (e executionPayloadElectra) ExcessBlobGas() (uint64, error) {
return e.p.ExcessBlobGas, nil
}
// DepositReceipts --
func (e executionPayloadElectra) DepositReceipts() []*enginev1.DepositReceipt {
return e.p.DepositReceipts
// DepositRequests --
func (e executionPayloadElectra) DepositRequests() []*enginev1.DepositRequest {
return e.p.DepositRequests
}
// WithdrawalRequests --
func (e executionPayloadElectra) WithdrawalRequests() []*enginev1.ExecutionLayerWithdrawalRequest {
func (e executionPayloadElectra) WithdrawalRequests() []*enginev1.WithdrawalRequest {
return e.p.WithdrawalRequests
}

View File

@@ -130,6 +130,6 @@ type ExecutionData interface {
type ExecutionDataElectra interface {
ExecutionData
DepositReceipts() []*enginev1.DepositReceipt
WithdrawalRequests() []*enginev1.ExecutionLayerWithdrawalRequest
DepositRequests() []*enginev1.DepositRequest
WithdrawalRequests() []*enginev1.WithdrawalRequest
}

View File

@@ -141,12 +141,12 @@ func WithdrawalSliceRoot(withdrawals []*enginev1.Withdrawal, limit uint64) ([32]
return MixInLength(bytesRoot, bytesRootBufRoot), nil
}
// DepositReceiptSliceRoot computes the HTR of a slice of deposit receipts.
// DepositRequestsSliceRoot computes the HTR of a slice of deposit receipts.
// The limit parameter is used as input to the bitwise merkleization algorithm.
func DepositReceiptSliceRoot(depositReceipts []*enginev1.DepositReceipt, limit uint64) ([32]byte, error) {
roots := make([][32]byte, len(depositReceipts))
for i := 0; i < len(depositReceipts); i++ {
r, err := depositReceipts[i].HashTreeRoot()
func DepositRequestsSliceRoot(depositRequests []*enginev1.DepositRequest, limit uint64) ([32]byte, error) {
roots := make([][32]byte, len(depositRequests))
for i := 0; i < len(depositRequests); i++ {
r, err := depositRequests[i].HashTreeRoot()
if err != nil {
return [32]byte{}, err
}
@@ -158,7 +158,7 @@ func DepositReceiptSliceRoot(depositReceipts []*enginev1.DepositReceipt, limit u
return [32]byte{}, errors.Wrap(err, "could not compute merkleization")
}
bytesRootBuf := new(bytes.Buffer)
if err := binary.Write(bytesRootBuf, binary.LittleEndian, uint64(len(depositReceipts))); err != nil {
if err := binary.Write(bytesRootBuf, binary.LittleEndian, uint64(len(depositRequests))); err != nil {
return [32]byte{}, errors.Wrap(err, "could not marshal length")
}
bytesRootBufRoot := make([]byte, 32)
@@ -166,9 +166,9 @@ func DepositReceiptSliceRoot(depositReceipts []*enginev1.DepositReceipt, limit u
return MixInLength(bytesRoot, bytesRootBufRoot), nil
}
// WithdrawalRequestSliceRoot computes the HTR of a slice of withdrawal requests from the EL.
// WithdrawalRequestsSliceRoot computes the HTR of a slice of withdrawal requests from the EL.
// The limit parameter is used as input to the bitwise merkleization algorithm.
func WithdrawalRequestSliceRoot(withdrawalRequests []*enginev1.ExecutionLayerWithdrawalRequest, limit uint64) ([32]byte, error) {
func WithdrawalRequestsSliceRoot(withdrawalRequests []*enginev1.WithdrawalRequest, limit uint64) ([32]byte, error) {
roots := make([][32]byte, len(withdrawalRequests))
for i := 0; i < len(withdrawalRequests); i++ {
r, err := withdrawalRequests[i].HashTreeRoot()

View File

@@ -281,21 +281,21 @@ func TestWithrawalSliceRoot(t *testing.T) {
}
}
func TestDepositReceiptSliceRoot(t *testing.T) {
func TestDepositRequestsSliceRoot(t *testing.T) {
tests := []struct {
name string
input []*enginev1.DepositReceipt
input []*enginev1.DepositRequest
limit uint64
want [32]byte
}{
{
name: "empty",
input: make([]*enginev1.DepositReceipt, 0),
input: make([]*enginev1.DepositRequest, 0),
want: [32]byte{0xf5, 0xa5, 0xfd, 0x42, 0xd1, 0x6a, 0x20, 0x30, 0x27, 0x98, 0xef, 0x6e, 0xd3, 0x9, 0x97, 0x9b, 0x43, 0x0, 0x3d, 0x23, 0x20, 0xd9, 0xf0, 0xe8, 0xea, 0x98, 0x31, 0xa9, 0x27, 0x59, 0xfb, 0x4b},
},
{
name: "non-empty",
input: []*enginev1.DepositReceipt{
input: []*enginev1.DepositRequest{
{
Pubkey: bytesutil.PadTo([]byte{0x01, 0x02}, 48),
WithdrawalCredentials: bytesutil.PadTo([]byte{0x03, 0x04}, 32),
@@ -311,7 +311,7 @@ func TestDepositReceiptSliceRoot(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := ssz.DepositReceiptSliceRoot(tt.input, tt.limit)
got, err := ssz.DepositRequestsSliceRoot(tt.input, tt.limit)
require.NoError(t, err)
require.DeepSSZEqual(t, tt.want, got)
})
@@ -321,18 +321,18 @@ func TestDepositReceiptSliceRoot(t *testing.T) {
func TestWithdrawalRequestSliceRoot(t *testing.T) {
tests := []struct {
name string
input []*enginev1.ExecutionLayerWithdrawalRequest
input []*enginev1.WithdrawalRequest
limit uint64
want [32]byte
}{
{
name: "empty",
input: make([]*enginev1.ExecutionLayerWithdrawalRequest, 0),
input: make([]*enginev1.WithdrawalRequest, 0),
want: [32]byte{0xf5, 0xa5, 0xfd, 0x42, 0xd1, 0x6a, 0x20, 0x30, 0x27, 0x98, 0xef, 0x6e, 0xd3, 0x9, 0x97, 0x9b, 0x43, 0x0, 0x3d, 0x23, 0x20, 0xd9, 0xf0, 0xe8, 0xea, 0x98, 0x31, 0xa9, 0x27, 0x59, 0xfb, 0x4b},
},
{
name: "non-empty",
input: []*enginev1.ExecutionLayerWithdrawalRequest{
input: []*enginev1.WithdrawalRequest{
{
SourceAddress: bytesutil.PadTo([]byte{0x01, 0x02}, 20),
ValidatorPubkey: bytesutil.PadTo([]byte{0x03, 0x04}, 48),
@@ -346,7 +346,7 @@ func TestWithdrawalRequestSliceRoot(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got, err := ssz.WithdrawalRequestSliceRoot(tt.input, tt.limit)
got, err := ssz.WithdrawalRequestsSliceRoot(tt.input, tt.limit)
require.NoError(t, err)
require.DeepSSZEqual(t, tt.want, got)
})

View File

@@ -46,8 +46,8 @@ ssz_gen_marshal(
"BlindedBlobsBundle",
"BlobsBundle",
"Withdrawal",
"ExecutionLayerWithdrawalRequest",
"DepositReceipt",
"WithdrawalRequest",
"DepositRequest",
],
)

View File

@@ -571,25 +571,25 @@ type ExecutionPayloadElectra struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
ParentHash []byte `protobuf:"bytes,1,opt,name=parent_hash,json=parentHash,proto3" json:"parent_hash,omitempty" ssz-size:"32"`
FeeRecipient []byte `protobuf:"bytes,2,opt,name=fee_recipient,json=feeRecipient,proto3" json:"fee_recipient,omitempty" ssz-size:"20"`
StateRoot []byte `protobuf:"bytes,3,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty" ssz-size:"32"`
ReceiptsRoot []byte `protobuf:"bytes,4,opt,name=receipts_root,json=receiptsRoot,proto3" json:"receipts_root,omitempty" ssz-size:"32"`
LogsBloom []byte `protobuf:"bytes,5,opt,name=logs_bloom,json=logsBloom,proto3" json:"logs_bloom,omitempty" ssz-size:"256"`
PrevRandao []byte `protobuf:"bytes,6,opt,name=prev_randao,json=prevRandao,proto3" json:"prev_randao,omitempty" ssz-size:"32"`
BlockNumber uint64 `protobuf:"varint,7,opt,name=block_number,json=blockNumber,proto3" json:"block_number,omitempty"`
GasLimit uint64 `protobuf:"varint,8,opt,name=gas_limit,json=gasLimit,proto3" json:"gas_limit,omitempty"`
GasUsed uint64 `protobuf:"varint,9,opt,name=gas_used,json=gasUsed,proto3" json:"gas_used,omitempty"`
Timestamp uint64 `protobuf:"varint,10,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ExtraData []byte `protobuf:"bytes,11,opt,name=extra_data,json=extraData,proto3" json:"extra_data,omitempty" ssz-max:"32"`
BaseFeePerGas []byte `protobuf:"bytes,12,opt,name=base_fee_per_gas,json=baseFeePerGas,proto3" json:"base_fee_per_gas,omitempty" ssz-size:"32"`
BlockHash []byte `protobuf:"bytes,13,opt,name=block_hash,json=blockHash,proto3" json:"block_hash,omitempty" ssz-size:"32"`
Transactions [][]byte `protobuf:"bytes,14,rep,name=transactions,proto3" json:"transactions,omitempty" ssz-max:"1048576,1073741824" ssz-size:"?,?"`
Withdrawals []*Withdrawal `protobuf:"bytes,15,rep,name=withdrawals,proto3" json:"withdrawals,omitempty" ssz-max:"16"`
BlobGasUsed uint64 `protobuf:"varint,16,opt,name=blob_gas_used,json=blobGasUsed,proto3" json:"blob_gas_used,omitempty"`
ExcessBlobGas uint64 `protobuf:"varint,17,opt,name=excess_blob_gas,json=excessBlobGas,proto3" json:"excess_blob_gas,omitempty"`
DepositReceipts []*DepositReceipt `protobuf:"bytes,18,rep,name=deposit_receipts,json=depositReceipts,proto3" json:"deposit_receipts,omitempty" ssz-max:"8192"`
WithdrawalRequests []*ExecutionLayerWithdrawalRequest `protobuf:"bytes,19,rep,name=withdrawal_requests,json=withdrawalRequests,proto3" json:"withdrawal_requests,omitempty" ssz-max:"16"`
ParentHash []byte `protobuf:"bytes,1,opt,name=parent_hash,json=parentHash,proto3" json:"parent_hash,omitempty" ssz-size:"32"`
FeeRecipient []byte `protobuf:"bytes,2,opt,name=fee_recipient,json=feeRecipient,proto3" json:"fee_recipient,omitempty" ssz-size:"20"`
StateRoot []byte `protobuf:"bytes,3,opt,name=state_root,json=stateRoot,proto3" json:"state_root,omitempty" ssz-size:"32"`
ReceiptsRoot []byte `protobuf:"bytes,4,opt,name=receipts_root,json=receiptsRoot,proto3" json:"receipts_root,omitempty" ssz-size:"32"`
LogsBloom []byte `protobuf:"bytes,5,opt,name=logs_bloom,json=logsBloom,proto3" json:"logs_bloom,omitempty" ssz-size:"256"`
PrevRandao []byte `protobuf:"bytes,6,opt,name=prev_randao,json=prevRandao,proto3" json:"prev_randao,omitempty" ssz-size:"32"`
BlockNumber uint64 `protobuf:"varint,7,opt,name=block_number,json=blockNumber,proto3" json:"block_number,omitempty"`
GasLimit uint64 `protobuf:"varint,8,opt,name=gas_limit,json=gasLimit,proto3" json:"gas_limit,omitempty"`
GasUsed uint64 `protobuf:"varint,9,opt,name=gas_used,json=gasUsed,proto3" json:"gas_used,omitempty"`
Timestamp uint64 `protobuf:"varint,10,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ExtraData []byte `protobuf:"bytes,11,opt,name=extra_data,json=extraData,proto3" json:"extra_data,omitempty" ssz-max:"32"`
BaseFeePerGas []byte `protobuf:"bytes,12,opt,name=base_fee_per_gas,json=baseFeePerGas,proto3" json:"base_fee_per_gas,omitempty" ssz-size:"32"`
BlockHash []byte `protobuf:"bytes,13,opt,name=block_hash,json=blockHash,proto3" json:"block_hash,omitempty" ssz-size:"32"`
Transactions [][]byte `protobuf:"bytes,14,rep,name=transactions,proto3" json:"transactions,omitempty" ssz-max:"1048576,1073741824" ssz-size:"?,?"`
Withdrawals []*Withdrawal `protobuf:"bytes,15,rep,name=withdrawals,proto3" json:"withdrawals,omitempty" ssz-max:"16"`
BlobGasUsed uint64 `protobuf:"varint,16,opt,name=blob_gas_used,json=blobGasUsed,proto3" json:"blob_gas_used,omitempty"`
ExcessBlobGas uint64 `protobuf:"varint,17,opt,name=excess_blob_gas,json=excessBlobGas,proto3" json:"excess_blob_gas,omitempty"`
DepositRequests []*DepositRequest `protobuf:"bytes,18,rep,name=deposit_requests,json=depositRequests,proto3" json:"deposit_requests,omitempty" ssz-max:"8192"`
WithdrawalRequests []*WithdrawalRequest `protobuf:"bytes,19,rep,name=withdrawal_requests,json=withdrawalRequests,proto3" json:"withdrawal_requests,omitempty" ssz-max:"16"`
}
func (x *ExecutionPayloadElectra) Reset() {
@@ -743,14 +743,14 @@ func (x *ExecutionPayloadElectra) GetExcessBlobGas() uint64 {
return 0
}
func (x *ExecutionPayloadElectra) GetDepositReceipts() []*DepositReceipt {
func (x *ExecutionPayloadElectra) GetDepositRequests() []*DepositRequest {
if x != nil {
return x.DepositReceipts
return x.DepositRequests
}
return nil
}
func (x *ExecutionPayloadElectra) GetWithdrawalRequests() []*ExecutionLayerWithdrawalRequest {
func (x *ExecutionPayloadElectra) GetWithdrawalRequests() []*WithdrawalRequest {
if x != nil {
return x.WithdrawalRequests
}
@@ -1461,7 +1461,7 @@ type ExecutionPayloadHeaderElectra struct {
WithdrawalsRoot []byte `protobuf:"bytes,15,opt,name=withdrawals_root,json=withdrawalsRoot,proto3" json:"withdrawals_root,omitempty" ssz-size:"32"`
BlobGasUsed uint64 `protobuf:"varint,16,opt,name=blob_gas_used,json=blobGasUsed,proto3" json:"blob_gas_used,omitempty"`
ExcessBlobGas uint64 `protobuf:"varint,17,opt,name=excess_blob_gas,json=excessBlobGas,proto3" json:"excess_blob_gas,omitempty"`
DepositReceiptsRoot []byte `protobuf:"bytes,18,opt,name=deposit_receipts_root,json=depositReceiptsRoot,proto3" json:"deposit_receipts_root,omitempty" ssz-size:"32"`
DepositRequestsRoot []byte `protobuf:"bytes,18,opt,name=deposit_requests_root,json=depositRequestsRoot,proto3" json:"deposit_requests_root,omitempty" ssz-size:"32"`
WithdrawalRequestsRoot []byte `protobuf:"bytes,19,opt,name=withdrawal_requests_root,json=withdrawalRequestsRoot,proto3" json:"withdrawal_requests_root,omitempty" ssz-size:"32"`
}
@@ -1616,9 +1616,9 @@ func (x *ExecutionPayloadHeaderElectra) GetExcessBlobGas() uint64 {
return 0
}
func (x *ExecutionPayloadHeaderElectra) GetDepositReceiptsRoot() []byte {
func (x *ExecutionPayloadHeaderElectra) GetDepositRequestsRoot() []byte {
if x != nil {
return x.DepositReceiptsRoot
return x.DepositRequestsRoot
}
return nil
}
@@ -2197,7 +2197,7 @@ func (x *ExchangeCapabilities) GetSupportedMethods() []string {
return nil
}
type ExecutionLayerWithdrawalRequest struct {
type WithdrawalRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
@@ -2207,8 +2207,8 @@ type ExecutionLayerWithdrawalRequest struct {
Amount uint64 `protobuf:"varint,3,opt,name=amount,proto3" json:"amount,omitempty"`
}
func (x *ExecutionLayerWithdrawalRequest) Reset() {
*x = ExecutionLayerWithdrawalRequest{}
func (x *WithdrawalRequest) Reset() {
*x = WithdrawalRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[20]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
@@ -2216,13 +2216,13 @@ func (x *ExecutionLayerWithdrawalRequest) Reset() {
}
}
func (x *ExecutionLayerWithdrawalRequest) String() string {
func (x *WithdrawalRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ExecutionLayerWithdrawalRequest) ProtoMessage() {}
func (*WithdrawalRequest) ProtoMessage() {}
func (x *ExecutionLayerWithdrawalRequest) ProtoReflect() protoreflect.Message {
func (x *WithdrawalRequest) ProtoReflect() protoreflect.Message {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[20]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
@@ -2234,33 +2234,33 @@ func (x *ExecutionLayerWithdrawalRequest) ProtoReflect() protoreflect.Message {
return mi.MessageOf(x)
}
// Deprecated: Use ExecutionLayerWithdrawalRequest.ProtoReflect.Descriptor instead.
func (*ExecutionLayerWithdrawalRequest) Descriptor() ([]byte, []int) {
// Deprecated: Use WithdrawalRequest.ProtoReflect.Descriptor instead.
func (*WithdrawalRequest) Descriptor() ([]byte, []int) {
return file_proto_engine_v1_execution_engine_proto_rawDescGZIP(), []int{20}
}
func (x *ExecutionLayerWithdrawalRequest) GetSourceAddress() []byte {
func (x *WithdrawalRequest) GetSourceAddress() []byte {
if x != nil {
return x.SourceAddress
}
return nil
}
func (x *ExecutionLayerWithdrawalRequest) GetValidatorPubkey() []byte {
func (x *WithdrawalRequest) GetValidatorPubkey() []byte {
if x != nil {
return x.ValidatorPubkey
}
return nil
}
func (x *ExecutionLayerWithdrawalRequest) GetAmount() uint64 {
func (x *WithdrawalRequest) GetAmount() uint64 {
if x != nil {
return x.Amount
}
return 0
}
type DepositReceipt struct {
type DepositRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
@@ -2272,8 +2272,8 @@ type DepositReceipt struct {
Index uint64 `protobuf:"varint,5,opt,name=index,proto3" json:"index,omitempty"`
}
func (x *DepositReceipt) Reset() {
*x = DepositReceipt{}
func (x *DepositRequest) Reset() {
*x = DepositRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[21]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
@@ -2281,13 +2281,13 @@ func (x *DepositReceipt) Reset() {
}
}
func (x *DepositReceipt) String() string {
func (x *DepositRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*DepositReceipt) ProtoMessage() {}
func (*DepositRequest) ProtoMessage() {}
func (x *DepositReceipt) ProtoReflect() protoreflect.Message {
func (x *DepositRequest) ProtoReflect() protoreflect.Message {
mi := &file_proto_engine_v1_execution_engine_proto_msgTypes[21]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
@@ -2299,40 +2299,40 @@ func (x *DepositReceipt) ProtoReflect() protoreflect.Message {
return mi.MessageOf(x)
}
// Deprecated: Use DepositReceipt.ProtoReflect.Descriptor instead.
func (*DepositReceipt) Descriptor() ([]byte, []int) {
// Deprecated: Use DepositRequest.ProtoReflect.Descriptor instead.
func (*DepositRequest) Descriptor() ([]byte, []int) {
return file_proto_engine_v1_execution_engine_proto_rawDescGZIP(), []int{21}
}
func (x *DepositReceipt) GetPubkey() []byte {
func (x *DepositRequest) GetPubkey() []byte {
if x != nil {
return x.Pubkey
}
return nil
}
func (x *DepositReceipt) GetWithdrawalCredentials() []byte {
func (x *DepositRequest) GetWithdrawalCredentials() []byte {
if x != nil {
return x.WithdrawalCredentials
}
return nil
}
func (x *DepositReceipt) GetAmount() uint64 {
func (x *DepositRequest) GetAmount() uint64 {
if x != nil {
return x.Amount
}
return 0
}
func (x *DepositReceipt) GetSignature() []byte {
func (x *DepositRequest) GetSignature() []byte {
if x != nil {
return x.Signature
}
return nil
}
func (x *DepositReceipt) GetIndex() uint64 {
func (x *DepositRequest) GetIndex() uint64 {
if x != nil {
return x.Index
}
@@ -2472,7 +2472,7 @@ var file_proto_engine_v1_execution_engine_proto_rawDesc = []byte{
0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x26,
0x0a, 0x0f, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61,
0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73, 0x42,
0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, 0x22, 0xac, 0x07, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63, 0x75,
0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, 0x22, 0x9e, 0x07, 0x0a, 0x17, 0x45, 0x78, 0x65, 0x63, 0x75,
0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6c, 0x65, 0x63, 0x74,
0x72, 0x61, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, 0x73,
0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
@@ -2519,62 +2519,182 @@ var file_proto_engine_v1_execution_engine_proto_rawDesc = []byte{
0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73,
0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x04, 0x52,
0x0d, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73, 0x42, 0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, 0x12, 0x57,
0x0a, 0x10, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70,
0x0a, 0x10, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x73, 0x18, 0x12, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65,
0x70, 0x6f, 0x73, 0x69, 0x74, 0x52, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x42, 0x08, 0x92, 0xb5,
0x70, 0x6f, 0x73, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x08, 0x92, 0xb5,
0x18, 0x04, 0x38, 0x31, 0x39, 0x32, 0x52, 0x0f, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x52,
0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x12, 0x6c, 0x0a, 0x13, 0x77, 0x69, 0x74, 0x68, 0x64,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x12, 0x5e, 0x0a, 0x13, 0x77, 0x69, 0x74, 0x68, 0x64,
0x72, 0x61, 0x77, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x13,
0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74,
0x69, 0x6f, 0x6e, 0x4c, 0x61, 0x79, 0x65, 0x72, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77,
0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31,
0x36, 0x52, 0x12, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x73, 0x22, 0x89, 0x02, 0x0a, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74,
0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72,
0x61, 0x57, 0x69, 0x74, 0x68, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x41, 0x6e, 0x64, 0x42, 0x6c, 0x6f,
0x62, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x45, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c,
0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45,
0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45,
0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12,
0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05,
0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x42, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x5f, 0x62,
0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x65, 0x74,
0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72,
0x61, 0x77, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x42, 0x06, 0x92, 0xb5, 0x18,
0x02, 0x31, 0x36, 0x52, 0x12, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x22, 0x89, 0x02, 0x0a, 0x2e, 0x45, 0x78, 0x65, 0x63,
0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x6c, 0x65, 0x63,
0x74, 0x72, 0x61, 0x57, 0x69, 0x74, 0x68, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x41, 0x6e, 0x64, 0x42,
0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x45, 0x0a, 0x07, 0x70, 0x61,
0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x0b, 0x62, 0x6c,
0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x68, 0x6f,
0x75, 0x6c, 0x64, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x5f, 0x62, 0x75, 0x69,
0x6c, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x73, 0x68, 0x6f, 0x75,
0x6c, 0x64, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65,
0x72, 0x22, 0x7f, 0x0a, 0x20, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61,
0x79, 0x6c, 0x6f, 0x61, 0x64, 0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x57, 0x69, 0x74, 0x68,
0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x45, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61,
0x64, 0x45, 0x6c, 0x65, 0x63, 0x74, 0x72, 0x61, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61,
0x64, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c,
0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x42, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x62, 0x73,
0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e,
0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e,
0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x0b,
0x62, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73,
0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x6f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x5f, 0x62,
0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x73, 0x68,
0x6f, 0x75, 0x6c, 0x64, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x42, 0x75, 0x69, 0x6c,
0x64, 0x65, 0x72, 0x22, 0x7f, 0x0a, 0x20, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x57, 0x69,
0x74, 0x68, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x45, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f,
0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72,
0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78,
0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x43, 0x61,
0x70, 0x65, 0x6c, 0x6c, 0x61, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14,
0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76,
0x61, 0x6c, 0x75, 0x65, 0x22, 0x85, 0x02, 0x0a, 0x2c, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69,
0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x57, 0x69,
0x74, 0x68, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x41, 0x6e, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x42,
0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x43, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63,
0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x43, 0x61, 0x70, 0x65,
0x6c, 0x6c, 0x61, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05,
0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c,
0x75, 0x65, 0x22, 0x85, 0x02, 0x0a, 0x2c, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x57, 0x69, 0x74, 0x68,
0x56, 0x61, 0x6c, 0x75, 0x65, 0x41, 0x6e, 0x64, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e,
0x64, 0x6c, 0x65, 0x12, 0x43, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74,
0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x52,
0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75,
0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x42,
0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x03,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x42,
0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75, 0x6e, 0x64,
0x6c, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x6f, 0x76, 0x65,
0x72, 0x72, 0x69, 0x64, 0x65, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20,
0x01, 0x28, 0x08, 0x52, 0x15, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x4f, 0x76, 0x65, 0x72, 0x72,
0x69, 0x64, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x22, 0xc0, 0x04, 0x0a, 0x16, 0x45,
0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48,
0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f,
0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x44, 0x65, 0x6e, 0x65,
0x62, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61,
0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
0x12, 0x42, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65,
0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75,
0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x6c, 0x6f, 0x62,
0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75,
0x6e, 0x64, 0x6c, 0x65, 0x12, 0x36, 0x0a, 0x17, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x6f,
0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x5f, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x18,
0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x15, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x4f, 0x76, 0x65,
0x72, 0x72, 0x69, 0x64, 0x65, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x65, 0x72, 0x22, 0xc0, 0x04, 0x0a,
0x16, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61,
0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e,
0x74, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5,
0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x48, 0x61, 0x73, 0x68,
0x12, 0x2b, 0x0a, 0x0d, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e,
0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52,
0x0c, 0x66, 0x65, 0x65, 0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x25, 0x0a,
0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65,
0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2b, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73,
0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
0x02, 0x33, 0x32, 0x52, 0x0c, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x52, 0x6f, 0x6f,
0x74, 0x12, 0x26, 0x0a, 0x0a, 0x6c, 0x6f, 0x67, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x6f, 0x6d, 0x18,
0x05, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x07, 0x8a, 0xb5, 0x18, 0x03, 0x32, 0x35, 0x36, 0x52, 0x09,
0x6c, 0x6f, 0x67, 0x73, 0x42, 0x6c, 0x6f, 0x6f, 0x6d, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x72, 0x65,
0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06,
0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52, 0x61, 0x6e, 0x64,
0x61, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62,
0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e,
0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x6c, 0x69, 0x6d,
0x69, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x67, 0x61, 0x73, 0x4c, 0x69, 0x6d,
0x69, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x67, 0x61, 0x73, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x09,
0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x67, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x1c, 0x0a,
0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x04,
0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x25, 0x0a, 0x0a, 0x65,
0x78, 0x74, 0x72, 0x61, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x92, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x65, 0x78, 0x74, 0x72, 0x61, 0x44, 0x61,
0x74, 0x61, 0x12, 0x2f, 0x0a, 0x10, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x70,
0x65, 0x72, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5,
0x18, 0x02, 0x33, 0x32, 0x52, 0x0d, 0x62, 0x61, 0x73, 0x65, 0x46, 0x65, 0x65, 0x50, 0x65, 0x72,
0x47, 0x61, 0x73, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73,
0x68, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x33, 0x0a, 0x11, 0x74, 0x72,
0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18,
0x0e, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x10, 0x74,
0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22,
0xfa, 0x04, 0x0a, 0x1d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79,
0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c,
0x61, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, 0x73, 0x68,
0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a,
0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x2b, 0x0a, 0x0d, 0x66, 0x65,
0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x0c, 0x66, 0x65, 0x65, 0x52, 0x65,
0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65,
0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2b,
0x0a, 0x0d, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18,
0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0c, 0x72,
0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x26, 0x0a, 0x0a, 0x6c,
0x6f, 0x67, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x6f, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x07, 0x8a, 0xb5, 0x18, 0x03, 0x32, 0x35, 0x36, 0x52, 0x09, 0x6c, 0x6f, 0x67, 0x73, 0x42, 0x6c,
0x6f, 0x6f, 0x6d, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64,
0x61, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32,
0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x21, 0x0a, 0x0c,
0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01,
0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12,
0x1b, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x08, 0x20, 0x01,
0x28, 0x04, 0x52, 0x08, 0x67, 0x61, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x19, 0x0a, 0x08,
0x67, 0x61, 0x73, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07,
0x67, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73,
0x74, 0x61, 0x6d, 0x70, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65,
0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x25, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x64,
0x61, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x33,
0x32, 0x52, 0x09, 0x65, 0x78, 0x74, 0x72, 0x61, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2f, 0x0a, 0x10,
0x62, 0x61, 0x73, 0x65, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x67, 0x61, 0x73,
0x18, 0x0c, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0d,
0x62, 0x61, 0x73, 0x65, 0x46, 0x65, 0x65, 0x50, 0x65, 0x72, 0x47, 0x61, 0x73, 0x12, 0x25, 0x0a,
0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x0d, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x48, 0x61, 0x73, 0x68, 0x12, 0x33, 0x0a, 0x11, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74,
0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x10, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x31, 0x0a, 0x10, 0x77, 0x69, 0x74,
0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0f, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x77, 0x69, 0x74,
0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xc4, 0x05, 0x0a,
0x1b, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61,
0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x12, 0x27, 0x0a, 0x0b,
0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e,
0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x2b, 0x0a, 0x0d, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63,
0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5,
0x18, 0x02, 0x32, 0x30, 0x52, 0x0c, 0x66, 0x65, 0x65, 0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65,
0x6e, 0x74, 0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74,
0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09,
0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2b, 0x0a, 0x0d, 0x72, 0x65, 0x63,
0x65, 0x69, 0x70, 0x74, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c,
0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0c, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70,
0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x26, 0x0a, 0x0a, 0x6c, 0x6f, 0x67, 0x73, 0x5f, 0x62,
0x6c, 0x6f, 0x6f, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x07, 0x8a, 0xb5, 0x18, 0x03,
0x32, 0x35, 0x36, 0x52, 0x09, 0x6c, 0x6f, 0x67, 0x73, 0x42, 0x6c, 0x6f, 0x6f, 0x6d, 0x12, 0x27,
0x0a, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x06, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65,
0x76, 0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x67, 0x61,
0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x67,
0x61, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x67, 0x61, 0x73, 0x5f, 0x75,
0x73, 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x67, 0x61, 0x73, 0x55, 0x73,
0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18,
0x0a, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70,
0x12, 0x25, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0b,
0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x65, 0x78,
0x74, 0x72, 0x61, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2f, 0x0a, 0x10, 0x62, 0x61, 0x73, 0x65, 0x5f,
0x66, 0x65, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0d, 0x62, 0x61, 0x73, 0x65, 0x46,
0x65, 0x65, 0x50, 0x65, 0x72, 0x47, 0x61, 0x73, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63,
0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5,
0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12,
0x33, 0x0a, 0x11, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f,
0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x33, 0x32, 0x52, 0x10, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73,
0x52, 0x6f, 0x6f, 0x74, 0x12, 0x31, 0x0a, 0x10, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77,
0x61, 0x6c, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06,
0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77,
0x61, 0x6c, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x62, 0x6c, 0x6f, 0x62, 0x5f,
0x67, 0x61, 0x73, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x10, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b,
0x62, 0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x26, 0x0a, 0x0f, 0x65,
0x78, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x11,
0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73, 0x42, 0x6c, 0x6f, 0x62,
0x47, 0x61, 0x73, 0x22, 0xc4, 0x06, 0x0a, 0x1d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,
0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6c,
0x65, 0x63, 0x74, 0x72, 0x61, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f,
0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x33, 0x32, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x2b,
0x0a, 0x0d, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18,
@@ -2608,278 +2728,156 @@ var file_proto_engine_v1_execution_engine_proto_rawDesc = []byte{
0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x33, 0x0a, 0x11, 0x74, 0x72, 0x61, 0x6e,
0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0e, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x10, 0x74, 0x72, 0x61,
0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xfa, 0x04,
0x0a, 0x1d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f,
0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x43, 0x61, 0x70, 0x65, 0x6c, 0x6c, 0x61, 0x12,
0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01,
0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x61,
0x72, 0x65, 0x6e, 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x2b, 0x0a, 0x0d, 0x66, 0x65, 0x65, 0x5f,
0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x0c, 0x66, 0x65, 0x65, 0x52, 0x65, 0x63, 0x69,
0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72,
0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33,
0x32, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2b, 0x0a, 0x0d,
0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0c, 0x72, 0x65, 0x63,
0x65, 0x69, 0x70, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x26, 0x0a, 0x0a, 0x6c, 0x6f, 0x67,
0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x6f, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x07, 0x8a,
0xb5, 0x18, 0x03, 0x32, 0x35, 0x36, 0x52, 0x09, 0x6c, 0x6f, 0x67, 0x73, 0x42, 0x6c, 0x6f, 0x6f,
0x6d, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f,
0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a,
0x70, 0x72, 0x65, 0x76, 0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c,
0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04,
0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1b, 0x0a,
0x09, 0x67, 0x61, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04,
0x52, 0x08, 0x67, 0x61, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x67, 0x61,
0x73, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x67, 0x61,
0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
0x6d, 0x70, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74,
0x61, 0x6d, 0x70, 0x12, 0x25, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x64, 0x61, 0x74,
0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
0x09, 0x65, 0x78, 0x74, 0x72, 0x61, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2f, 0x0a, 0x10, 0x62, 0x61,
0x73, 0x65, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x0c,
0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0d, 0x62, 0x61,
0x73, 0x65, 0x46, 0x65, 0x65, 0x50, 0x65, 0x72, 0x47, 0x61, 0x73, 0x12, 0x25, 0x0a, 0x0a, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61,
0x73, 0x68, 0x12, 0x33, 0x0a, 0x11, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f,
0x6e, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a,
0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x10, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69,
0x6f, 0x6e, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x31, 0x0a, 0x10, 0x77, 0x69, 0x74, 0x68, 0x64,
0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x77, 0x69, 0x74, 0x68, 0x64,
0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0xc4, 0x05, 0x0a, 0x1b, 0x45,
0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48,
0x65, 0x61, 0x64, 0x65, 0x72, 0x44, 0x65, 0x6e, 0x65, 0x62, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61,
0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x48,
0x61, 0x73, 0x68, 0x12, 0x2b, 0x0a, 0x0d, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70,
0x69, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x32, 0x30, 0x52, 0x0c, 0x66, 0x65, 0x65, 0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74,
0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03,
0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74,
0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x2b, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x65, 0x69,
0x70, 0x74, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06,
0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0c, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73,
0x52, 0x6f, 0x6f, 0x74, 0x12, 0x26, 0x0a, 0x0a, 0x6c, 0x6f, 0x67, 0x73, 0x5f, 0x62, 0x6c, 0x6f,
0x6f, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x07, 0x8a, 0xb5, 0x18, 0x03, 0x32, 0x35,
0x36, 0x52, 0x09, 0x6c, 0x6f, 0x67, 0x73, 0x42, 0x6c, 0x6f, 0x6f, 0x6d, 0x12, 0x27, 0x0a, 0x0b,
0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52,
0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e,
0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x1b, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f,
0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x67, 0x61, 0x73,
0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x67, 0x61, 0x73, 0x5f, 0x75, 0x73, 0x65,
0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x67, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64,
0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0a, 0x20,
0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x25,
0x0a, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01,
0x28, 0x0c, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x65, 0x78, 0x74, 0x72,
0x61, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2f, 0x0a, 0x10, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x66, 0x65,
0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0d, 0x62, 0x61, 0x73, 0x65, 0x46, 0x65, 0x65,
0x50, 0x65, 0x72, 0x47, 0x61, 0x73, 0x12, 0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f,
0x68, 0x61, 0x73, 0x68, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x33, 0x0a,
0x11, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x72, 0x6f,
0x6f, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32,
0x52, 0x10, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x6f,
0x6f, 0x74, 0x12, 0x31, 0x0a, 0x10, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c,
0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5,
0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c,
0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x22, 0x0a, 0x0d, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61,
0x73, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x10, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c,
0x6f, 0x62, 0x47, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x78, 0x63,
0x65, 0x73, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x11, 0x20, 0x01,
0x28, 0x04, 0x52, 0x0d, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73, 0x42, 0x6c, 0x6f, 0x62, 0x47, 0x61,
0x73, 0x22, 0xc4, 0x06, 0x0a, 0x1d, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x50,
0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x45, 0x6c, 0x65, 0x63,
0x74, 0x72, 0x61, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x5f, 0x68, 0x61,
0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32,
0x52, 0x0a, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x48, 0x61, 0x73, 0x68, 0x12, 0x2b, 0x0a, 0x0d,
0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x0c, 0x66, 0x65, 0x65,
0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x25, 0x0a, 0x0a, 0x73, 0x74, 0x61,
0x74, 0x65, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a,
0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6f, 0x74,
0x12, 0x2b, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x5f, 0x72, 0x6f, 0x6f,
0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
0x0c, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x26, 0x0a,
0x0a, 0x6c, 0x6f, 0x67, 0x73, 0x5f, 0x62, 0x6c, 0x6f, 0x6f, 0x6d, 0x18, 0x05, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x07, 0x8a, 0xb5, 0x18, 0x03, 0x32, 0x35, 0x36, 0x52, 0x09, 0x6c, 0x6f, 0x67, 0x73,
0x42, 0x6c, 0x6f, 0x6f, 0x6d, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61,
0x6e, 0x64, 0x61, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x21,
0x0a, 0x0c, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x07,
0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x4e, 0x75, 0x6d, 0x62, 0x65,
0x72, 0x12, 0x1b, 0x0a, 0x09, 0x67, 0x61, 0x73, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x08,
0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x67, 0x61, 0x73, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x19,
0x0a, 0x08, 0x67, 0x61, 0x73, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04,
0x52, 0x07, 0x67, 0x61, 0x73, 0x55, 0x73, 0x65, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d,
0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69,
0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x25, 0x0a, 0x0a, 0x65, 0x78, 0x74, 0x72, 0x61,
0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x92, 0xb5, 0x18,
0x02, 0x33, 0x32, 0x52, 0x09, 0x65, 0x78, 0x74, 0x72, 0x61, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2f,
0x0a, 0x10, 0x62, 0x61, 0x73, 0x65, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x67,
0x61, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32,
0x52, 0x0d, 0x62, 0x61, 0x73, 0x65, 0x46, 0x65, 0x65, 0x50, 0x65, 0x72, 0x47, 0x61, 0x73, 0x12,
0x25, 0x0a, 0x0a, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x0d, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x09, 0x62, 0x6c, 0x6f,
0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x12, 0x33, 0x0a, 0x11, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x10, 0x74, 0x72, 0x61, 0x6e, 0x73,
0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x31, 0x0a, 0x10, 0x77,
0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18,
0x0f, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0f, 0x77,
0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x22,
0x0a, 0x0d, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x75, 0x73, 0x65, 0x64, 0x18,
0x10, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, 0x55, 0x73,
0x65, 0x64, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x6c, 0x6f,
0x62, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x65, 0x78, 0x63,
0x65, 0x73, 0x73, 0x42, 0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, 0x12, 0x3a, 0x0a, 0x15, 0x64, 0x65,
0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x5f, 0x72,
0x6f, 0x6f, 0x74, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33,
0x32, 0x52, 0x13, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x52, 0x65, 0x63, 0x65, 0x69, 0x70,
0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x40, 0x0a, 0x18, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72,
0x61, 0x77, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x5f, 0x72, 0x6f,
0x6f, 0x74, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32,
0x52, 0x16, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x9a, 0x01, 0x0a, 0x11, 0x50, 0x61, 0x79,
0x6c, 0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x12, 0x1c,
0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28,
0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0b,
0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52,
0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x3e, 0x0a, 0x17, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74,
0x65, 0x64, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74,
0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x15,
0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x65, 0x64, 0x46, 0x65, 0x65, 0x52, 0x65, 0x63, 0x69,
0x70, 0x69, 0x65, 0x6e, 0x74, 0x22, 0xe6, 0x01, 0x0a, 0x13, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61,
0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x56, 0x32, 0x12, 0x1c, 0x0a,
0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04,
0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0b, 0x70,
0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c,
0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52, 0x61,
0x6e, 0x64, 0x61, 0x6f, 0x12, 0x3e, 0x0a, 0x17, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x65,
0x64, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18,
0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x15, 0x73,
0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x65, 0x64, 0x46, 0x65, 0x65, 0x52, 0x65, 0x63, 0x69, 0x70,
0x69, 0x65, 0x6e, 0x74, 0x12, 0x48, 0x0a, 0x0b, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77,
0x61, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x65, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57,
0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31,
0x36, 0x52, 0x0b, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x22, 0xa7,
0x02, 0x0a, 0x13, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62,
0x75, 0x74, 0x65, 0x73, 0x56, 0x33, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74,
0x61, 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73,
0x74, 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e,
0x64, 0x61, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33,
0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x3e, 0x0a,
0x17, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x72,
0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06,
0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x15, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x65,
0x64, 0x46, 0x65, 0x65, 0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x48, 0x0a,
0x0b, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03,
0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e,
0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77,
0x61, 0x6c, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, 0x0b, 0x77, 0x69, 0x74, 0x68,
0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x12, 0x3f, 0x0a, 0x18, 0x70, 0x61, 0x72, 0x65, 0x6e,
0x74, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x72,
0x6f, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33,
0x32, 0x52, 0x15, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x42, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x42,
0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x92, 0x02, 0x0a, 0x0d, 0x50, 0x61, 0x79,
0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x40, 0x0a, 0x06, 0x73, 0x74,
0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x28, 0x2e, 0x65, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x53, 0x74,
0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x32, 0x0a, 0x11,
0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x5f, 0x68, 0x61, 0x73,
0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
0x0f, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x48, 0x61, 0x73, 0x68,
0x12, 0x29, 0x0a, 0x10, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x65,
0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x76, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x60, 0x0a, 0x06, 0x53,
0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e,
0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a,
0x07, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x59,
0x4e, 0x43, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x41, 0x43, 0x43, 0x45, 0x50,
0x54, 0x45, 0x44, 0x10, 0x04, 0x12, 0x16, 0x0a, 0x12, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44,
0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x48, 0x41, 0x53, 0x48, 0x10, 0x05, 0x22, 0xab, 0x01,
0x0a, 0x0f, 0x46, 0x6f, 0x72, 0x6b, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74,
0x65, 0x12, 0x2e, 0x0a, 0x0f, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f,
0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x33, 0x32, 0x52, 0x0d, 0x68, 0x65, 0x61, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73,
0x68, 0x12, 0x2e, 0x0a, 0x0f, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f,
0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x33, 0x32, 0x52, 0x0d, 0x73, 0x61, 0x66, 0x65, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73,
0x68, 0x12, 0x38, 0x0a, 0x14, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x62,
0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x12, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a,
0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x22, 0xd6, 0x01, 0x0a, 0x0a,
0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e,
0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x12, 0x78, 0x0a, 0x0f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x69, 0x6e,
0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18, 0x4b, 0x67,
0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x31, 0x0a,
0x10, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x5f, 0x72, 0x6f, 0x6f,
0x74, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
0x0f, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x52, 0x6f, 0x6f, 0x74,
0x12, 0x22, 0x0a, 0x0d, 0x62, 0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x5f, 0x75, 0x73, 0x65,
0x64, 0x18, 0x10, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73,
0x55, 0x73, 0x65, 0x64, 0x12, 0x26, 0x0a, 0x0f, 0x65, 0x78, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62,
0x6c, 0x6f, 0x62, 0x5f, 0x67, 0x61, 0x73, 0x18, 0x11, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x65,
0x78, 0x63, 0x65, 0x73, 0x73, 0x42, 0x6c, 0x6f, 0x62, 0x47, 0x61, 0x73, 0x12, 0x3a, 0x0a, 0x15,
0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73,
0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
0x02, 0x33, 0x32, 0x52, 0x13, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x12, 0x40, 0x0a, 0x18, 0x77, 0x69, 0x74, 0x68,
0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x5f,
0x72, 0x6f, 0x6f, 0x74, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02,
0x33, 0x32, 0x52, 0x16, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x9a, 0x01, 0x0a, 0x11, 0x50,
0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73,
0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x01, 0x20,
0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x27,
0x0a, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65,
0x76, 0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x3e, 0x0a, 0x17, 0x73, 0x75, 0x67, 0x67, 0x65,
0x73, 0x74, 0x65, 0x64, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65,
0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30,
0x52, 0x15, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x65, 0x64, 0x46, 0x65, 0x65, 0x52, 0x65,
0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x22, 0xe6, 0x01, 0x0a, 0x13, 0x50, 0x61, 0x79, 0x6c,
0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x56, 0x32, 0x12,
0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01,
0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a,
0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x02, 0x20, 0x01,
0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76,
0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12, 0x3e, 0x0a, 0x17, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73,
0x74, 0x65, 0x64, 0x5f, 0x66, 0x65, 0x65, 0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e,
0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52,
0x15, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x65, 0x64, 0x46, 0x65, 0x65, 0x52, 0x65, 0x63,
0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x48, 0x0a, 0x0b, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72,
0x61, 0x77, 0x61, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x65, 0x74,
0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
0x2e, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x42, 0x06, 0x92, 0xb5, 0x18,
0x02, 0x31, 0x36, 0x52, 0x0b, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73,
0x22, 0xa7, 0x02, 0x0a, 0x13, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x41, 0x74, 0x74, 0x72,
0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x56, 0x33, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65,
0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d,
0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x27, 0x0a, 0x0b, 0x70, 0x72, 0x65, 0x76, 0x5f, 0x72,
0x61, 0x6e, 0x64, 0x61, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
0x02, 0x33, 0x32, 0x52, 0x0a, 0x70, 0x72, 0x65, 0x76, 0x52, 0x61, 0x6e, 0x64, 0x61, 0x6f, 0x12,
0x3e, 0x0a, 0x17, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x66, 0x65, 0x65,
0x5f, 0x72, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c,
0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x15, 0x73, 0x75, 0x67, 0x67, 0x65, 0x73,
0x74, 0x65, 0x64, 0x46, 0x65, 0x65, 0x52, 0x65, 0x63, 0x69, 0x70, 0x69, 0x65, 0x6e, 0x74, 0x12,
0x48, 0x0a, 0x0b, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x18, 0x04,
0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e,
0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72,
0x61, 0x77, 0x61, 0x6c, 0x42, 0x06, 0x92, 0xb5, 0x18, 0x02, 0x31, 0x36, 0x52, 0x0b, 0x77, 0x69,
0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x73, 0x12, 0x3f, 0x0a, 0x18, 0x70, 0x61, 0x72,
0x65, 0x6e, 0x74, 0x5f, 0x62, 0x65, 0x61, 0x63, 0x6f, 0x6e, 0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b,
0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
0x02, 0x33, 0x32, 0x52, 0x15, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x42, 0x65, 0x61, 0x63, 0x6f,
0x6e, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x6f, 0x6f, 0x74, 0x22, 0x92, 0x02, 0x0a, 0x0d, 0x50,
0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x40, 0x0a, 0x06,
0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x28, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76,
0x31, 0x2e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e,
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x32,
0x0a, 0x11, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x5f, 0x68,
0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33,
0x32, 0x52, 0x0f, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x48, 0x61,
0x73, 0x68, 0x12, 0x29, 0x0a, 0x10, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x5f, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x76, 0x61,
0x6c, 0x69, 0x64, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x60, 0x0a,
0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x4e, 0x4b, 0x4e, 0x4f,
0x57, 0x4e, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x01, 0x12,
0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07,
0x53, 0x59, 0x4e, 0x43, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x41, 0x43, 0x43,
0x45, 0x50, 0x54, 0x45, 0x44, 0x10, 0x04, 0x12, 0x16, 0x0a, 0x12, 0x49, 0x4e, 0x56, 0x41, 0x4c,
0x49, 0x44, 0x5f, 0x42, 0x4c, 0x4f, 0x43, 0x4b, 0x5f, 0x48, 0x41, 0x53, 0x48, 0x10, 0x05, 0x22,
0xab, 0x01, 0x0a, 0x0f, 0x46, 0x6f, 0x72, 0x6b, 0x63, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x53, 0x74,
0x61, 0x74, 0x65, 0x12, 0x2e, 0x0a, 0x0f, 0x68, 0x65, 0x61, 0x64, 0x5f, 0x62, 0x6c, 0x6f, 0x63,
0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5,
0x18, 0x02, 0x33, 0x32, 0x52, 0x0d, 0x68, 0x65, 0x61, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48,
0x61, 0x73, 0x68, 0x12, 0x2e, 0x0a, 0x0f, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x62, 0x6c, 0x6f, 0x63,
0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5,
0x18, 0x02, 0x33, 0x32, 0x52, 0x0d, 0x73, 0x61, 0x66, 0x65, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48,
0x61, 0x73, 0x68, 0x12, 0x38, 0x0a, 0x14, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x64,
0x5f, 0x62, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x12, 0x66, 0x69, 0x6e, 0x61, 0x6c,
0x69, 0x7a, 0x65, 0x64, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x48, 0x61, 0x73, 0x68, 0x22, 0xd6, 0x01,
0x0a, 0x0a, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x12, 0x14, 0x0a, 0x05,
0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64,
0x65, 0x78, 0x12, 0x78, 0x0a, 0x0f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x5f,
0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x42, 0x4f, 0x82, 0xb5, 0x18,
0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73,
0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f,
0x76, 0x35, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70,
0x65, 0x73, 0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61,
0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0e, 0x76, 0x61,
0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x20, 0x0a, 0x07,
0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a,
0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x16,
0x0a, 0x06, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06,
0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x9e, 0x01, 0x0a, 0x0b, 0x42, 0x6c, 0x6f, 0x62, 0x73,
0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x39, 0x0a, 0x0f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f,
0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x42,
0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39,
0x36, 0x52, 0x0e, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74,
0x73, 0x12, 0x28, 0x0a, 0x06, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28,
0x0c, 0x42, 0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04, 0x34,
0x30, 0x39, 0x36, 0x52, 0x06, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x12, 0x2a, 0x0a, 0x05, 0x62,
0x6c, 0x6f, 0x62, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x14, 0x8a, 0xb5, 0x18, 0x08,
0x3f, 0x2c, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36,
0x52, 0x05, 0x62, 0x6c, 0x6f, 0x62, 0x73, 0x22, 0x26, 0x0a, 0x04, 0x42, 0x6c, 0x6f, 0x62, 0x12,
0x1e, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a, 0x8a,
0xb5, 0x18, 0x06, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22,
0x43, 0x0a, 0x14, 0x45, 0x78, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x43, 0x61, 0x70, 0x61, 0x62,
0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x73, 0x75, 0x70, 0x70, 0x6f,
0x72, 0x74, 0x65, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03,
0x28, 0x09, 0x52, 0x10, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74,
0x68, 0x6f, 0x64, 0x73, 0x22, 0x8d, 0x01, 0x0a, 0x11, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61,
0x77, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x0e, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01,
0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x0d, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x31, 0x0a, 0x10, 0x76, 0x61, 0x6c,
0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x70, 0x75, 0x62, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x0f, 0x76, 0x61, 0x6c,
0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x50, 0x75, 0x62, 0x6b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06,
0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x61, 0x6d,
0x6f, 0x75, 0x6e, 0x74, 0x22, 0xc3, 0x01, 0x0a, 0x0e, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1e, 0x0a, 0x06, 0x70, 0x75, 0x62, 0x6b, 0x65,
0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52,
0x06, 0x70, 0x75, 0x62, 0x6b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x16, 0x77, 0x69, 0x74, 0x68, 0x64,
0x72, 0x61, 0x77, 0x61, 0x6c, 0x5f, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c,
0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52,
0x15, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x43, 0x72, 0x65, 0x64, 0x65,
0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74,
0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x24,
0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28,
0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61,
0x74, 0x75, 0x72, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20,
0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x96, 0x01, 0x0a, 0x16, 0x6f,
0x72, 0x67, 0x2e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69,
0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x14, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67,
0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61,
0x74, 0x69, 0x63, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35,
0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x65, 0x6e, 0x73, 0x75, 0x73, 0x2d, 0x74, 0x79, 0x70, 0x65, 0x73,
0x2f, 0x70, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x2e, 0x56, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0e, 0x76, 0x61, 0x6c, 0x69,
0x64, 0x61, 0x74, 0x6f, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x20, 0x0a, 0x07, 0x61, 0x64,
0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18,
0x02, 0x32, 0x30, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x16, 0x0a, 0x06,
0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x61, 0x6d,
0x6f, 0x75, 0x6e, 0x74, 0x22, 0x9e, 0x01, 0x0a, 0x0b, 0x42, 0x6c, 0x6f, 0x62, 0x73, 0x42, 0x75,
0x6e, 0x64, 0x6c, 0x65, 0x12, 0x39, 0x0a, 0x0f, 0x6b, 0x7a, 0x67, 0x5f, 0x63, 0x6f, 0x6d, 0x6d,
0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x10, 0x8a,
0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52,
0x0e, 0x6b, 0x7a, 0x67, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12,
0x28, 0x0a, 0x06, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0c, 0x42,
0x10, 0x8a, 0xb5, 0x18, 0x04, 0x3f, 0x2c, 0x34, 0x38, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39,
0x36, 0x52, 0x06, 0x70, 0x72, 0x6f, 0x6f, 0x66, 0x73, 0x12, 0x2a, 0x0a, 0x05, 0x62, 0x6c, 0x6f,
0x62, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0c, 0x42, 0x14, 0x8a, 0xb5, 0x18, 0x08, 0x3f, 0x2c,
0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x92, 0xb5, 0x18, 0x04, 0x34, 0x30, 0x39, 0x36, 0x52, 0x05,
0x62, 0x6c, 0x6f, 0x62, 0x73, 0x22, 0x26, 0x0a, 0x04, 0x42, 0x6c, 0x6f, 0x62, 0x12, 0x1e, 0x0a,
0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x0a, 0x8a, 0xb5, 0x18,
0x06, 0x31, 0x33, 0x31, 0x30, 0x37, 0x32, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x43, 0x0a,
0x14, 0x45, 0x78, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x43, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c,
0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74,
0x65, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09,
0x52, 0x10, 0x73, 0x75, 0x70, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x4d, 0x65, 0x74, 0x68, 0x6f,
0x64, 0x73, 0x22, 0x9b, 0x01, 0x0a, 0x1f, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
0x4c, 0x61, 0x79, 0x65, 0x72, 0x57, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x0e, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06,
0x8a, 0xb5, 0x18, 0x02, 0x32, 0x30, 0x52, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x41, 0x64,
0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x31, 0x0a, 0x10, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x6f, 0x72, 0x5f, 0x70, 0x75, 0x62, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x42,
0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x0f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74,
0x6f, 0x72, 0x50, 0x75, 0x62, 0x6b, 0x65, 0x79, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x6d, 0x6f, 0x75,
0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74,
0x22, 0xc3, 0x01, 0x0a, 0x0e, 0x44, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x52, 0x65, 0x63, 0x65,
0x69, 0x70, 0x74, 0x12, 0x1e, 0x0a, 0x06, 0x70, 0x75, 0x62, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x34, 0x38, 0x52, 0x06, 0x70, 0x75, 0x62,
0x6b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x16, 0x77, 0x69, 0x74, 0x68, 0x64, 0x72, 0x61, 0x77, 0x61,
0x6c, 0x5f, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x73, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a, 0xb5, 0x18, 0x02, 0x33, 0x32, 0x52, 0x15, 0x77, 0x69, 0x74,
0x68, 0x64, 0x72, 0x61, 0x77, 0x61, 0x6c, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61,
0x6c, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01,
0x28, 0x04, 0x52, 0x06, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x24, 0x0a, 0x09, 0x73, 0x69,
0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x42, 0x06, 0x8a,
0xb5, 0x18, 0x02, 0x39, 0x36, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65,
0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x04, 0x52,
0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x96, 0x01, 0x0a, 0x16, 0x6f, 0x72, 0x67, 0x2e, 0x65,
0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x2e, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x76,
0x31, 0x42, 0x14, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x67, 0x69,
0x6e, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x3a, 0x67, 0x69, 0x74, 0x68, 0x75,
0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x61, 0x74, 0x69, 0x63, 0x6c,
0x61, 0x62, 0x73, 0x2f, 0x70, 0x72, 0x79, 0x73, 0x6d, 0x2f, 0x76, 0x35, 0x2f, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x2f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x65, 0x6e, 0x67,
0x69, 0x6e, 0x65, 0x76, 0x31, 0xaa, 0x02, 0x12, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d,
0x2e, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x12, 0x45, 0x74, 0x68,
0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x5c, 0x76, 0x31, 0x62,
0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2f, 0x76, 0x31,
0x3b, 0x65, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x76, 0x31, 0xaa, 0x02, 0x12, 0x45, 0x74, 0x68, 0x65,
0x72, 0x65, 0x75, 0x6d, 0x2e, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65, 0x2e, 0x56, 0x31, 0xca, 0x02,
0x12, 0x45, 0x74, 0x68, 0x65, 0x72, 0x65, 0x75, 0x6d, 0x5c, 0x45, 0x6e, 0x67, 0x69, 0x6e, 0x65,
0x5c, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@@ -2918,15 +2916,15 @@ var file_proto_engine_v1_execution_engine_proto_goTypes = []interface{}{
(*BlobsBundle)(nil), // 18: ethereum.engine.v1.BlobsBundle
(*Blob)(nil), // 19: ethereum.engine.v1.Blob
(*ExchangeCapabilities)(nil), // 20: ethereum.engine.v1.ExchangeCapabilities
(*ExecutionLayerWithdrawalRequest)(nil), // 21: ethereum.engine.v1.ExecutionLayerWithdrawalRequest
(*DepositReceipt)(nil), // 22: ethereum.engine.v1.DepositReceipt
(*WithdrawalRequest)(nil), // 21: ethereum.engine.v1.WithdrawalRequest
(*DepositRequest)(nil), // 22: ethereum.engine.v1.DepositRequest
}
var file_proto_engine_v1_execution_engine_proto_depIdxs = []int32{
17, // 0: ethereum.engine.v1.ExecutionPayloadCapella.withdrawals:type_name -> ethereum.engine.v1.Withdrawal
17, // 1: ethereum.engine.v1.ExecutionPayloadDeneb.withdrawals:type_name -> ethereum.engine.v1.Withdrawal
17, // 2: ethereum.engine.v1.ExecutionPayloadElectra.withdrawals:type_name -> ethereum.engine.v1.Withdrawal
22, // 3: ethereum.engine.v1.ExecutionPayloadElectra.deposit_receipts:type_name -> ethereum.engine.v1.DepositReceipt
21, // 4: ethereum.engine.v1.ExecutionPayloadElectra.withdrawal_requests:type_name -> ethereum.engine.v1.ExecutionLayerWithdrawalRequest
22, // 3: ethereum.engine.v1.ExecutionPayloadElectra.deposit_requests:type_name -> ethereum.engine.v1.DepositRequest
21, // 4: ethereum.engine.v1.ExecutionPayloadElectra.withdrawal_requests:type_name -> ethereum.engine.v1.WithdrawalRequest
4, // 5: ethereum.engine.v1.ExecutionPayloadElectraWithValueAndBlobsBundle.payload:type_name -> ethereum.engine.v1.ExecutionPayloadElectra
18, // 6: ethereum.engine.v1.ExecutionPayloadElectraWithValueAndBlobsBundle.blobs_bundle:type_name -> ethereum.engine.v1.BlobsBundle
2, // 7: ethereum.engine.v1.ExecutionPayloadCapellaWithValue.payload:type_name -> ethereum.engine.v1.ExecutionPayloadCapella
@@ -3189,7 +3187,7 @@ func file_proto_engine_v1_execution_engine_proto_init() {
}
}
file_proto_engine_v1_execution_engine_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ExecutionLayerWithdrawalRequest); i {
switch v := v.(*WithdrawalRequest); i {
case 0:
return &v.state
case 1:
@@ -3201,7 +3199,7 @@ func file_proto_engine_v1_execution_engine_proto_init() {
}
}
file_proto_engine_v1_execution_engine_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*DepositReceipt); i {
switch v := v.(*DepositRequest); i {
case 0:
return &v.state
case 1:

View File

@@ -100,8 +100,8 @@ message ExecutionPayloadElectra {
repeated Withdrawal withdrawals = 15 [(ethereum.eth.ext.ssz_max) = "withdrawal.size"];
uint64 blob_gas_used = 16;
uint64 excess_blob_gas = 17;
repeated DepositReceipt deposit_receipts = 18 [(ethereum.eth.ext.ssz_max) = "max_deposit_receipts"]; // new in electra, eip6110
repeated ExecutionLayerWithdrawalRequest withdrawal_requests = 19 [(ethereum.eth.ext.ssz_max) = "max_withdrawal_requests_per_payload.size"]; // new in electra, eip7002, eip7251
repeated DepositRequest deposit_requests = 18 [(ethereum.eth.ext.ssz_max) = "max_deposit_requests_per_payload.size"]; // new in electra, eip6110
repeated WithdrawalRequest withdrawal_requests = 19 [(ethereum.eth.ext.ssz_max) = "max_withdrawal_requests_per_payload.size"]; // new in electra, eip7002, eip7251
}
message ExecutionPayloadElectraWithValueAndBlobsBundle {
@@ -197,7 +197,7 @@ message ExecutionPayloadHeaderElectra {
bytes withdrawals_root = 15 [(ethereum.eth.ext.ssz_size) = "32"];
uint64 blob_gas_used = 16;
uint64 excess_blob_gas = 17;
bytes deposit_receipts_root = 18 [(ethereum.eth.ext.ssz_size) = "32"]; // new in electra, eip6110
bytes deposit_requests_root = 18 [(ethereum.eth.ext.ssz_size) = "32"]; // new in electra, eip6110
bytes withdrawal_requests_root = 19 [(ethereum.eth.ext.ssz_size) = "32"]; // new in electra, eip7002, eip7251
}
@@ -280,9 +280,9 @@ message ExchangeCapabilities {
repeated string supported_methods = 1;
}
// ExecutionLayerWithdrawalRequest is the message from the execution layer to trigger the withdrawal of a validator's balance to its withdrawal address
// WithdrawalRequest is the message from the execution layer to trigger the withdrawal of a validator's balance to its withdrawal address
// new in Electra
message ExecutionLayerWithdrawalRequest {
message WithdrawalRequest {
// The execution address receiving the funds
bytes source_address = 1 [(ethereum.eth.ext.ssz_size) = "20"];
@@ -293,9 +293,9 @@ message ExecutionLayerWithdrawalRequest {
uint64 amount = 3;
}
// DepositReceipt is the message from the execution layer to trigger the deposit of a validator's balance to its balance
// DepositRequest is the message from the execution layer to trigger the deposit of a validator's balance to its balance
// new in Electra
message DepositReceipt {
message DepositRequest {
bytes pubkey = 1 [(ethereum.eth.ext.ssz_size) = "48"];
bytes withdrawal_credentials = 2 [(ethereum.eth.ext.ssz_size) = "32"];
uint64 amount = 3;

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: 5e73dfd1d7df4fb984d773b9aa92658a1bc4db66b0e050f9b869a470e61416d3
// Hash: fead48d52c26c5db4af980b0552c414ffc58387819201e533a44a3e8f4dde5df
package enginev1
import (
@@ -1413,9 +1413,9 @@ func (e *ExecutionPayloadElectra) MarshalSSZTo(buf []byte) (dst []byte, err erro
// Field (16) 'ExcessBlobGas'
dst = ssz.MarshalUint64(dst, e.ExcessBlobGas)
// Offset (17) 'DepositReceipts'
// Offset (17) 'DepositRequests'
dst = ssz.WriteOffset(dst, offset)
offset += len(e.DepositReceipts) * 192
offset += len(e.DepositRequests) * 192
// Offset (18) 'WithdrawalRequests'
dst = ssz.WriteOffset(dst, offset)
@@ -1459,13 +1459,13 @@ func (e *ExecutionPayloadElectra) MarshalSSZTo(buf []byte) (dst []byte, err erro
}
}
// Field (17) 'DepositReceipts'
if size := len(e.DepositReceipts); size > 8192 {
err = ssz.ErrListTooBigFn("--.DepositReceipts", size, 8192)
// Field (17) 'DepositRequests'
if size := len(e.DepositRequests); size > 8192 {
err = ssz.ErrListTooBigFn("--.DepositRequests", size, 8192)
return
}
for ii := 0; ii < len(e.DepositReceipts); ii++ {
if dst, err = e.DepositReceipts[ii].MarshalSSZTo(dst); err != nil {
for ii := 0; ii < len(e.DepositRequests); ii++ {
if dst, err = e.DepositRequests[ii].MarshalSSZTo(dst); err != nil {
return
}
}
@@ -1580,7 +1580,7 @@ func (e *ExecutionPayloadElectra) UnmarshalSSZ(buf []byte) error {
// Field (16) 'ExcessBlobGas'
e.ExcessBlobGas = ssz.UnmarshallUint64(buf[520:528])
// Offset (17) 'DepositReceipts'
// Offset (17) 'DepositRequests'
if o17 = ssz.ReadOffset(buf[528:532]); o17 > size || o14 > o17 {
return ssz.ErrOffset
}
@@ -1643,19 +1643,19 @@ func (e *ExecutionPayloadElectra) UnmarshalSSZ(buf []byte) error {
}
}
// Field (17) 'DepositReceipts'
// Field (17) 'DepositRequests'
{
buf = tail[o17:o18]
num, err := ssz.DivideInt2(len(buf), 192, 8192)
if err != nil {
return err
}
e.DepositReceipts = make([]*DepositReceipt, num)
e.DepositRequests = make([]*DepositRequest, num)
for ii := 0; ii < num; ii++ {
if e.DepositReceipts[ii] == nil {
e.DepositReceipts[ii] = new(DepositReceipt)
if e.DepositRequests[ii] == nil {
e.DepositRequests[ii] = new(DepositRequest)
}
if err = e.DepositReceipts[ii].UnmarshalSSZ(buf[ii*192 : (ii+1)*192]); err != nil {
if err = e.DepositRequests[ii].UnmarshalSSZ(buf[ii*192 : (ii+1)*192]); err != nil {
return err
}
}
@@ -1668,10 +1668,10 @@ func (e *ExecutionPayloadElectra) UnmarshalSSZ(buf []byte) error {
if err != nil {
return err
}
e.WithdrawalRequests = make([]*ExecutionLayerWithdrawalRequest, num)
e.WithdrawalRequests = make([]*WithdrawalRequest, num)
for ii := 0; ii < num; ii++ {
if e.WithdrawalRequests[ii] == nil {
e.WithdrawalRequests[ii] = new(ExecutionLayerWithdrawalRequest)
e.WithdrawalRequests[ii] = new(WithdrawalRequest)
}
if err = e.WithdrawalRequests[ii].UnmarshalSSZ(buf[ii*76 : (ii+1)*76]); err != nil {
return err
@@ -1697,8 +1697,8 @@ func (e *ExecutionPayloadElectra) SizeSSZ() (size int) {
// Field (14) 'Withdrawals'
size += len(e.Withdrawals) * 44
// Field (17) 'DepositReceipts'
size += len(e.DepositReceipts) * 192
// Field (17) 'DepositRequests'
size += len(e.DepositRequests) * 192
// Field (18) 'WithdrawalRequests'
size += len(e.WithdrawalRequests) * 76
@@ -1856,15 +1856,15 @@ func (e *ExecutionPayloadElectra) HashTreeRootWith(hh *ssz.Hasher) (err error) {
// Field (16) 'ExcessBlobGas'
hh.PutUint64(e.ExcessBlobGas)
// Field (17) 'DepositReceipts'
// Field (17) 'DepositRequests'
{
subIndx := hh.Index()
num := uint64(len(e.DepositReceipts))
num := uint64(len(e.DepositRequests))
if num > 8192 {
err = ssz.ErrIncorrectListSize
return
}
for _, elem := range e.DepositReceipts {
for _, elem := range e.DepositRequests {
if err = elem.HashTreeRootWith(hh); err != nil {
return
}
@@ -3015,12 +3015,12 @@ func (e *ExecutionPayloadHeaderElectra) MarshalSSZTo(buf []byte) (dst []byte, er
// Field (16) 'ExcessBlobGas'
dst = ssz.MarshalUint64(dst, e.ExcessBlobGas)
// Field (17) 'DepositReceiptsRoot'
if size := len(e.DepositReceiptsRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.DepositReceiptsRoot", size, 32)
// Field (17) 'DepositRequestsRoot'
if size := len(e.DepositRequestsRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.DepositRequestsRoot", size, 32)
return
}
dst = append(dst, e.DepositReceiptsRoot...)
dst = append(dst, e.DepositRequestsRoot...)
// Field (18) 'WithdrawalRequestsRoot'
if size := len(e.WithdrawalRequestsRoot); size != 32 {
@@ -3137,11 +3137,11 @@ func (e *ExecutionPayloadHeaderElectra) UnmarshalSSZ(buf []byte) error {
// Field (16) 'ExcessBlobGas'
e.ExcessBlobGas = ssz.UnmarshallUint64(buf[576:584])
// Field (17) 'DepositReceiptsRoot'
if cap(e.DepositReceiptsRoot) == 0 {
e.DepositReceiptsRoot = make([]byte, 0, len(buf[584:616]))
// Field (17) 'DepositRequestsRoot'
if cap(e.DepositRequestsRoot) == 0 {
e.DepositRequestsRoot = make([]byte, 0, len(buf[584:616]))
}
e.DepositReceiptsRoot = append(e.DepositReceiptsRoot, buf[584:616]...)
e.DepositRequestsRoot = append(e.DepositRequestsRoot, buf[584:616]...)
// Field (18) 'WithdrawalRequestsRoot'
if cap(e.WithdrawalRequestsRoot) == 0 {
@@ -3286,12 +3286,12 @@ func (e *ExecutionPayloadHeaderElectra) HashTreeRootWith(hh *ssz.Hasher) (err er
// Field (16) 'ExcessBlobGas'
hh.PutUint64(e.ExcessBlobGas)
// Field (17) 'DepositReceiptsRoot'
if size := len(e.DepositReceiptsRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.DepositReceiptsRoot", size, 32)
// Field (17) 'DepositRequestsRoot'
if size := len(e.DepositRequestsRoot); size != 32 {
err = ssz.ErrBytesLengthFn("--.DepositRequestsRoot", size, 32)
return
}
hh.PutBytes(e.DepositReceiptsRoot)
hh.PutBytes(e.DepositRequestsRoot)
// Field (18) 'WithdrawalRequestsRoot'
if size := len(e.WithdrawalRequestsRoot); size != 32 {
@@ -3647,37 +3647,37 @@ func (b *BlobsBundle) HashTreeRootWith(hh *ssz.Hasher) (err error) {
return
}
// MarshalSSZ ssz marshals the ExecutionLayerWithdrawalRequest object
func (e *ExecutionLayerWithdrawalRequest) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(e)
// MarshalSSZ ssz marshals the WithdrawalRequest object
func (w *WithdrawalRequest) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(w)
}
// MarshalSSZTo ssz marshals the ExecutionLayerWithdrawalRequest object to a target array
func (e *ExecutionLayerWithdrawalRequest) MarshalSSZTo(buf []byte) (dst []byte, err error) {
// MarshalSSZTo ssz marshals the WithdrawalRequest object to a target array
func (w *WithdrawalRequest) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
// Field (0) 'SourceAddress'
if size := len(e.SourceAddress); size != 20 {
if size := len(w.SourceAddress); size != 20 {
err = ssz.ErrBytesLengthFn("--.SourceAddress", size, 20)
return
}
dst = append(dst, e.SourceAddress...)
dst = append(dst, w.SourceAddress...)
// Field (1) 'ValidatorPubkey'
if size := len(e.ValidatorPubkey); size != 48 {
if size := len(w.ValidatorPubkey); size != 48 {
err = ssz.ErrBytesLengthFn("--.ValidatorPubkey", size, 48)
return
}
dst = append(dst, e.ValidatorPubkey...)
dst = append(dst, w.ValidatorPubkey...)
// Field (2) 'Amount'
dst = ssz.MarshalUint64(dst, e.Amount)
dst = ssz.MarshalUint64(dst, w.Amount)
return
}
// UnmarshalSSZ ssz unmarshals the ExecutionLayerWithdrawalRequest object
func (e *ExecutionLayerWithdrawalRequest) UnmarshalSSZ(buf []byte) error {
// UnmarshalSSZ ssz unmarshals the WithdrawalRequest object
func (w *WithdrawalRequest) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size != 76 {
@@ -3685,54 +3685,54 @@ func (e *ExecutionLayerWithdrawalRequest) UnmarshalSSZ(buf []byte) error {
}
// Field (0) 'SourceAddress'
if cap(e.SourceAddress) == 0 {
e.SourceAddress = make([]byte, 0, len(buf[0:20]))
if cap(w.SourceAddress) == 0 {
w.SourceAddress = make([]byte, 0, len(buf[0:20]))
}
e.SourceAddress = append(e.SourceAddress, buf[0:20]...)
w.SourceAddress = append(w.SourceAddress, buf[0:20]...)
// Field (1) 'ValidatorPubkey'
if cap(e.ValidatorPubkey) == 0 {
e.ValidatorPubkey = make([]byte, 0, len(buf[20:68]))
if cap(w.ValidatorPubkey) == 0 {
w.ValidatorPubkey = make([]byte, 0, len(buf[20:68]))
}
e.ValidatorPubkey = append(e.ValidatorPubkey, buf[20:68]...)
w.ValidatorPubkey = append(w.ValidatorPubkey, buf[20:68]...)
// Field (2) 'Amount'
e.Amount = ssz.UnmarshallUint64(buf[68:76])
w.Amount = ssz.UnmarshallUint64(buf[68:76])
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the ExecutionLayerWithdrawalRequest object
func (e *ExecutionLayerWithdrawalRequest) SizeSSZ() (size int) {
// SizeSSZ returns the ssz encoded size in bytes for the WithdrawalRequest object
func (w *WithdrawalRequest) SizeSSZ() (size int) {
size = 76
return
}
// HashTreeRoot ssz hashes the ExecutionLayerWithdrawalRequest object
func (e *ExecutionLayerWithdrawalRequest) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(e)
// HashTreeRoot ssz hashes the WithdrawalRequest object
func (w *WithdrawalRequest) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(w)
}
// HashTreeRootWith ssz hashes the ExecutionLayerWithdrawalRequest object with a hasher
func (e *ExecutionLayerWithdrawalRequest) HashTreeRootWith(hh *ssz.Hasher) (err error) {
// HashTreeRootWith ssz hashes the WithdrawalRequest object with a hasher
func (w *WithdrawalRequest) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'SourceAddress'
if size := len(e.SourceAddress); size != 20 {
if size := len(w.SourceAddress); size != 20 {
err = ssz.ErrBytesLengthFn("--.SourceAddress", size, 20)
return
}
hh.PutBytes(e.SourceAddress)
hh.PutBytes(w.SourceAddress)
// Field (1) 'ValidatorPubkey'
if size := len(e.ValidatorPubkey); size != 48 {
if size := len(w.ValidatorPubkey); size != 48 {
err = ssz.ErrBytesLengthFn("--.ValidatorPubkey", size, 48)
return
}
hh.PutBytes(e.ValidatorPubkey)
hh.PutBytes(w.ValidatorPubkey)
// Field (2) 'Amount'
hh.PutUint64(e.Amount)
hh.PutUint64(w.Amount)
if ssz.EnableVectorizedHTR {
hh.MerkleizeVectorizedHTR(indx)
@@ -3742,13 +3742,13 @@ func (e *ExecutionLayerWithdrawalRequest) HashTreeRootWith(hh *ssz.Hasher) (err
return
}
// MarshalSSZ ssz marshals the DepositReceipt object
func (d *DepositReceipt) MarshalSSZ() ([]byte, error) {
// MarshalSSZ ssz marshals the DepositRequest object
func (d *DepositRequest) MarshalSSZ() ([]byte, error) {
return ssz.MarshalSSZ(d)
}
// MarshalSSZTo ssz marshals the DepositReceipt object to a target array
func (d *DepositReceipt) MarshalSSZTo(buf []byte) (dst []byte, err error) {
// MarshalSSZTo ssz marshals the DepositRequest object to a target array
func (d *DepositRequest) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = buf
// Field (0) 'Pubkey'
@@ -3781,8 +3781,8 @@ func (d *DepositReceipt) MarshalSSZTo(buf []byte) (dst []byte, err error) {
return
}
// UnmarshalSSZ ssz unmarshals the DepositReceipt object
func (d *DepositReceipt) UnmarshalSSZ(buf []byte) error {
// UnmarshalSSZ ssz unmarshals the DepositRequest object
func (d *DepositRequest) UnmarshalSSZ(buf []byte) error {
var err error
size := uint64(len(buf))
if size != 192 {
@@ -3816,19 +3816,19 @@ func (d *DepositReceipt) UnmarshalSSZ(buf []byte) error {
return err
}
// SizeSSZ returns the ssz encoded size in bytes for the DepositReceipt object
func (d *DepositReceipt) SizeSSZ() (size int) {
// SizeSSZ returns the ssz encoded size in bytes for the DepositRequest object
func (d *DepositRequest) SizeSSZ() (size int) {
size = 192
return
}
// HashTreeRoot ssz hashes the DepositReceipt object
func (d *DepositReceipt) HashTreeRoot() ([32]byte, error) {
// HashTreeRoot ssz hashes the DepositRequest object
func (d *DepositRequest) HashTreeRoot() ([32]byte, error) {
return ssz.HashWithDefaultHasher(d)
}
// HashTreeRootWith ssz hashes the DepositReceipt object with a hasher
func (d *DepositReceipt) HashTreeRootWith(hh *ssz.Hasher) (err error) {
// HashTreeRootWith ssz hashes the DepositRequest object with a hasher
func (d *DepositRequest) HashTreeRootWith(hh *ssz.Hasher) (err error) {
indx := hh.Index()
// Field (0) 'Pubkey'

View File

@@ -417,7 +417,7 @@ type ExecutionPayloadDenebJSON struct {
// https://github.com/ethereum/execution-apis/blob/main/src/engine/prague.md#withdrawalrequestv1
type WithdrawalRequestV1 struct {
SourceAddress *common.Address `json:"sourceAddress"`
ValidatorPubkey *BlsPubkey `json:"validatorPublicKey"`
ValidatorPubkey *BlsPubkey `json:"validatorPubkey"`
Amount *hexutil.Uint64 `json:"amount"`
}
@@ -426,7 +426,7 @@ func (r WithdrawalRequestV1) Validate() error {
return errors.Wrap(errJsonNilField, "missing required field 'sourceAddress' for WithdrawalRequestV1")
}
if r.ValidatorPubkey == nil {
return errors.Wrap(errJsonNilField, "missing required field 'validatorPublicKey' for WithdrawalRequestV1")
return errors.Wrap(errJsonNilField, "missing required field 'validatorPubkey' for WithdrawalRequestV1")
}
if r.Amount == nil {
return errors.Wrap(errJsonNilField, "missing required field 'amount' for WithdrawalRequestV1")
@@ -984,19 +984,19 @@ func (e *ExecutionPayloadElectra) MarshalJSON() ([]byte, error) {
BlobGasUsed: &blobGasUsed,
ExcessBlobGas: &excessBlobGas,
WithdrawalRequests: ProtoWithdrawalRequestsToJson(e.WithdrawalRequests),
DepositRequests: ProtoDepositRequestsToJson(e.DepositReceipts),
DepositRequests: ProtoDepositRequestsToJson(e.DepositRequests),
})
}
func JsonDepositRequestsToProto(j []DepositRequestV1) ([]*DepositReceipt, error) {
reqs := make([]*DepositReceipt, len(j))
func JsonDepositRequestsToProto(j []DepositRequestV1) ([]*DepositRequest, error) {
reqs := make([]*DepositRequest, len(j))
for i := range j {
req := j[i]
if err := req.Validate(); err != nil {
return nil, err
}
reqs[i] = &DepositReceipt{
reqs[i] = &DepositRequest{
Pubkey: req.PubKey.Bytes(),
WithdrawalCredentials: req.WithdrawalCredentials.Bytes(),
Amount: uint64(*req.Amount),
@@ -1008,7 +1008,7 @@ func JsonDepositRequestsToProto(j []DepositRequestV1) ([]*DepositReceipt, error)
return reqs, nil
}
func ProtoDepositRequestsToJson(reqs []*DepositReceipt) []DepositRequestV1 {
func ProtoDepositRequestsToJson(reqs []*DepositRequest) []DepositRequestV1 {
j := make([]DepositRequestV1, len(reqs))
for i := range reqs {
r := reqs[i]
@@ -1030,15 +1030,15 @@ func ProtoDepositRequestsToJson(reqs []*DepositReceipt) []DepositRequestV1 {
return j
}
func JsonWithdrawalRequestsToProto(j []WithdrawalRequestV1) ([]*ExecutionLayerWithdrawalRequest, error) {
reqs := make([]*ExecutionLayerWithdrawalRequest, len(j))
func JsonWithdrawalRequestsToProto(j []WithdrawalRequestV1) ([]*WithdrawalRequest, error) {
reqs := make([]*WithdrawalRequest, len(j))
for i := range j {
req := j[i]
if err := req.Validate(); err != nil {
return nil, err
}
reqs[i] = &ExecutionLayerWithdrawalRequest{
reqs[i] = &WithdrawalRequest{
SourceAddress: req.SourceAddress.Bytes(),
ValidatorPubkey: req.ValidatorPubkey.Bytes(),
Amount: uint64(*req.Amount),
@@ -1048,7 +1048,7 @@ func JsonWithdrawalRequestsToProto(j []WithdrawalRequestV1) ([]*ExecutionLayerWi
return reqs, nil
}
func ProtoWithdrawalRequestsToJson(reqs []*ExecutionLayerWithdrawalRequest) []WithdrawalRequestV1 {
func ProtoWithdrawalRequestsToJson(reqs []*WithdrawalRequest) []WithdrawalRequestV1 {
j := make([]WithdrawalRequestV1, len(reqs))
for i := range reqs {
r := reqs[i]
@@ -1105,7 +1105,7 @@ func (j *ExecutionPayloadElectraJSON) ElectraPayload() (*ExecutionPayloadElectra
Withdrawals: j.Withdrawals,
BlobGasUsed: uint64(*j.BlobGasUsed),
ExcessBlobGas: uint64(*j.ExcessBlobGas),
DepositReceipts: dr,
DepositRequests: dr,
WithdrawalRequests: wr,
}, nil
}

View File

@@ -289,7 +289,7 @@ func TestJsonMarshalUnmarshal(t *testing.T) {
bgu := hexutil.Uint64(5)
ebg := hexutil.Uint64(6)
withdrawalReq := []*enginev1.ExecutionLayerWithdrawalRequest{
withdrawalReq := []*enginev1.WithdrawalRequest{
{
SourceAddress: bytesutil.PadTo([]byte("sourceAddress-1"), 20),
ValidatorPubkey: bytesutil.PadTo([]byte("pubKey-1"), 48),
@@ -306,7 +306,7 @@ func TestJsonMarshalUnmarshal(t *testing.T) {
Amount: 3,
},
}
depositReq := []*enginev1.DepositReceipt{
depositReq := []*enginev1.DepositRequest{
{
Pubkey: bytesutil.PadTo([]byte("pubKey-1"), 48),
WithdrawalCredentials: bytesutil.PadTo([]byte("creds-1"), 32),
@@ -410,9 +410,9 @@ func TestJsonMarshalUnmarshal(t *testing.T) {
for i := range pb.Payload.WithdrawalRequests {
require.DeepEqual(t, pb.Payload.WithdrawalRequests[i], withdrawalReq[i])
}
require.Equal(t, len(pb.Payload.DepositReceipts), len(depositReq))
for i := range pb.Payload.DepositReceipts {
require.DeepEqual(t, pb.Payload.DepositReceipts[i], depositReq[i])
require.Equal(t, len(pb.Payload.DepositRequests), len(depositReq))
for i := range pb.Payload.DepositRequests {
require.DeepEqual(t, pb.Payload.DepositRequests[i], depositReq[i])
}
})
t.Run("execution block", func(t *testing.T) {

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: 2874e1dadeb47411763f48fe31e5daaa91ac663e796933d9a508c2e7be94fa5e
// Hash: d06a72227c2f5e350916cce3e89f4e855135a2a22f6ea263dedc68fa506c1ba7
package v1
import (

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: 2755944d1f9b0e6e8b7e58d3cbba1d6028bbbb740ec05f5de43137d5b5459411
// Hash: 0394f3a965a034ef0de1398e168f3971f32a9ccce9db6549dd6e6d258c11c89b
package eth
import (

View File

@@ -1867,7 +1867,7 @@ type BeaconStateElectra struct {
NextWithdrawalIndex uint64 `protobuf:"varint,11001,opt,name=next_withdrawal_index,json=nextWithdrawalIndex,proto3" json:"next_withdrawal_index,omitempty"`
NextWithdrawalValidatorIndex github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.ValidatorIndex `protobuf:"varint,11002,opt,name=next_withdrawal_validator_index,json=nextWithdrawalValidatorIndex,proto3" json:"next_withdrawal_validator_index,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.ValidatorIndex"`
HistoricalSummaries []*HistoricalSummary `protobuf:"bytes,11003,rep,name=historical_summaries,json=historicalSummaries,proto3" json:"historical_summaries,omitempty" ssz-max:"16777216"`
DepositReceiptsStartIndex uint64 `protobuf:"varint,12001,opt,name=deposit_receipts_start_index,json=depositReceiptsStartIndex,proto3" json:"deposit_receipts_start_index,omitempty"`
DepositRequestsStartIndex uint64 `protobuf:"varint,12001,opt,name=deposit_requests_start_index,json=depositRequestsStartIndex,proto3" json:"deposit_requests_start_index,omitempty"`
DepositBalanceToConsume github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Gwei `protobuf:"varint,12002,opt,name=deposit_balance_to_consume,json=depositBalanceToConsume,proto3" json:"deposit_balance_to_consume,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Gwei"`
ExitBalanceToConsume github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Gwei `protobuf:"varint,12003,opt,name=exit_balance_to_consume,json=exitBalanceToConsume,proto3" json:"exit_balance_to_consume,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Gwei"`
EarliestExitEpoch github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Epoch `protobuf:"varint,12004,opt,name=earliest_exit_epoch,json=earliestExitEpoch,proto3" json:"earliest_exit_epoch,omitempty" cast-type:"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Epoch"`
@@ -2106,9 +2106,9 @@ func (x *BeaconStateElectra) GetHistoricalSummaries() []*HistoricalSummary {
return nil
}
func (x *BeaconStateElectra) GetDepositReceiptsStartIndex() uint64 {
func (x *BeaconStateElectra) GetDepositRequestsStartIndex() uint64 {
if x != nil {
return x.DepositReceiptsStartIndex
return x.DepositRequestsStartIndex
}
return 0
}
@@ -3147,10 +3147,10 @@ var file_proto_prysm_v1alpha1_beacon_state_proto_rawDesc = []byte{
0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x79, 0x42, 0x0c, 0x92,
0xb5, 0x18, 0x08, 0x31, 0x36, 0x37, 0x37, 0x37, 0x32, 0x31, 0x36, 0x52, 0x13, 0x68, 0x69, 0x73,
0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x53, 0x75, 0x6d, 0x6d, 0x61, 0x72, 0x69, 0x65, 0x73,
0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x63, 0x65,
0x69, 0x70, 0x74, 0x73, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x12, 0x40, 0x0a, 0x1c, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x72, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x73, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x18, 0xe1, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x19, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74,
0x52, 0x65, 0x63, 0x65, 0x69, 0x70, 0x74, 0x73, 0x53, 0x74, 0x61, 0x72, 0x74, 0x49, 0x6e, 0x64,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x53, 0x74, 0x61, 0x72, 0x74, 0x49, 0x6e, 0x64,
0x65, 0x78, 0x12, 0x83, 0x01, 0x0a, 0x1a, 0x64, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x5f, 0x62,
0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x74, 0x6f, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d,
0x65, 0x18, 0xe2, 0x5d, 0x20, 0x01, 0x28, 0x04, 0x42, 0x45, 0x82, 0xb5, 0x18, 0x41, 0x67, 0x69,

View File

@@ -396,7 +396,7 @@ message BeaconStateElectra {
repeated HistoricalSummary historical_summaries = 11003 [(ethereum.eth.ext.ssz_max) = "16777216"];
// Fields introduced in EIP-7251 fork [12001-13000]
uint64 deposit_receipts_start_index = 12001;
uint64 deposit_requests_start_index = 12001;
uint64 deposit_balance_to_consume = 12002 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Gwei"];
uint64 exit_balance_to_consume = 12003 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Gwei"];
uint64 earliest_exit_epoch = 12004 [(ethereum.eth.ext.cast_type) = "github.com/prysmaticlabs/prysm/v5/consensus-types/primitives.Epoch"];

View File

@@ -1046,19 +1046,19 @@ func CopyExecutionPayloadElectra(payload *enginev1.ExecutionPayloadElectra) *eng
Withdrawals: CopyWithdrawalSlice(payload.Withdrawals),
BlobGasUsed: payload.BlobGasUsed,
ExcessBlobGas: payload.ExcessBlobGas,
DepositReceipts: CopyDepositReceipts(payload.DepositReceipts),
DepositRequests: CopyDepositRequests(payload.DepositRequests),
WithdrawalRequests: CopyWithdrawalRequests(payload.WithdrawalRequests),
}
}
func CopyDepositReceipts(dr []*enginev1.DepositReceipt) []*enginev1.DepositReceipt {
func CopyDepositRequests(dr []*enginev1.DepositRequest) []*enginev1.DepositRequest {
if dr == nil {
return nil
}
newDr := make([]*enginev1.DepositReceipt, len(dr))
newDr := make([]*enginev1.DepositRequest, len(dr))
for i, d := range dr {
newDr[i] = &enginev1.DepositReceipt{
newDr[i] = &enginev1.DepositRequest{
Pubkey: bytesutil.SafeCopyBytes(d.Pubkey),
WithdrawalCredentials: bytesutil.SafeCopyBytes(d.WithdrawalCredentials),
Amount: d.Amount,
@@ -1069,13 +1069,13 @@ func CopyDepositReceipts(dr []*enginev1.DepositReceipt) []*enginev1.DepositRecei
return newDr
}
func CopyWithdrawalRequests(wr []*enginev1.ExecutionLayerWithdrawalRequest) []*enginev1.ExecutionLayerWithdrawalRequest {
func CopyWithdrawalRequests(wr []*enginev1.WithdrawalRequest) []*enginev1.WithdrawalRequest {
if wr == nil {
return nil
}
newWr := make([]*enginev1.ExecutionLayerWithdrawalRequest, len(wr))
newWr := make([]*enginev1.WithdrawalRequest, len(wr))
for i, w := range wr {
newWr[i] = &enginev1.ExecutionLayerWithdrawalRequest{
newWr[i] = &enginev1.WithdrawalRequest{
SourceAddress: bytesutil.SafeCopyBytes(w.SourceAddress),
ValidatorPubkey: bytesutil.SafeCopyBytes(w.ValidatorPubkey),
Amount: w.Amount,
@@ -1107,7 +1107,7 @@ func CopyExecutionPayloadHeaderElectra(payload *enginev1.ExecutionPayloadHeaderE
WithdrawalsRoot: bytesutil.SafeCopyBytes(payload.WithdrawalsRoot),
BlobGasUsed: payload.BlobGasUsed,
ExcessBlobGas: payload.ExcessBlobGas,
DepositReceiptsRoot: bytesutil.SafeCopyBytes(payload.DepositReceiptsRoot),
DepositRequestsRoot: bytesutil.SafeCopyBytes(payload.DepositRequestsRoot),
WithdrawalRequestsRoot: bytesutil.SafeCopyBytes(payload.WithdrawalRequestsRoot),
}
}

View File

@@ -685,12 +685,12 @@ func TestCopyExecutionPayloadElectra(t *testing.T) {
}
}
func TestCopyDepositReceipts(t *testing.T) {
drs := genDepositReceipts(10)
func TestCopyDepositRequests(t *testing.T) {
drs := genDepositRequests(10)
got := v1alpha1.CopyDepositReceipts(drs)
got := v1alpha1.CopyDepositRequests(drs)
if !reflect.DeepEqual(got, drs) {
t.Errorf("TestCopyDepositReceipts() = %v, want %v", got, drs)
t.Errorf("TestCopyDepositRequests() = %v, want %v", got, drs)
}
}
@@ -1442,7 +1442,7 @@ func genExecutionPayloadHeaderElectra() *enginev1.ExecutionPayloadHeaderElectra
WithdrawalsRoot: bytes(32),
BlobGasUsed: 5,
ExcessBlobGas: 6,
DepositReceiptsRoot: bytes(32),
DepositRequestsRoot: bytes(32),
WithdrawalRequestsRoot: bytes(32),
}
}
@@ -1524,21 +1524,21 @@ func genExecutionPayloadElectra() *enginev1.ExecutionPayloadElectra {
Withdrawals: genWithdrawals(10),
BlobGasUsed: 5,
ExcessBlobGas: 6,
DepositReceipts: genDepositReceipts(10),
DepositRequests: genDepositRequests(10),
WithdrawalRequests: genWithdrawalRequests(10),
}
}
func genDepositReceipts(num int) []*enginev1.DepositReceipt {
drs := make([]*enginev1.DepositReceipt, num)
func genDepositRequests(num int) []*enginev1.DepositRequest {
drs := make([]*enginev1.DepositRequest, num)
for i := 0; i < num; i++ {
drs[i] = genDepositReceipt()
drs[i] = genDepositRequest()
}
return drs
}
func genDepositReceipt() *enginev1.DepositReceipt {
return &enginev1.DepositReceipt{
func genDepositRequest() *enginev1.DepositRequest {
return &enginev1.DepositRequest{
Pubkey: bytes(48),
WithdrawalCredentials: bytes(32),
Amount: 55555,
@@ -1547,16 +1547,16 @@ func genDepositReceipt() *enginev1.DepositReceipt {
}
}
func genWithdrawalRequests(num int) []*enginev1.ExecutionLayerWithdrawalRequest {
wrs := make([]*enginev1.ExecutionLayerWithdrawalRequest, num)
func genWithdrawalRequests(num int) []*enginev1.WithdrawalRequest {
wrs := make([]*enginev1.WithdrawalRequest, num)
for i := 0; i < num; i++ {
wrs[i] = genWithdrawalRequest()
}
return wrs
}
func genWithdrawalRequest() *enginev1.ExecutionLayerWithdrawalRequest {
return &enginev1.ExecutionLayerWithdrawalRequest{
func genWithdrawalRequest() *enginev1.WithdrawalRequest {
return &enginev1.WithdrawalRequest{
SourceAddress: bytes(20),
ValidatorPubkey: bytes(48),
Amount: 55555,

View File

@@ -1,5 +1,5 @@
// Code generated by fastssz. DO NOT EDIT.
// Hash: 75d48d13b4efa7867468bae2df70b80e9606b9a44e621915d2093a4f20ae111f
// Hash: f1b72be7fea0fb1c1fbf8396b0e460f440ff5f7551fe46f0f9a5c7a4be634e1e
package eth
import (
@@ -17642,8 +17642,8 @@ func (b *BeaconStateElectra) MarshalSSZTo(buf []byte) (dst []byte, err error) {
dst = ssz.WriteOffset(dst, offset)
offset += len(b.HistoricalSummaries) * 64
// Field (28) 'DepositReceiptsStartIndex'
dst = ssz.MarshalUint64(dst, b.DepositReceiptsStartIndex)
// Field (28) 'DepositRequestsStartIndex'
dst = ssz.MarshalUint64(dst, b.DepositRequestsStartIndex)
// Field (29) 'DepositBalanceToConsume'
dst = ssz.MarshalUint64(dst, uint64(b.DepositBalanceToConsume))
@@ -17975,8 +17975,8 @@ func (b *BeaconStateElectra) UnmarshalSSZ(buf []byte) error {
return ssz.ErrOffset
}
// Field (28) 'DepositReceiptsStartIndex'
b.DepositReceiptsStartIndex = ssz.UnmarshallUint64(buf[2736653:2736661])
// Field (28) 'DepositRequestsStartIndex'
b.DepositRequestsStartIndex = ssz.UnmarshallUint64(buf[2736653:2736661])
// Field (29) 'DepositBalanceToConsume'
b.DepositBalanceToConsume = github_com_prysmaticlabs_prysm_v5_consensus_types_primitives.Gwei(ssz.UnmarshallUint64(buf[2736661:2736669]))
@@ -18563,8 +18563,8 @@ func (b *BeaconStateElectra) HashTreeRootWith(hh *ssz.Hasher) (err error) {
}
}
// Field (28) 'DepositReceiptsStartIndex'
hh.PutUint64(b.DepositReceiptsStartIndex)
// Field (28) 'DepositRequestsStartIndex'
hh.PutUint64(b.DepositRequestsStartIndex)
// Field (29) 'DepositBalanceToConsume'
hh.PutUint64(uint64(b.DepositBalanceToConsume))

View File

@@ -16,7 +16,6 @@ import (
v1alpha1 "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
_ "google.golang.org/protobuf/types/known/wrapperspb"
)
const (
@@ -673,9 +672,7 @@ var file_proto_prysm_v1alpha1_validator_client_keymanager_proto_rawDesc = []byte
0x2d, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x2f, 0x6b, 0x65, 0x79, 0x6d, 0x61, 0x6e, 0x61, 0x67,
0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x1e, 0x65, 0x74, 0x68, 0x65, 0x72, 0x65,
0x75, 0x6d, 0x2e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x2e, 0x61, 0x63, 0x63,
0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x77, 0x72, 0x61, 0x70, 0x70, 0x65,
0x72, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f,
0x6f, 0x75, 0x6e, 0x74, 0x73, 0x2e, 0x76, 0x32, 0x1a, 0x1b, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f,
0x65, 0x74, 0x68, 0x2f, 0x65, 0x78, 0x74, 0x2f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x26, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x72, 0x79,
0x73, 0x6d, 0x2f, 0x76, 0x31, 0x61, 0x6c, 0x70, 0x68, 0x61, 0x31, 0x2f, 0x61, 0x74, 0x74, 0x65,

View File

@@ -27,7 +27,7 @@ mainnet = {
"max_blob_commitments.size": "4096",
"kzg_commitment_inclusion_proof_depth.size": "17",
"max_withdrawal_requests_per_payload.size":"16",
"max_deposit_receipts": "8192",
"max_deposit_requests_per_payload.size": "8192",
"max_attesting_indices.size": "131072",
"max_committees_per_slot.size": "64",
"committee_bits.size": "8",
@@ -58,7 +58,7 @@ minimal = {
"max_blob_commitments.size": "16",
"kzg_commitment_inclusion_proof_depth.size": "9",
"max_withdrawal_requests_per_payload.size":"2",
"max_deposit_receipts": "4",
"max_deposit_requests_per_payload.size": "4",
"max_attesting_indices.size": "8192",
"max_committees_per_slot.size": "4",
"committee_bits.size": "1",

View File

@@ -6,6 +6,6 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/spectest/shared/electra/operations"
)
func TestMainnet_Electra_Operations_ExecutionLayerWithdrawalRequest(t *testing.T) {
operations.RunExecutionLayerWithdrawalRequestTest(t, "mainnet")
func TestMainnet_Electra_Operations_WithdrawalRequest(t *testing.T) {
operations.RunWithdrawalRequestTest(t, "mainnet")
}

View File

@@ -6,6 +6,6 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/spectest/shared/electra/operations"
)
func TestMinimal_Electra_Operations_ExecutionLayerWithdrawalRequest(t *testing.T) {
operations.RunExecutionLayerWithdrawalRequestTest(t, "minimal")
func TestMinimal_Electra_Operations_WithdrawalRequest(t *testing.T) {
operations.RunWithdrawalRequestTest(t, "minimal")
}

View File

@@ -9,13 +9,13 @@ go_library(
"block_header.go",
"bls_to_execution_changes.go",
"consolidations.go",
"deposit_receipt.go",
"execution_layer_withdrawal_request.go",
"deposit_request.go",
"execution_payload.go",
"helpers.go",
"proposer_slashing.go",
"sync_committee.go",
"voluntary_exit.go",
"withdrawal_request.go",
"withdrawals.go",
],
importpath = "github.com/prysmaticlabs/prysm/v5/testing/spectest/shared/electra/operations",

View File

@@ -16,7 +16,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/util"
)
func RunExecutionLayerWithdrawalRequestTest(t *testing.T, config string) {
func RunWithdrawalRequestTest(t *testing.T, config string) {
require.NoError(t, utils.SetConfig(t, config))
testFolders, testsFolderPath := utils.TestFolders(t, config, "electra", "operations/execution_layer_withdrawal_request/pyspec_tests")
if len(testFolders) == 0 {
@@ -25,14 +25,14 @@ func RunExecutionLayerWithdrawalRequestTest(t *testing.T, config string) {
for _, folder := range testFolders {
t.Run(folder.Name(), func(t *testing.T) {
folderPath := path.Join(testsFolderPath, folder.Name())
executionLayerWithdrawalRequestFile, err := util.BazelFileBytes(folderPath, "execution_layer_withdrawal_request.ssz_snappy")
withdrawalRequestFile, err := util.BazelFileBytes(folderPath, "execution_layer_withdrawal_request.ssz_snappy")
require.NoError(t, err)
executionLayerWithdrawalRequestSSZ, err := snappy.Decode(nil /* dst */, executionLayerWithdrawalRequestFile)
withdrawalRequestSSZ, err := snappy.Decode(nil /* dst */, withdrawalRequestFile)
require.NoError(t, err, "Failed to decompress")
withdrawalRequest := &enginev1.ExecutionLayerWithdrawalRequest{}
require.NoError(t, withdrawalRequest.UnmarshalSSZ(executionLayerWithdrawalRequestSSZ), "Failed to unmarshal")
withdrawalRequest := &enginev1.WithdrawalRequest{}
require.NoError(t, withdrawalRequest.UnmarshalSSZ(withdrawalRequestSSZ), "Failed to unmarshal")
body := &ethpb.BeaconBlockBodyElectra{ExecutionPayload: &enginev1.ExecutionPayloadElectra{
WithdrawalRequests: []*enginev1.ExecutionLayerWithdrawalRequest{
WithdrawalRequests: []*enginev1.WithdrawalRequest{
withdrawalRequest,
},
}}
@@ -43,7 +43,7 @@ func RunExecutionLayerWithdrawalRequestTest(t *testing.T, config string) {
require.NoError(t, err)
exe, ok := e.(interfaces.ExecutionDataElectra)
require.Equal(t, true, ok)
return electra.ProcessExecutionLayerWithdrawalRequests(ctx, s, exe.WithdrawalRequests())
return electra.ProcessWithdrawalRequests(ctx, s, exe.WithdrawalRequests())
})
})
}

View File

@@ -151,10 +151,10 @@ func UnmarshalledSSZ(t *testing.T, serializedBytes []byte, folderName string) (i
obj = &ethpb.SignedConsolidation{}
case "PendingConsolidation":
obj = &ethpb.PendingConsolidation{}
case "ExecutionLayerWithdrawalRequest":
obj = &enginev1.ExecutionLayerWithdrawalRequest{}
case "DepositReceipt":
obj = &enginev1.DepositReceipt{}
case "ExecutionLayerWithdrawalRequest": // TODO: update in spectest update
obj = &enginev1.WithdrawalRequest{}
case "DepositReceipt": // TODO: update in spectest update
obj = &enginev1.DepositRequest{}
default:
return nil, errors.New("type not found")
}

View File

@@ -1371,8 +1371,8 @@ func HydrateBeaconBlockBodyElectra(b *ethpb.BeaconBlockBodyElectra) *ethpb.Beaco
BlockHash: make([]byte, fieldparams.RootLength),
Transactions: make([][]byte, 0),
Withdrawals: make([]*enginev1.Withdrawal, 0),
DepositReceipts: make([]*enginev1.DepositReceipt, 0),
WithdrawalRequests: make([]*enginev1.ExecutionLayerWithdrawalRequest, 0),
DepositRequests: make([]*enginev1.DepositRequest, 0),
WithdrawalRequests: make([]*enginev1.WithdrawalRequest, 0),
}
}
return b
@@ -1578,7 +1578,7 @@ func HydrateBlindedBeaconBlockBodyElectra(b *ethpb.BlindedBeaconBlockBodyElectra
TransactionsRoot: make([]byte, fieldparams.RootLength),
WithdrawalsRoot: make([]byte, fieldparams.RootLength),
WithdrawalRequestsRoot: make([]byte, fieldparams.RootLength),
DepositReceiptsRoot: make([]byte, fieldparams.RootLength),
DepositRequestsRoot: make([]byte, fieldparams.RootLength),
}
}
return b

View File

@@ -114,7 +114,7 @@ func GenerateTestElectraBlockWithSidecar(t *testing.T, parent [32]byte, slot pri
Withdrawals: make([]*enginev1.Withdrawal, 0),
BlobGasUsed: 0,
ExcessBlobGas: 0,
DepositReceipts: generateTestDepositRequests(uint64(g.slot), 4),
DepositRequests: generateTestDepositRequests(uint64(g.slot), 4),
WithdrawalRequests: generateTestWithdrawalRequests(uint64(g.slot), 4),
}
}
@@ -171,11 +171,11 @@ func GenerateTestElectraBlockWithSidecar(t *testing.T, parent [32]byte, slot pri
return rob, sidecars
}
func generateTestDepositRequests(offset, n uint64) []*enginev1.DepositReceipt {
r := make([]*enginev1.DepositReceipt, n)
func generateTestDepositRequests(offset, n uint64) []*enginev1.DepositRequest {
r := make([]*enginev1.DepositRequest, n)
var i uint64
for i = 0; i < n; i++ {
r[i] = &enginev1.DepositReceipt{
r[i] = &enginev1.DepositRequest{
Pubkey: make([]byte, 48),
WithdrawalCredentials: make([]byte, 32),
Amount: offset + i,
@@ -186,11 +186,11 @@ func generateTestDepositRequests(offset, n uint64) []*enginev1.DepositReceipt {
return r
}
func generateTestWithdrawalRequests(offset, n uint64) []*enginev1.ExecutionLayerWithdrawalRequest {
r := make([]*enginev1.ExecutionLayerWithdrawalRequest, n)
func generateTestWithdrawalRequests(offset, n uint64) []*enginev1.WithdrawalRequest {
r := make([]*enginev1.WithdrawalRequest, n)
var i uint64
for i = 0; i < n; i++ {
r[i] = &enginev1.ExecutionLayerWithdrawalRequest{
r[i] = &enginev1.WithdrawalRequest{
SourceAddress: make([]byte, 20),
ValidatorPubkey: make([]byte, 48),
Amount: offset + i,

View File

@@ -191,7 +191,7 @@ func buildGenesisBeaconStateElectra(genesisTime uint64, preState state.BeaconSta
Eth1DepositIndex: preState.Eth1DepositIndex(),
// Electra Data
DepositReceiptsStartIndex: params.BeaconConfig().UnsetDepositReceiptsStartIndex,
DepositRequestsStartIndex: params.BeaconConfig().UnsetDepositRequestsStartIndex,
ExitBalanceToConsume: helpers.ActivationExitChurnLimit(primitives.Gwei(tab)),
EarliestConsolidationEpoch: helpers.ActivationExitEpoch(slots.ToEpoch(preState.Slot())),
ConsolidationBalanceToConsume: helpers.ConsolidationChurnLimit(primitives.Gwei(tab)),
@@ -224,8 +224,8 @@ func buildGenesisBeaconStateElectra(genesisTime uint64, preState state.BeaconSta
BlockHash: make([]byte, 32),
Transactions: make([][]byte, 0),
Withdrawals: make([]*enginev1.Withdrawal, 0),
DepositReceipts: make([]*enginev1.DepositReceipt, 0),
WithdrawalRequests: make([]*enginev1.ExecutionLayerWithdrawalRequest, 0),
DepositRequests: make([]*enginev1.DepositRequest, 0),
WithdrawalRequests: make([]*enginev1.WithdrawalRequest, 0),
},
}).HashTreeRoot()
if err != nil {
@@ -269,7 +269,7 @@ func buildGenesisBeaconStateElectra(genesisTime uint64, preState state.BeaconSta
BlockHash: make([]byte, 32),
TransactionsRoot: make([]byte, 32),
WithdrawalsRoot: make([]byte, 32),
DepositReceiptsRoot: make([]byte, 32),
DepositRequestsRoot: make([]byte, 32),
WithdrawalRequestsRoot: make([]byte, 32),
}

View File

@@ -436,7 +436,7 @@ func NewBeaconStateElectra(options ...func(state *ethpb.BeaconStateElectra) erro
BlockHash: make([]byte, 32),
TransactionsRoot: make([]byte, 32),
WithdrawalsRoot: make([]byte, 32),
DepositReceiptsRoot: make([]byte, 32),
DepositRequestsRoot: make([]byte, 32),
WithdrawalRequestsRoot: make([]byte, 32),
},
}