diff --git a/crypto/bls/aggregation.go b/crypto/bls/aggregation.go deleted file mode 100644 index d8f96d70..00000000 --- a/crypto/bls/aggregation.go +++ /dev/null @@ -1,232 +0,0 @@ -package bls - -import ( - "errors" - "math/big" - - "github.com/Layr-Labs/eigensdk-go/logging" - "github.com/Layr-Labs/eigensdk-go/utils" - "github.com/ethereum/go-ethereum/common/hexutil" -) - -const PercentMultiplier = 100 - -var ( - ErrPubKeysNotEqual = errors.New("public keys are not equal") - ErrInsufficientEthSigs = errors.New("insufficient eth signatures") - ErrAggSigNotValid = errors.New("aggregated signature is not valid") -) - -type SignerMessage struct { - Signature *Signature - Operator OperatorId - Err error -} - -type SignatureAggregation struct { - NonSigners []*G1Point - QuorumAggPubKeys []*G1Point - AggPubKey *G2Point - AggSignature *Signature -} - -// SignatureAggregator is an interface for aggregating the signatures returned by DA nodes so that they can be verified -// by the DA contract -type SignatureAggregator interface { - - // AggregateSignatures blocks until it recieves a response for each operator in the operator state via messageChan, - // and then returns the aggregated signature. - // If the aggregated signature is invalid, an error is returned. - AggregateSignatures( - state *IndexedOperatorState, - quorumParams []QuorumParam, - message [32]byte, - messageChan chan SignerMessage, - ) (*SignatureAggregation, error) -} - -type StdSignatureAggregator struct { - Logger logging.Logger -} - -func NewStdSignatureAggregator(logger logging.Logger) *StdSignatureAggregator { - return &StdSignatureAggregator{ - Logger: logger, - } -} - -var _ SignatureAggregator = (*StdSignatureAggregator)(nil) - -func (a *StdSignatureAggregator) AggregateSignatures( - state *IndexedOperatorState, - quorumParams []QuorumParam, - message [32]byte, - messageChan chan SignerMessage, -) (*SignatureAggregation, error) { - - // TODO: Add logging - - // Ensure all quorums are found in state - for _, quorum := range quorumParams { - _, found := state.Operators[quorum.QuorumID] - if !found { - return nil, errors.New("quorum not found") - } - } - - stakeSigned := make([]*big.Int, len(quorumParams)) - for ind := range quorumParams { - stakeSigned[ind] = big.NewInt(0) - } - aggSigs := make([]*Signature, len(quorumParams)) - aggPubKeys := make([]*G2Point, len(quorumParams)) - - signerMap := make(map[OperatorId]bool) - - // Aggregate Signatures - numOperators := len(state.IndexedOperators) - - for numReply := 0; numReply < numOperators; numReply++ { - r := <-messageChan - - if r.Err != nil { - a.Logger.Warn("Error returned from messageChan", "err", r.Err) - continue - } - - op, found := state.IndexedOperators[r.Operator] - if !found { - a.Logger.Error("Operator not found in state", "operator", r.Operator) - continue - } - - // Verify Signature - sig := r.Signature - ok, err := sig.Verify(op.PubkeyG2, message) - if err != nil { - a.Logger.Error("Error verifying signature", "err", err) - continue - } - if !ok { - a.Logger.Error("Signature is not valid", "pubkey", hexutil.Encode(op.PubkeyG2.Serialize())) - continue - } - - for ind, quorumParam := range quorumParams { - - // Get stake amounts for operator - ops := state.Operators[quorumParam.QuorumID] - opInfo, ok := ops[r.Operator] - - // If operator is not in quorum, skip - if !ok { - a.Logger.Error("Operator not found in quorum", "operator", r.Operator) - continue - } - - signerMap[r.Operator] = true - - // Add to stake signed - stakeSigned[ind].Add(stakeSigned[ind], opInfo.Stake) - - // Add to agg signature - if aggSigs[ind] == nil { - aggSigs[ind] = &Signature{sig.Deserialize(sig.Serialize())} - aggPubKeys[ind] = op.PubkeyG2.Deserialize(op.PubkeyG2.Serialize()) - } else { - aggSigs[ind].Add(sig) - aggPubKeys[ind].Add(op.PubkeyG2) - } - - } - - } - - // Aggregrate Non signer Pubkey Id - nonSignerKeys := make([]*G1Point, 0) - nonSignerOperatorIds := make([]OperatorId, 0) - - for id, op := range state.IndexedOperators { - _, found := signerMap[id] - if !found { - nonSignerKeys = append(nonSignerKeys, op.PubkeyG1) - nonSignerOperatorIds = append(nonSignerOperatorIds, id) - } - } - - quorumAggPubKeys := make([]*G1Point, len(quorumParams)) - - // Validate the amount signed and aggregate signatures for each quorum - for ind, quorum := range quorumParams { - - // Check that quorum has sufficient stake - threshold := GetStakeThreshold(state.OperatorState, quorum.QuorumID, quorum.QuorumThreshold) - if stakeSigned[ind].Cmp(threshold) == -1 { - return nil, ErrInsufficientEthSigs - } - - // Verify that the aggregated public key for the quorum matches the on-chain quorum aggregate public key sans - // non-signers of the quorum - quorumAggKey := state.AggKeys[quorum.QuorumID] - quorumAggPubKeys[ind] = quorumAggKey - - signersAggKey := quorumAggKey.Deserialize(quorumAggKey.Serialize()) - for opInd, nsk := range nonSignerKeys { - ops := state.Operators[quorum.QuorumID] - if _, ok := ops[nonSignerOperatorIds[opInd]]; ok { - signersAggKey.Sub(nsk) - } - } - - if aggPubKeys[ind] == nil { - return nil, ErrAggSigNotValid - } - - ok, err := signersAggKey.VerifyEquivalence(aggPubKeys[ind]) - if err != nil { - return nil, err - } - if !ok { - return nil, ErrPubKeysNotEqual - } - - // Verify the aggregated signature for the quorum - ok, err = aggSigs[ind].Verify(aggPubKeys[ind], message) - if err != nil { - return nil, err - } - if !ok { - return nil, ErrAggSigNotValid - } - - } - - // Aggregate the aggregated signatures. We reuse the first aggregated signature as the accumulator - for i := 1; i < len(aggSigs); i++ { - aggSigs[0].Add(aggSigs[i]) - } - - // Aggregate the aggregated public keys. We reuse the first aggregated public key as the accumulator - for i := 1; i < len(aggPubKeys); i++ { - aggPubKeys[0].Add(aggPubKeys[i]) - } - - return &SignatureAggregation{ - NonSigners: nonSignerKeys, - QuorumAggPubKeys: quorumAggPubKeys, - AggPubKey: aggPubKeys[0], - AggSignature: aggSigs[0], - }, nil - -} - -func GetStakeThreshold(state *OperatorState, quorum QuorumID, quorumThreshold uint8) *big.Int { - - // Get stake threshold - quorumThresholdBig := new(big.Int).SetUint64(uint64(quorumThreshold)) - stakeThreshold := new(big.Int) - stakeThreshold.Mul(quorumThresholdBig, state.Totals[quorum].Stake) - stakeThreshold = utils.RoundUpDivideBig(stakeThreshold, new(big.Int).SetUint64(PercentMultiplier)) - - return stakeThreshold -} diff --git a/crypto/bls/aggregation_test.go b/crypto/bls/aggregation_test.go deleted file mode 100644 index 85b45dda..00000000 --- a/crypto/bls/aggregation_test.go +++ /dev/null @@ -1,147 +0,0 @@ -package bls_test - -import ( - "errors" - "log" - "strconv" - "testing" - - loggingmock "github.com/Layr-Labs/eigensdk-go/logging/mocks" - "github.com/stretchr/testify/assert" - "go.uber.org/mock/gomock" - - "github.com/Layr-Labs/eigensdk-go/crypto/bls" - "github.com/Layr-Labs/eigensdk-go/crypto/bls/mock" -) - -func TestSignatureAggregation(t *testing.T) { - t.Parallel() - const NUM_OPERATORS = 10 - - type fields struct { - logger *loggingmock.MockLogger - } - - dat, err := mock.NewChainDataMock(NUM_OPERATORS) - if err != nil { - log.Fatal(err) - } - - var tests = map[string]struct { - quorums []bls.QuorumParam - message [32]byte - advCount uint - success bool - prepare func(f *fields) - }{ - "Succeeds when all operators sign at quorum threshold 100": { - quorums: []bls.QuorumParam{ - { - QuorumID: 0, - QuorumThreshold: 100, - }, - }, - message: [32]byte{1, 2, 3, 4, 5, 6}, - advCount: 0, - success: true, - }, - "Succeeds when 9 out of 10 operators sign at quorum threshold 80": { - quorums: []bls.QuorumParam{ - { - QuorumID: 0, - QuorumThreshold: 80, - }, - }, - message: [32]byte{1, 2, 3, 4, 5, 6}, - advCount: 1, - prepare: func(f *fields) { - // 1 node operators are adversarial so we expect 1 such warning - f.logger.EXPECT(). - Warn("Error returned from messageChan", "err", errors.New("adversary")) - }, - success: true, - }, - "Fails when 8 out of 10 operators sign at quorum threshold 90": { - quorums: []bls.QuorumParam{ - { - QuorumID: 0, - QuorumThreshold: 90, - }, - }, - message: [32]byte{1, 2, 3, 4, 5, 6}, - advCount: 2, - prepare: func(f *fields) { - // 2 node operators are adversarial so we expect 2 such warnings - f.logger.EXPECT(). - Warn("Error returned from messageChan", "err", errors.New("adversary")). - Times(2) - }, - success: false, - }, - } - - for name, test := range tests { - t.Run(name, func(t *testing.T) { - mockCtrl := gomock.NewController(t) - f := fields{ - logger: loggingmock.NewMockLogger(mockCtrl), - } - if test.prepare != nil { - test.prepare(&f) - } - - agg := bls.NewStdSignatureAggregator(f.logger) - state := dat.GetTotalOperatorState(0) - update := make(chan bls.SignerMessage) - - go simulateOperators(*state, test.message, update, test.advCount) - - _, err := agg.AggregateSignatures(state.IndexedOperatorState, test.quorums, test.message, update) - if test.success { - assert.Nil(t, err) - } else { - assert.NotNil(t, err) - } - }) - } -} - -func makeOperatorId(id int) bls.OperatorId { - data := [32]byte{} - copy(data[:], []byte(strconv.FormatInt(int64(id), 10))) - return data -} - -func simulateOperators( - state mock.PrivateOperatorState, - message [32]byte, - update chan bls.SignerMessage, - advCount uint, -) { - - count := 0 - - // Simulate the operators signing the message. - // In real life, the ordering will be random, but we simulate the signing in a fixed order - // to simulate stakes deterministically - for i := 0; i < len(state.PrivateOperators); i++ { - id := makeOperatorId(i) - op := state.PrivateOperators[id] - sig := op.KeyPair.SignMessage(message) - if count < len(state.IndexedOperators)-int(advCount) { - update <- bls.SignerMessage{ - Signature: sig, - Operator: id, - Err: nil, - } - } else { - update <- bls.SignerMessage{ - Signature: nil, - Operator: id, - Err: errors.New("adversary"), - } - } - - count += 1 - } -} diff --git a/crypto/bls/attestation.go b/crypto/bls/attestation.go index 5341b220..2c80b4ad 100644 --- a/crypto/bls/attestation.go +++ b/crypto/bls/attestation.go @@ -14,7 +14,6 @@ import ( "github.com/consensys/gnark-crypto/ecc/bn254/fp" "github.com/consensys/gnark-crypto/ecc/bn254/fr" "github.com/ethereum/go-ethereum/accounts/keystore" - "github.com/ethereum/go-ethereum/crypto" ) // We are using similar structure for saving bls keys as ethereum keystore @@ -141,12 +140,6 @@ func (s *Signature) Verify(pubkey *G2Point, message [32]byte) (bool, error) { return ok, nil } -func (p *G1Point) GetOperatorID() OperatorId { - x := p.X.BigInt(new(big.Int)) - y := p.Y.BigInt(new(big.Int)) - return crypto.Keccak256Hash(append(x.Bytes(), y.Bytes()...)) -} - type PrivateKey = fr.Element func NewPrivateKey(sk string) (*PrivateKey, error) { @@ -281,7 +274,3 @@ func (k *KeyPair) GetPubKeyG2() *G2Point { func (k *KeyPair) GetPubKeyG1() *G1Point { return k.PubKey } - -func (k *KeyPair) GetOperatorID() OperatorId { - return k.PubKey.GetOperatorID() -} diff --git a/crypto/bls/data.go b/crypto/bls/data.go deleted file mode 100644 index 5cdcf693..00000000 --- a/crypto/bls/data.go +++ /dev/null @@ -1,31 +0,0 @@ -package bls - -// TODO: move these somewhere more appropriate. -// -// these are avs types, nothing to do with bls -// -// also there's already an OperatorId in types, but we can't use it here because it would form a circular import -// since types using bls.Signature.. we prob just need to move all of these types to types/avs.go or something -type OperatorId = [32]byte -type OperatorIndex uint - -// Security and Quorum Parameters - -// QuorumID is a unique identifier for a quorum; initially EigenDA will support up to 256 quorums -type QuorumID uint8 - -// SecurityParam contains the quorum ID and the adversary threshold for the quorum; -type SecurityParam struct { - QuorumID QuorumID - // AdversaryThreshold is the maximum amount of stake that can be controlled by an adversary in the quorum as a - // percentage of the total stake in the quorum - AdversaryThreshold uint8 -} - -// QuorumParam contains the quorum ID and the quorum threshold for the quorum -type QuorumParam struct { - QuorumID QuorumID - // QuorumThreshold is the amount of stake that must sign a message for it to be considered valid as a percentage of - // the total stake in the quorum - QuorumThreshold uint8 -} diff --git a/crypto/bls/mock/state.go b/crypto/bls/mock/state.go deleted file mode 100644 index 365af6a9..00000000 --- a/crypto/bls/mock/state.go +++ /dev/null @@ -1,177 +0,0 @@ -package mock - -import ( - "context" - "fmt" - "math/big" - - "github.com/Layr-Labs/eigensdk-go/crypto/bls" -) - -type ChainDataMock struct { - KeyPairs []*bls.KeyPair - NumOperators bls.OperatorIndex -} - -var _ bls.ChainState = (*ChainDataMock)(nil) -var _ bls.IndexedChainState = (*ChainDataMock)(nil) - -type PrivateOperatorInfo struct { - *bls.OperatorInfo - *bls.IndexedOperatorInfo - KeyPair *bls.KeyPair - Host string - Port string -} - -type PrivateOperatorState struct { - *bls.OperatorState - *bls.IndexedOperatorState - PrivateOperators map[bls.OperatorId]*PrivateOperatorInfo -} - -func makeOperatorId(id int) bls.OperatorId { - data := [32]byte{} - copy(data[:], []byte(fmt.Sprintf("%d", id))) - return data -} - -func NewChainDataMock(numOperators bls.OperatorIndex) (*ChainDataMock, error) { - - keyPairs := make([]*bls.KeyPair, numOperators) - for ind := bls.OperatorIndex(0); ind < numOperators; ind++ { - keyPair, err := bls.GenRandomBlsKeys() - if err != nil { - return nil, err - } - keyPairs[ind] = keyPair - } - - return &ChainDataMock{ - NumOperators: numOperators, - KeyPairs: keyPairs, - }, nil -} - -func (d *ChainDataMock) GetTotalOperatorState(blockNumber uint) *PrivateOperatorState { - - indexedOperators := make(map[bls.OperatorId]*bls.IndexedOperatorInfo, d.NumOperators) - storedOperators := make(map[bls.OperatorId]*bls.OperatorInfo) - privateOperators := make(map[bls.OperatorId]*PrivateOperatorInfo, d.NumOperators) - - var aggPubKey *bls.G1Point - - quorumStake := 0 - - for ind := bls.OperatorIndex(0); ind < d.NumOperators; ind++ { - - if ind == 0 { - key := d.KeyPairs[ind].GetPubKeyG1() - aggPubKey = key.Deserialize(key.Serialize()) - } else { - aggPubKey.Add(d.KeyPairs[ind].GetPubKeyG1()) - } - - stake := ind + 1 - - host := "0.0.0.0" - port := fmt.Sprintf("3%03v", int(ind)) - - stored := &bls.OperatorInfo{ - Stake: big.NewInt(int64(stake)), - Index: ind, - } - - indexed := &bls.IndexedOperatorInfo{ - Socket: fmt.Sprintf("%v:%v", host, port), - PubkeyG1: d.KeyPairs[ind].GetPubKeyG1(), - PubkeyG2: d.KeyPairs[ind].GetPubKeyG2(), - } - - private := &PrivateOperatorInfo{ - OperatorInfo: stored, - IndexedOperatorInfo: indexed, - KeyPair: d.KeyPairs[ind], - Host: host, - Port: port, - } - - id := makeOperatorId(int(ind)) - storedOperators[id] = stored - indexedOperators[id] = indexed - privateOperators[id] = private - - quorumStake += int(stake) - - } - - totals := map[bls.QuorumID]*bls.OperatorInfo{ - 0: { - Stake: big.NewInt(int64(quorumStake)), - Index: d.NumOperators, - }, - } - - operatorState := &bls.OperatorState{ - Operators: map[bls.QuorumID]map[bls.OperatorId]*bls.OperatorInfo{ - 0: storedOperators, - }, - Totals: totals, - BlockNumber: blockNumber, - } - - indexedState := &bls.IndexedOperatorState{ - OperatorState: operatorState, - IndexedOperators: indexedOperators, - AggKeys: map[bls.QuorumID]*bls.G1Point{ - 0: aggPubKey, - }, - } - - privateOperatorState := &PrivateOperatorState{ - OperatorState: operatorState, - IndexedOperatorState: indexedState, - PrivateOperators: privateOperators, - } - - return privateOperatorState - -} - -func (d *ChainDataMock) GetOperatorState(blockNumber uint, quorums []bls.QuorumID) (*bls.OperatorState, error) { - - state := d.GetTotalOperatorState(blockNumber) - - return state.OperatorState, nil - -} - -func (d *ChainDataMock) GetOperatorStateByOperator( - blockNumber uint, - operator bls.OperatorId, -) (*bls.OperatorState, error) { - - state := d.GetTotalOperatorState(blockNumber) - - return state.OperatorState, nil - -} - -func (d *ChainDataMock) GetIndexedOperatorState( - blockNumber uint, - quorums []bls.QuorumID, -) (*bls.IndexedOperatorState, error) { - - state := d.GetTotalOperatorState(blockNumber) - - return state.IndexedOperatorState, nil - -} - -func (d *ChainDataMock) GetCurrentBlockNumber() (uint, error) { - return 0, nil -} - -func (d *ChainDataMock) Start(context.Context) error { - return nil -} diff --git a/crypto/bls/state.go b/crypto/bls/state.go deleted file mode 100644 index c879990d..00000000 --- a/crypto/bls/state.go +++ /dev/null @@ -1,71 +0,0 @@ -package bls - -// TODO: put in proper module later - -import ( - "context" - "math/big" -) - -// Operators - -type StakeAmount *big.Int - -// OperatorInfo contains information about an operator which is stored on the blockchain state, -// corresponding to a particular quorum -type OperatorInfo struct { - // Stake is the amount of stake held by the operator in the quorum - Stake StakeAmount - // Index is the index of the operator within the quorum - Index OperatorIndex -} - -// OperatorState contains information about the current state of operators which is stored in the blockchain state -type OperatorState struct { - // Operators is a map from quorum ID to a map from the operators in that quorum to their StoredOperatorInfo. - // Membership - // in the map implies membership in the quorum. - Operators map[QuorumID]map[OperatorId]*OperatorInfo - // Totals is a map from quorum ID to the total stake (Stake) and total count (Index) of all operators in that quorum - Totals map[QuorumID]*OperatorInfo - // BlockNumber is the block number at which this state was retrieved - BlockNumber uint -} - -// IndexedOperatorInfo contains information about an operator which is contained in events from the EigenDA smart -// contracts. Note that -// this information does not depend on the quorum. -type IndexedOperatorInfo struct { - // PubKeyG1 and PubKeyG2 are the public keys of the operator, which are retrieved from the BlsApkRegistry - // smart contract - PubkeyG1 *G1Point - PubkeyG2 *G2Point - // Socket is the socket address of the operator, in the form "host:port" - Socket string -} - -// IndexedOperatorState contains information about the current state of operators which is contained in events from the -// EigenDA smart contracts, -// in addition to the information contained in OperatorState -type IndexedOperatorState struct { - *OperatorState - // IndexedOperators is a map from operator ID to the IndexedOperatorInfo for that operator. - IndexedOperators map[OperatorId]*IndexedOperatorInfo - // AggKeys is a map from quorum ID to the aggregate public key of the operators in that quorum - AggKeys map[QuorumID]*G1Point -} - -// ChainState is an interface for getting information about the current chain state. -type ChainState interface { - GetCurrentBlockNumber() (uint, error) - GetOperatorState(blockNumber uint, quorums []QuorumID) (*OperatorState, error) - GetOperatorStateByOperator(blockNumber uint, operator OperatorId) (*OperatorState, error) - // GetOperatorQuorums(blockNumber uint, operator OperatorId) ([]uint, error) -} - -// ChainState is an interface for getting information about the current chain state. -type IndexedChainState interface { - ChainState - GetIndexedOperatorState(blockNumber uint, quorums []QuorumID) (*IndexedOperatorState, error) - Start(context context.Context) error -} diff --git a/services/bls_aggregation/blsagg.go b/services/bls_aggregation/blsagg.go index 51e2ff90..3ac6168e 100644 --- a/services/bls_aggregation/blsagg.go +++ b/services/bls_aggregation/blsagg.go @@ -90,7 +90,7 @@ type BlsAggregationService interface { taskIndex types.TaskIndex, taskResponseDigest types.TaskResponseDigest, blsSignature *bls.Signature, - operatorId bls.OperatorId, + operatorId types.OperatorId, ) error // GetResponseChannel returns the single channel that meant to be used as the response channel @@ -172,7 +172,7 @@ func (a *BlsAggregatorService) ProcessNewSignature( taskIndex types.TaskIndex, taskResponseDigest types.TaskResponseDigest, blsSignature *bls.Signature, - operatorId bls.OperatorId, + operatorId types.OperatorId, ) error { a.taskChansMutex.Lock() taskC, taskInitialized := a.signedTaskRespsCs[taskIndex] diff --git a/services/mocks/blsagg/blsaggregation.go b/services/mocks/blsagg/blsaggregation.go index da29cbf1..a2c3f629 100644 --- a/services/mocks/blsagg/blsaggregation.go +++ b/services/mocks/blsagg/blsaggregation.go @@ -72,7 +72,7 @@ func (mr *MockBlsAggregationServiceMockRecorder) InitializeNewTask(arg0, arg1, a } // ProcessNewSignature mocks base method. -func (m *MockBlsAggregationService) ProcessNewSignature(arg0 context.Context, arg1 uint32, arg2 types.Bytes32, arg3 *bls.Signature, arg4 [32]byte) error { +func (m *MockBlsAggregationService) ProcessNewSignature(arg0 context.Context, arg1 uint32, arg2 types.Bytes32, arg3 *bls.Signature, arg4 types.Bytes32) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ProcessNewSignature", arg0, arg1, arg2, arg3, arg4) ret0, _ := ret[0].(error) diff --git a/types/operator.go b/types/operator.go index a2cd0687..4ff9f712 100644 --- a/types/operator.go +++ b/types/operator.go @@ -11,6 +11,8 @@ import ( "github.com/Layr-Labs/eigensdk-go/crypto/bls" "github.com/Layr-Labs/eigensdk-go/utils" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/math" + "github.com/ethereum/go-ethereum/crypto" ) const ( @@ -88,8 +90,19 @@ type OperatorAddr = common.Address type StakeAmount = *big.Int // OperatorId is the ID of an operator, defined by the AVS registry -// It is currently the hash of the operator's G1 pubkey (in the bls pubkey registry) +// It is the hash of the operator's G1 pubkey type OperatorId = Bytes32 + +func OperatorIdFromPubkey(pubkey *bls.G1Point) OperatorId { + x := pubkey.X.BigInt(new(big.Int)) + y := pubkey.Y.BigInt(new(big.Int)) + return OperatorId(crypto.Keccak256Hash(append(math.U256Bytes(x), math.U256Bytes(y)...))) +} + +func OperatorIdFromKeyPair(keyPair *bls.KeyPair) OperatorId { + return OperatorIdFromPubkey(keyPair.GetPubKeyG1()) +} + type QuorumNums []QuorumNum func (q QuorumNums) LogValue() slog.Value {