Browse Source

privval: duplicate SecretConnection from p2p package (#5672)

This is so that the `privval` package will not be affected when we refactor `p2p` (#5670). We will be migrating to gRPC shortly (#4698).
pull/5676/head
Erik Grinaker 3 years ago
committed by GitHub
parent
commit
2d0fcf498d
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 759 additions and 52 deletions
  1. +469
    -0
      privval/secret_connection.go
  2. +1
    -2
      privval/socket_dialers.go
  3. +1
    -2
      privval/socket_listeners.go
  4. +280
    -48
      proto/tendermint/privval/types.pb.go
  5. +8
    -0
      proto/tendermint/privval/types.proto

+ 469
- 0
privval/secret_connection.go View File

@ -0,0 +1,469 @@
package privval
import (
"bytes"
"crypto/cipher"
crand "crypto/rand"
"crypto/sha256"
"encoding/binary"
"errors"
"fmt"
"io"
"math"
"net"
"time"
gogotypes "github.com/gogo/protobuf/types"
"github.com/gtank/merlin"
pool "github.com/libp2p/go-buffer-pool"
"golang.org/x/crypto/chacha20poly1305"
"golang.org/x/crypto/curve25519"
"golang.org/x/crypto/hkdf"
"golang.org/x/crypto/nacl/box"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
cryptoenc "github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/libs/async"
"github.com/tendermint/tendermint/libs/protoio"
tmsync "github.com/tendermint/tendermint/libs/sync"
tmprivval "github.com/tendermint/tendermint/proto/tendermint/privval"
)
// This code has been duplicated from p2p/conn prior to the P2P refactor.
// It is left here temporarily until we migrate privval to gRPC.
// https://github.com/tendermint/tendermint/issues/4698
// 4 + 1024 == 1028 total frame size
const (
dataLenSize = 4
dataMaxSize = 1024
totalFrameSize = dataMaxSize + dataLenSize
aeadSizeOverhead = 16 // overhead of poly 1305 authentication tag
aeadKeySize = chacha20poly1305.KeySize
aeadNonceSize = chacha20poly1305.NonceSize
)
var (
ErrSmallOrderRemotePubKey = errors.New("detected low order point from remote peer")
labelEphemeralLowerPublicKey = []byte("EPHEMERAL_LOWER_PUBLIC_KEY")
labelEphemeralUpperPublicKey = []byte("EPHEMERAL_UPPER_PUBLIC_KEY")
labelDHSecret = []byte("DH_SECRET")
labelSecretConnectionMac = []byte("SECRET_CONNECTION_MAC")
secretConnKeyAndChallengeGen = []byte("TENDERMINT_SECRET_CONNECTION_KEY_AND_CHALLENGE_GEN")
)
// SecretConnection implements net.Conn.
// It is an implementation of the STS protocol.
// See https://github.com/tendermint/tendermint/blob/0.1/docs/sts-final.pdf for
// details on the protocol.
//
// Consumers of the SecretConnection are responsible for authenticating
// the remote peer's pubkey against known information, like a nodeID.
// Otherwise they are vulnerable to MITM.
// (TODO(ismail): see also https://github.com/tendermint/tendermint/issues/3010)
type SecretConnection struct {
// immutable
recvAead cipher.AEAD
sendAead cipher.AEAD
remPubKey crypto.PubKey
conn io.ReadWriteCloser
// net.Conn must be thread safe:
// https://golang.org/pkg/net/#Conn.
// Since we have internal mutable state,
// we need mtxs. But recv and send states
// are independent, so we can use two mtxs.
// All .Read are covered by recvMtx,
// all .Write are covered by sendMtx.
recvMtx tmsync.Mutex
recvBuffer []byte
recvNonce *[aeadNonceSize]byte
sendMtx tmsync.Mutex
sendNonce *[aeadNonceSize]byte
}
// MakeSecretConnection performs handshake and returns a new authenticated
// SecretConnection.
// Returns nil if there is an error in handshake.
// Caller should call conn.Close()
// See docs/sts-final.pdf for more information.
func MakeSecretConnection(conn io.ReadWriteCloser, locPrivKey crypto.PrivKey) (*SecretConnection, error) {
var (
locPubKey = locPrivKey.PubKey()
)
// Generate ephemeral keys for perfect forward secrecy.
locEphPub, locEphPriv := genEphKeys()
// Write local ephemeral pubkey and receive one too.
// NOTE: every 32-byte string is accepted as a Curve25519 public key (see
// DJB's Curve25519 paper: http://cr.yp.to/ecdh/curve25519-20060209.pdf)
remEphPub, err := shareEphPubKey(conn, locEphPub)
if err != nil {
return nil, err
}
// Sort by lexical order.
loEphPub, hiEphPub := sort32(locEphPub, remEphPub)
transcript := merlin.NewTranscript("TENDERMINT_SECRET_CONNECTION_TRANSCRIPT_HASH")
transcript.AppendMessage(labelEphemeralLowerPublicKey, loEphPub[:])
transcript.AppendMessage(labelEphemeralUpperPublicKey, hiEphPub[:])
// Check if the local ephemeral public key was the least, lexicographically
// sorted.
locIsLeast := bytes.Equal(locEphPub[:], loEphPub[:])
// Compute common diffie hellman secret using X25519.
dhSecret, err := computeDHSecret(remEphPub, locEphPriv)
if err != nil {
return nil, err
}
transcript.AppendMessage(labelDHSecret, dhSecret[:])
// Generate the secret used for receiving, sending, challenge via HKDF-SHA2
// on the transcript state (which itself also uses HKDF-SHA2 to derive a key
// from the dhSecret).
recvSecret, sendSecret := deriveSecrets(dhSecret, locIsLeast)
const challengeSize = 32
var challenge [challengeSize]byte
challengeSlice := transcript.ExtractBytes(labelSecretConnectionMac, challengeSize)
copy(challenge[:], challengeSlice[0:challengeSize])
sendAead, err := chacha20poly1305.New(sendSecret[:])
if err != nil {
return nil, errors.New("invalid send SecretConnection Key")
}
recvAead, err := chacha20poly1305.New(recvSecret[:])
if err != nil {
return nil, errors.New("invalid receive SecretConnection Key")
}
sc := &SecretConnection{
conn: conn,
recvBuffer: nil,
recvNonce: new([aeadNonceSize]byte),
sendNonce: new([aeadNonceSize]byte),
recvAead: recvAead,
sendAead: sendAead,
}
// Sign the challenge bytes for authentication.
locSignature, err := signChallenge(&challenge, locPrivKey)
if err != nil {
return nil, err
}
// Share (in secret) each other's pubkey & challenge signature
authSigMsg, err := shareAuthSignature(sc, locPubKey, locSignature)
if err != nil {
return nil, err
}
remPubKey, remSignature := authSigMsg.Key, authSigMsg.Sig
if _, ok := remPubKey.(ed25519.PubKey); !ok {
return nil, fmt.Errorf("expected ed25519 pubkey, got %T", remPubKey)
}
if !remPubKey.VerifySignature(challenge[:], remSignature) {
return nil, errors.New("challenge verification failed")
}
// We've authorized.
sc.remPubKey = remPubKey
return sc, nil
}
// RemotePubKey returns authenticated remote pubkey
func (sc *SecretConnection) RemotePubKey() crypto.PubKey {
return sc.remPubKey
}
// Writes encrypted frames of `totalFrameSize + aeadSizeOverhead`.
// CONTRACT: data smaller than dataMaxSize is written atomically.
func (sc *SecretConnection) Write(data []byte) (n int, err error) {
sc.sendMtx.Lock()
defer sc.sendMtx.Unlock()
for 0 < len(data) {
if err := func() error {
var sealedFrame = pool.Get(aeadSizeOverhead + totalFrameSize)
var frame = pool.Get(totalFrameSize)
defer func() {
pool.Put(sealedFrame)
pool.Put(frame)
}()
var chunk []byte
if dataMaxSize < len(data) {
chunk = data[:dataMaxSize]
data = data[dataMaxSize:]
} else {
chunk = data
data = nil
}
chunkLength := len(chunk)
binary.LittleEndian.PutUint32(frame, uint32(chunkLength))
copy(frame[dataLenSize:], chunk)
// encrypt the frame
sc.sendAead.Seal(sealedFrame[:0], sc.sendNonce[:], frame, nil)
incrNonce(sc.sendNonce)
// end encryption
_, err = sc.conn.Write(sealedFrame)
if err != nil {
return err
}
n += len(chunk)
return nil
}(); err != nil {
return n, err
}
}
return n, err
}
// CONTRACT: data smaller than dataMaxSize is read atomically.
func (sc *SecretConnection) Read(data []byte) (n int, err error) {
sc.recvMtx.Lock()
defer sc.recvMtx.Unlock()
// read off and update the recvBuffer, if non-empty
if 0 < len(sc.recvBuffer) {
n = copy(data, sc.recvBuffer)
sc.recvBuffer = sc.recvBuffer[n:]
return
}
// read off the conn
var sealedFrame = pool.Get(aeadSizeOverhead + totalFrameSize)
defer pool.Put(sealedFrame)
_, err = io.ReadFull(sc.conn, sealedFrame)
if err != nil {
return
}
// decrypt the frame.
// reads and updates the sc.recvNonce
var frame = pool.Get(totalFrameSize)
defer pool.Put(frame)
_, err = sc.recvAead.Open(frame[:0], sc.recvNonce[:], sealedFrame, nil)
if err != nil {
return n, fmt.Errorf("failed to decrypt SecretConnection: %w", err)
}
incrNonce(sc.recvNonce)
// end decryption
// copy checkLength worth into data,
// set recvBuffer to the rest.
var chunkLength = binary.LittleEndian.Uint32(frame) // read the first four bytes
if chunkLength > dataMaxSize {
return 0, errors.New("chunkLength is greater than dataMaxSize")
}
var chunk = frame[dataLenSize : dataLenSize+chunkLength]
n = copy(data, chunk)
if n < len(chunk) {
sc.recvBuffer = make([]byte, len(chunk)-n)
copy(sc.recvBuffer, chunk[n:])
}
return n, err
}
// Implements net.Conn
// nolint
func (sc *SecretConnection) Close() error { return sc.conn.Close() }
func (sc *SecretConnection) LocalAddr() net.Addr { return sc.conn.(net.Conn).LocalAddr() }
func (sc *SecretConnection) RemoteAddr() net.Addr { return sc.conn.(net.Conn).RemoteAddr() }
func (sc *SecretConnection) SetDeadline(t time.Time) error { return sc.conn.(net.Conn).SetDeadline(t) }
func (sc *SecretConnection) SetReadDeadline(t time.Time) error {
return sc.conn.(net.Conn).SetReadDeadline(t)
}
func (sc *SecretConnection) SetWriteDeadline(t time.Time) error {
return sc.conn.(net.Conn).SetWriteDeadline(t)
}
func genEphKeys() (ephPub, ephPriv *[32]byte) {
var err error
// TODO: Probably not a problem but ask Tony: different from the rust implementation (uses x25519-dalek),
// we do not "clamp" the private key scalar:
// see: https://github.com/dalek-cryptography/x25519-dalek/blob/34676d336049df2bba763cc076a75e47ae1f170f/src/x25519.rs#L56-L74
ephPub, ephPriv, err = box.GenerateKey(crand.Reader)
if err != nil {
panic("Could not generate ephemeral key-pair")
}
return
}
func shareEphPubKey(conn io.ReadWriter, locEphPub *[32]byte) (remEphPub *[32]byte, err error) {
// Send our pubkey and receive theirs in tandem.
var trs, _ = async.Parallel(
func(_ int) (val interface{}, abort bool, err error) {
lc := *locEphPub
_, err = protoio.NewDelimitedWriter(conn).WriteMsg(&gogotypes.BytesValue{Value: lc[:]})
if err != nil {
return nil, true, err // abort
}
return nil, false, nil
},
func(_ int) (val interface{}, abort bool, err error) {
var bytes gogotypes.BytesValue
err = protoio.NewDelimitedReader(conn, 1024*1024).ReadMsg(&bytes)
if err != nil {
return nil, true, err // abort
}
var _remEphPub [32]byte
copy(_remEphPub[:], bytes.Value)
return _remEphPub, false, nil
},
)
// If error:
if trs.FirstError() != nil {
err = trs.FirstError()
return
}
// Otherwise:
var _remEphPub = trs.FirstValue().([32]byte)
return &_remEphPub, nil
}
func deriveSecrets(
dhSecret *[32]byte,
locIsLeast bool,
) (recvSecret, sendSecret *[aeadKeySize]byte) {
hash := sha256.New
hkdf := hkdf.New(hash, dhSecret[:], nil, secretConnKeyAndChallengeGen)
// get enough data for 2 aead keys, and a 32 byte challenge
res := new([2*aeadKeySize + 32]byte)
_, err := io.ReadFull(hkdf, res[:])
if err != nil {
panic(err)
}
recvSecret = new([aeadKeySize]byte)
sendSecret = new([aeadKeySize]byte)
// bytes 0 through aeadKeySize - 1 are one aead key.
// bytes aeadKeySize through 2*aeadKeySize -1 are another aead key.
// which key corresponds to sending and receiving key depends on whether
// the local key is less than the remote key.
if locIsLeast {
copy(recvSecret[:], res[0:aeadKeySize])
copy(sendSecret[:], res[aeadKeySize:aeadKeySize*2])
} else {
copy(sendSecret[:], res[0:aeadKeySize])
copy(recvSecret[:], res[aeadKeySize:aeadKeySize*2])
}
return
}
// computeDHSecret computes a Diffie-Hellman shared secret key
// from our own local private key and the other's public key.
func computeDHSecret(remPubKey, locPrivKey *[32]byte) (*[32]byte, error) {
shrKey, err := curve25519.X25519(locPrivKey[:], remPubKey[:])
if err != nil {
return nil, err
}
var shrKeyArray [32]byte
copy(shrKeyArray[:], shrKey)
return &shrKeyArray, nil
}
func sort32(foo, bar *[32]byte) (lo, hi *[32]byte) {
if bytes.Compare(foo[:], bar[:]) < 0 {
lo = foo
hi = bar
} else {
lo = bar
hi = foo
}
return
}
func signChallenge(challenge *[32]byte, locPrivKey crypto.PrivKey) ([]byte, error) {
signature, err := locPrivKey.Sign(challenge[:])
if err != nil {
return nil, err
}
return signature, nil
}
type authSigMessage struct {
Key crypto.PubKey
Sig []byte
}
func shareAuthSignature(sc io.ReadWriter, pubKey crypto.PubKey, signature []byte) (recvMsg authSigMessage, err error) {
// Send our info and receive theirs in tandem.
var trs, _ = async.Parallel(
func(_ int) (val interface{}, abort bool, err error) {
pbpk, err := cryptoenc.PubKeyToProto(pubKey)
if err != nil {
return nil, true, err
}
_, err = protoio.NewDelimitedWriter(sc).WriteMsg(&tmprivval.AuthSigMessage{PubKey: pbpk, Sig: signature})
if err != nil {
return nil, true, err // abort
}
return nil, false, nil
},
func(_ int) (val interface{}, abort bool, err error) {
var pba tmprivval.AuthSigMessage
err = protoio.NewDelimitedReader(sc, 1024*1024).ReadMsg(&pba)
if err != nil {
return nil, true, err // abort
}
pk, err := cryptoenc.PubKeyFromProto(pba.PubKey)
if err != nil {
return nil, true, err // abort
}
_recvMsg := authSigMessage{
Key: pk,
Sig: pba.Sig,
}
return _recvMsg, false, nil
},
)
// If error:
if trs.FirstError() != nil {
err = trs.FirstError()
return
}
var _recvMsg = trs.FirstValue().(authSigMessage)
return _recvMsg, nil
}
//--------------------------------------------------------------------------------
// Increment nonce little-endian by 1 with wraparound.
// Due to chacha20poly1305 expecting a 12 byte nonce we do not use the first four
// bytes. We only increment a 64 bit unsigned int in the remaining 8 bytes
// (little-endian in nonce[4:]).
func incrNonce(nonce *[aeadNonceSize]byte) {
counter := binary.LittleEndian.Uint64(nonce[4:])
if counter == math.MaxUint64 {
// Terminates the session and makes sure the nonce would not re-used.
// See https://github.com/tendermint/tendermint/issues/3531
panic("can't increase nonce without overflow")
}
counter++
binary.LittleEndian.PutUint64(nonce[4:], counter)
}

+ 1
- 2
privval/socket_dialers.go View File

@ -7,7 +7,6 @@ import (
"github.com/tendermint/tendermint/crypto"
tmnet "github.com/tendermint/tendermint/libs/net"
p2pconn "github.com/tendermint/tendermint/p2p/conn"
)
// Socket errors.
@ -28,7 +27,7 @@ func DialTCPFn(addr string, timeoutReadWrite time.Duration, privKey crypto.PrivK
err = conn.SetDeadline(deadline)
}
if err == nil {
conn, err = p2pconn.MakeSecretConnection(conn, privKey)
conn, err = MakeSecretConnection(conn, privKey)
}
return conn, err
}


+ 1
- 2
privval/socket_listeners.go View File

@ -5,7 +5,6 @@ import (
"time"
"github.com/tendermint/tendermint/crypto/ed25519"
p2pconn "github.com/tendermint/tendermint/p2p/conn"
)
const (
@ -76,7 +75,7 @@ func (ln *TCPListener) Accept() (net.Conn, error) {
// Wrap the conn in our timeout and encryption wrappers
timeoutConn := newTimeoutConn(tc, ln.timeoutReadWrite)
secretConn, err := p2pconn.MakeSecretConnection(timeoutConn, ln.secretConnKey)
secretConn, err := MakeSecretConnection(timeoutConn, ln.secretConnKey)
if err != nil {
return nil, err
}


+ 280
- 48
proto/tendermint/privval/types.pb.go View File

@ -661,6 +661,61 @@ func (*Message) XXX_OneofWrappers() []interface{} {
}
}
// AuthSigMessage is duplicated from p2p prior to the P2P refactor.
// It is used for the SecretConnection until we migrate privval to gRPC.
// https://github.com/tendermint/tendermint/issues/4698
type AuthSigMessage struct {
PubKey crypto.PublicKey `protobuf:"bytes,1,opt,name=pub_key,json=pubKey,proto3" json:"pub_key"`
Sig []byte `protobuf:"bytes,2,opt,name=sig,proto3" json:"sig,omitempty"`
}
func (m *AuthSigMessage) Reset() { *m = AuthSigMessage{} }
func (m *AuthSigMessage) String() string { return proto.CompactTextString(m) }
func (*AuthSigMessage) ProtoMessage() {}
func (*AuthSigMessage) Descriptor() ([]byte, []int) {
return fileDescriptor_cb4e437a5328cf9c, []int{10}
}
func (m *AuthSigMessage) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
}
func (m *AuthSigMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
if deterministic {
return xxx_messageInfo_AuthSigMessage.Marshal(b, m, deterministic)
} else {
b = b[:cap(b)]
n, err := m.MarshalToSizedBuffer(b)
if err != nil {
return nil, err
}
return b[:n], nil
}
}
func (m *AuthSigMessage) XXX_Merge(src proto.Message) {
xxx_messageInfo_AuthSigMessage.Merge(m, src)
}
func (m *AuthSigMessage) XXX_Size() int {
return m.Size()
}
func (m *AuthSigMessage) XXX_DiscardUnknown() {
xxx_messageInfo_AuthSigMessage.DiscardUnknown(m)
}
var xxx_messageInfo_AuthSigMessage proto.InternalMessageInfo
func (m *AuthSigMessage) GetPubKey() crypto.PublicKey {
if m != nil {
return m.PubKey
}
return crypto.PublicKey{}
}
func (m *AuthSigMessage) GetSig() []byte {
if m != nil {
return m.Sig
}
return nil
}
func init() {
proto.RegisterEnum("tendermint.privval.Errors", Errors_name, Errors_value)
proto.RegisterType((*RemoteSignerError)(nil), "tendermint.privval.RemoteSignerError")
@ -673,59 +728,62 @@ func init() {
proto.RegisterType((*PingRequest)(nil), "tendermint.privval.PingRequest")
proto.RegisterType((*PingResponse)(nil), "tendermint.privval.PingResponse")
proto.RegisterType((*Message)(nil), "tendermint.privval.Message")
proto.RegisterType((*AuthSigMessage)(nil), "tendermint.privval.AuthSigMessage")
}
func init() { proto.RegisterFile("tendermint/privval/types.proto", fileDescriptor_cb4e437a5328cf9c) }
var fileDescriptor_cb4e437a5328cf9c = []byte{
// 750 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x55, 0x4d, 0x4f, 0x13, 0x41,
0x18, 0xde, 0x85, 0x7e, 0xc0, 0x5b, 0x5a, 0xca, 0x80, 0x58, 0x1a, 0x5c, 0x6a, 0x8d, 0x4a, 0x7a,
0x68, 0x0d, 0x26, 0x26, 0x06, 0x2f, 0x02, 0x1b, 0xdb, 0x34, 0x6c, 0xeb, 0xb4, 0x08, 0x21, 0x31,
0x9b, 0x7e, 0x8c, 0xcb, 0x06, 0xba, 0x3b, 0xee, 0x6c, 0x49, 0x7a, 0xf6, 0xe6, 0xc9, 0xc4, 0x3f,
0xe1, 0xd9, 0x5f, 0xc1, 0x91, 0xa3, 0x27, 0x63, 0xe0, 0x8f, 0x98, 0xce, 0x4e, 0xb7, 0xdb, 0x2f,
0xa2, 0xe1, 0xb6, 0xf3, 0xbe, 0xef, 0x3c, 0x1f, 0x33, 0xcf, 0x66, 0x40, 0x71, 0x89, 0xd5, 0x26,
0x4e, 0xc7, 0xb4, 0xdc, 0x02, 0x75, 0xcc, 0xcb, 0xcb, 0xc6, 0x45, 0xc1, 0xed, 0x51, 0xc2, 0xf2,
0xd4, 0xb1, 0x5d, 0x1b, 0xa1, 0x61, 0x3f, 0x2f, 0xfa, 0xe9, 0xcd, 0xc0, 0x9e, 0x96, 0xd3, 0xa3,
0xae, 0x5d, 0x38, 0x27, 0x3d, 0xb1, 0x63, 0xa4, 0xcb, 0x91, 0x82, 0x78, 0xe9, 0x35, 0xc3, 0x36,
0x6c, 0xfe, 0x59, 0xe8, 0x7f, 0x79, 0xd5, 0x6c, 0x09, 0x56, 0x30, 0xe9, 0xd8, 0x2e, 0xa9, 0x99,
0x86, 0x45, 0x1c, 0xd5, 0x71, 0x6c, 0x07, 0x21, 0x08, 0xb5, 0xec, 0x36, 0x49, 0xc9, 0x19, 0x79,
0x3b, 0x8c, 0xf9, 0x37, 0xca, 0x40, 0xac, 0x4d, 0x58, 0xcb, 0x31, 0xa9, 0x6b, 0xda, 0x56, 0x6a,
0x2e, 0x23, 0x6f, 0x2f, 0xe2, 0x60, 0x29, 0x9b, 0x83, 0x78, 0xb5, 0xdb, 0x2c, 0x93, 0x1e, 0x26,
0x9f, 0xbb, 0x84, 0xb9, 0x68, 0x03, 0x16, 0x5a, 0x67, 0x0d, 0xd3, 0xd2, 0xcd, 0x36, 0x87, 0x5a,
0xc4, 0x51, 0xbe, 0x2e, 0xb5, 0xb3, 0x5f, 0x65, 0x48, 0x0c, 0x86, 0x19, 0xb5, 0x2d, 0x46, 0xd0,
0x2e, 0x44, 0x69, 0xb7, 0xa9, 0x9f, 0x93, 0x1e, 0x1f, 0x8e, 0xed, 0x6c, 0xe6, 0x03, 0x27, 0xe0,
0xb9, 0xcd, 0x57, 0xbb, 0xcd, 0x0b, 0xb3, 0x55, 0x26, 0xbd, 0xbd, 0xd0, 0xd5, 0xef, 0x2d, 0x09,
0x47, 0x28, 0x07, 0x41, 0xbb, 0x10, 0x26, 0x7d, 0xe9, 0x5c, 0x57, 0x6c, 0xe7, 0x69, 0x7e, 0xf2,
0xf0, 0xf2, 0x13, 0x3e, 0xb1, 0xb7, 0x27, 0x7b, 0x02, 0xcb, 0xfd, 0xea, 0x07, 0xdb, 0x25, 0x03,
0xe9, 0x39, 0x08, 0x5d, 0xda, 0x2e, 0x11, 0x4a, 0xd6, 0x83, 0x70, 0xde, 0x99, 0xf2, 0x61, 0x3e,
0x33, 0x62, 0x73, 0x6e, 0xd4, 0xe6, 0x17, 0x19, 0x10, 0x27, 0x6c, 0x7b, 0xe0, 0xc2, 0xea, 0x8b,
0x7f, 0x41, 0x17, 0x0e, 0x3d, 0x8e, 0x7b, 0xf9, 0x3b, 0x83, 0xd5, 0x7e, 0xb5, 0xea, 0xd8, 0xd4,
0x66, 0x8d, 0x8b, 0x81, 0xc7, 0x57, 0xb0, 0x40, 0x45, 0x49, 0x28, 0x49, 0x4f, 0x2a, 0xf1, 0x37,
0xf9, 0xb3, 0x77, 0xf9, 0xfd, 0x2e, 0xc3, 0xba, 0xe7, 0x77, 0x48, 0x26, 0x3c, 0xbf, 0xf9, 0x1f,
0x36, 0xe1, 0x7d, 0xc8, 0x79, 0x2f, 0xff, 0x71, 0x88, 0x55, 0x4d, 0xcb, 0x10, 0xbe, 0xb3, 0x09,
0x58, 0xf2, 0x96, 0x9e, 0xb2, 0xec, 0xcf, 0x30, 0x44, 0x0f, 0x09, 0x63, 0x0d, 0x83, 0xa0, 0x32,
0x2c, 0x8b, 0x10, 0xea, 0x8e, 0x37, 0x2e, 0xc4, 0x3e, 0x9e, 0xc6, 0x38, 0x12, 0xf7, 0xa2, 0x84,
0xe3, 0x74, 0x24, 0xff, 0x1a, 0x24, 0x87, 0x60, 0x1e, 0x99, 0xd0, 0x9f, 0xbd, 0x0b, 0xcd, 0x9b,
0x2c, 0x4a, 0x38, 0x41, 0x47, 0xff, 0x90, 0xf7, 0xb0, 0xc2, 0x4c, 0xc3, 0xd2, 0xfb, 0x89, 0xf0,
0xe5, 0xcd, 0x73, 0xc0, 0x27, 0xd3, 0x00, 0xc7, 0x42, 0x5d, 0x94, 0xf0, 0x32, 0x1b, 0xcb, 0xf9,
0x29, 0xac, 0x31, 0x7e, 0x5f, 0x03, 0x50, 0x21, 0x33, 0xc4, 0x51, 0x9f, 0xcd, 0x42, 0x1d, 0xcd,
0x73, 0x51, 0xc2, 0x88, 0x4d, 0xa6, 0xfc, 0x23, 0x3c, 0xe0, 0x72, 0x07, 0x97, 0xe8, 0x4b, 0x0e,
0x73, 0xf0, 0xe7, 0xb3, 0xc0, 0xc7, 0x72, 0x5a, 0x94, 0xf0, 0x2a, 0x9b, 0x12, 0xdf, 0x4f, 0x90,
0x12, 0xd2, 0x03, 0x04, 0x42, 0x7e, 0x84, 0x33, 0xe4, 0x66, 0xcb, 0x1f, 0x8f, 0x67, 0x51, 0xc2,
0xeb, 0x6c, 0x7a, 0x70, 0x0f, 0x60, 0x89, 0x9a, 0x96, 0xe1, 0xab, 0x8f, 0x72, 0xec, 0xad, 0xa9,
0x37, 0x38, 0x4c, 0x59, 0x51, 0xc2, 0x31, 0x3a, 0x5c, 0xa2, 0x77, 0x10, 0x17, 0x28, 0x42, 0xe2,
0x02, 0x87, 0xc9, 0xcc, 0x86, 0xf1, 0x85, 0x2d, 0xd1, 0xc0, 0x7a, 0x2f, 0x0c, 0xf3, 0xac, 0xdb,
0xc9, 0xfd, 0x90, 0x21, 0xc2, 0x43, 0xce, 0x10, 0x82, 0x84, 0x8a, 0x71, 0x05, 0xd7, 0xf4, 0x23,
0xad, 0xac, 0x55, 0x8e, 0xb5, 0xa4, 0x84, 0x14, 0x48, 0xfb, 0x35, 0xf5, 0xa4, 0xaa, 0xee, 0xd7,
0xd5, 0x03, 0x1d, 0xab, 0xb5, 0x6a, 0x45, 0xab, 0xa9, 0x49, 0x19, 0xa5, 0x60, 0x4d, 0xf4, 0xb5,
0x8a, 0xbe, 0x5f, 0xd1, 0x34, 0x75, 0xbf, 0x5e, 0xaa, 0x68, 0xc9, 0x39, 0xf4, 0x08, 0x36, 0x44,
0x67, 0x58, 0xd6, 0xeb, 0xa5, 0x43, 0xb5, 0x72, 0x54, 0x4f, 0xce, 0xa3, 0x87, 0xb0, 0x2a, 0xda,
0x58, 0x7d, 0x7b, 0xe0, 0x37, 0x42, 0x01, 0xc4, 0x63, 0x5c, 0xaa, 0xab, 0x7e, 0x27, 0xbc, 0x57,
0xbb, 0xba, 0x51, 0xe4, 0xeb, 0x1b, 0x45, 0xfe, 0x73, 0xa3, 0xc8, 0xdf, 0x6e, 0x15, 0xe9, 0xfa,
0x56, 0x91, 0x7e, 0xdd, 0x2a, 0xd2, 0xe9, 0x6b, 0xc3, 0x74, 0xcf, 0xba, 0xcd, 0x7c, 0xcb, 0xee,
0x14, 0x82, 0x6f, 0x57, 0xf0, 0x61, 0xec, 0xbf, 0x57, 0x93, 0x2f, 0x65, 0x33, 0xc2, 0x3b, 0x2f,
0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0xa5, 0x2a, 0xe5, 0x4a, 0x46, 0x07, 0x00, 0x00,
// 779 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x55, 0x4b, 0x4f, 0xdb, 0x4a,
0x14, 0xb6, 0xc9, 0x0b, 0x4e, 0x1e, 0x84, 0x81, 0xcb, 0x0d, 0x11, 0xd7, 0xe4, 0xfa, 0xea, 0xb6,
0x28, 0x8b, 0xa4, 0xa2, 0x52, 0xa5, 0x8a, 0x6e, 0x78, 0x58, 0x4d, 0x14, 0xe1, 0xa4, 0x93, 0x50,
0x10, 0x52, 0x65, 0xe5, 0x31, 0x75, 0x2c, 0x88, 0xed, 0x7a, 0x1c, 0xa4, 0xac, 0xbb, 0xeb, 0xaa,
0x52, 0xff, 0x44, 0xd7, 0xfd, 0x15, 0x2c, 0x59, 0x76, 0x55, 0x55, 0xf0, 0x47, 0xaa, 0x8c, 0x27,
0x8e, 0xf3, 0x42, 0xad, 0xd8, 0xcd, 0x9c, 0x73, 0xe6, 0x3b, 0xdf, 0x37, 0xf3, 0xd9, 0x07, 0x24,
0x97, 0x98, 0x1d, 0xe2, 0xf4, 0x0c, 0xd3, 0x2d, 0xda, 0x8e, 0x71, 0x7d, 0xdd, 0xbc, 0x2a, 0xba,
0x03, 0x9b, 0xd0, 0x82, 0xed, 0x58, 0xae, 0x85, 0xd0, 0x38, 0x5f, 0xe0, 0xf9, 0xec, 0x76, 0xe0,
0x4c, 0xdb, 0x19, 0xd8, 0xae, 0x55, 0xbc, 0x24, 0x03, 0x7e, 0x62, 0x22, 0xcb, 0x90, 0x82, 0x78,
0xd9, 0x0d, 0xdd, 0xd2, 0x2d, 0xb6, 0x2c, 0x0e, 0x57, 0x5e, 0x54, 0x2e, 0xc3, 0x1a, 0x26, 0x3d,
0xcb, 0x25, 0x75, 0x43, 0x37, 0x89, 0xa3, 0x38, 0x8e, 0xe5, 0x20, 0x04, 0xe1, 0xb6, 0xd5, 0x21,
0x19, 0x31, 0x27, 0xee, 0x46, 0x30, 0x5b, 0xa3, 0x1c, 0xc4, 0x3b, 0x84, 0xb6, 0x1d, 0xc3, 0x76,
0x0d, 0xcb, 0xcc, 0x2c, 0xe5, 0xc4, 0xdd, 0x15, 0x1c, 0x0c, 0xc9, 0x79, 0x48, 0xd6, 0xfa, 0xad,
0x0a, 0x19, 0x60, 0xf2, 0xa1, 0x4f, 0xa8, 0x8b, 0xb6, 0x60, 0xb9, 0xdd, 0x6d, 0x1a, 0xa6, 0x66,
0x74, 0x18, 0xd4, 0x0a, 0x8e, 0xb1, 0x7d, 0xb9, 0x23, 0x7f, 0x12, 0x21, 0x35, 0x2a, 0xa6, 0xb6,
0x65, 0x52, 0x82, 0xf6, 0x21, 0x66, 0xf7, 0x5b, 0xda, 0x25, 0x19, 0xb0, 0xe2, 0xf8, 0xde, 0x76,
0x21, 0x70, 0x03, 0x9e, 0xda, 0x42, 0xad, 0xdf, 0xba, 0x32, 0xda, 0x15, 0x32, 0x38, 0x0c, 0xdf,
0xfc, 0xd8, 0x11, 0x70, 0xd4, 0x66, 0x20, 0x68, 0x1f, 0x22, 0x64, 0x48, 0x9d, 0xf1, 0x8a, 0xef,
0xfd, 0x5f, 0x98, 0xbd, 0xbc, 0xc2, 0x8c, 0x4e, 0xec, 0x9d, 0x91, 0xcf, 0x61, 0x75, 0x18, 0x7d,
0x6b, 0xb9, 0x64, 0x44, 0x3d, 0x0f, 0xe1, 0x6b, 0xcb, 0x25, 0x9c, 0xc9, 0x66, 0x10, 0xce, 0xbb,
0x53, 0x56, 0xcc, 0x6a, 0x26, 0x64, 0x2e, 0x4d, 0xca, 0xfc, 0x28, 0x02, 0x62, 0x0d, 0x3b, 0x1e,
0x38, 0x97, 0xfa, 0xec, 0x77, 0xd0, 0xb9, 0x42, 0xaf, 0xc7, 0xa3, 0xf4, 0x75, 0x61, 0x7d, 0x18,
0xad, 0x39, 0x96, 0x6d, 0xd1, 0xe6, 0xd5, 0x48, 0xe3, 0x0b, 0x58, 0xb6, 0x79, 0x88, 0x33, 0xc9,
0xce, 0x32, 0xf1, 0x0f, 0xf9, 0xb5, 0x0f, 0xe9, 0xfd, 0x22, 0xc2, 0xa6, 0xa7, 0x77, 0xdc, 0x8c,
0x6b, 0x7e, 0xf5, 0x27, 0xdd, 0xb8, 0xf6, 0x71, 0xcf, 0x47, 0xe9, 0x4f, 0x42, 0xbc, 0x66, 0x98,
0x3a, 0xd7, 0x2d, 0xa7, 0x20, 0xe1, 0x6d, 0x3d, 0x66, 0xf2, 0xb7, 0x08, 0xc4, 0x4e, 0x08, 0xa5,
0x4d, 0x9d, 0xa0, 0x0a, 0xac, 0x72, 0x13, 0x6a, 0x8e, 0x57, 0xce, 0xc9, 0xfe, 0x3b, 0xaf, 0xe3,
0x84, 0xdd, 0x4b, 0x02, 0x4e, 0xda, 0x13, 0xfe, 0x57, 0x21, 0x3d, 0x06, 0xf3, 0x9a, 0x71, 0xfe,
0xf2, 0x43, 0x68, 0x5e, 0x65, 0x49, 0xc0, 0x29, 0x7b, 0xf2, 0x0b, 0x79, 0x03, 0x6b, 0xd4, 0xd0,
0x4d, 0x6d, 0xe8, 0x08, 0x9f, 0x5e, 0x88, 0x01, 0xfe, 0x37, 0x0f, 0x70, 0xca, 0xd4, 0x25, 0x01,
0xaf, 0xd2, 0x29, 0x9f, 0x5f, 0xc0, 0x06, 0x65, 0xef, 0x35, 0x02, 0xe5, 0x34, 0xc3, 0x0c, 0xf5,
0xc9, 0x22, 0xd4, 0x49, 0x3f, 0x97, 0x04, 0x8c, 0xe8, 0xac, 0xcb, 0xdf, 0xc1, 0x5f, 0x8c, 0xee,
0xe8, 0x11, 0x7d, 0xca, 0x11, 0x06, 0xfe, 0x74, 0x11, 0xf8, 0x94, 0x4f, 0x4b, 0x02, 0x5e, 0xa7,
0x73, 0xec, 0xfb, 0x1e, 0x32, 0x9c, 0x7a, 0xa0, 0x01, 0xa7, 0x1f, 0x65, 0x1d, 0xf2, 0x8b, 0xe9,
0x4f, 0xdb, 0xb3, 0x24, 0xe0, 0x4d, 0x3a, 0xdf, 0xb8, 0xc7, 0x90, 0xb0, 0x0d, 0x53, 0xf7, 0xd9,
0xc7, 0x18, 0xf6, 0xce, 0xdc, 0x17, 0x1c, 0xbb, 0xac, 0x24, 0xe0, 0xb8, 0x3d, 0xde, 0xa2, 0xd7,
0x90, 0xe4, 0x28, 0x9c, 0xe2, 0x32, 0x83, 0xc9, 0x2d, 0x86, 0xf1, 0x89, 0x25, 0xec, 0xc0, 0xfe,
0x30, 0x02, 0x21, 0xda, 0xef, 0xc9, 0x1a, 0xa4, 0x0e, 0xfa, 0x6e, 0xb7, 0x6e, 0xe8, 0x23, 0xeb,
0x3e, 0xea, 0xff, 0x99, 0x86, 0x10, 0x35, 0x74, 0xe6, 0xce, 0x04, 0x1e, 0x2e, 0xf3, 0x5f, 0x45,
0x88, 0xb2, 0xaf, 0x88, 0x22, 0x04, 0x29, 0x05, 0xe3, 0x2a, 0xae, 0x6b, 0xa7, 0x6a, 0x45, 0xad,
0x9e, 0xa9, 0x69, 0x01, 0x49, 0x90, 0xf5, 0x63, 0xca, 0x79, 0x4d, 0x39, 0x6a, 0x28, 0xc7, 0x1a,
0x56, 0xea, 0xb5, 0xaa, 0x5a, 0x57, 0xd2, 0x22, 0xca, 0xc0, 0x06, 0xcf, 0xab, 0x55, 0xed, 0xa8,
0xaa, 0xaa, 0xca, 0x51, 0xa3, 0x5c, 0x55, 0xd3, 0x4b, 0xe8, 0x1f, 0xd8, 0xe2, 0x99, 0x71, 0x58,
0x6b, 0x94, 0x4f, 0x94, 0xea, 0x69, 0x23, 0x1d, 0x42, 0x7f, 0xc3, 0x3a, 0x4f, 0x63, 0xe5, 0xe0,
0xd8, 0x4f, 0x84, 0x03, 0x88, 0x67, 0xb8, 0xdc, 0x50, 0xfc, 0x4c, 0xe4, 0xb0, 0x7e, 0x73, 0x27,
0x89, 0xb7, 0x77, 0x92, 0xf8, 0xf3, 0x4e, 0x12, 0x3f, 0xdf, 0x4b, 0xc2, 0xed, 0xbd, 0x24, 0x7c,
0xbf, 0x97, 0x84, 0x8b, 0x97, 0xba, 0xe1, 0x76, 0xfb, 0xad, 0x42, 0xdb, 0xea, 0x15, 0x83, 0xc3,
0x31, 0x38, 0x79, 0x87, 0x03, 0x71, 0x76, 0x14, 0xb7, 0xa2, 0x2c, 0xf3, 0xfc, 0x57, 0x00, 0x00,
0x00, 0xff, 0xff, 0x46, 0x64, 0xeb, 0xa4, 0xa7, 0x07, 0x00, 0x00,
}
func (m *RemoteSignerError) Marshal() (dAtA []byte, err error) {
@ -1258,6 +1316,46 @@ func (m *Message_PingResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
}
return len(dAtA) - i, nil
}
func (m *AuthSigMessage) Marshal() (dAtA []byte, err error) {
size := m.Size()
dAtA = make([]byte, size)
n, err := m.MarshalToSizedBuffer(dAtA[:size])
if err != nil {
return nil, err
}
return dAtA[:n], nil
}
func (m *AuthSigMessage) MarshalTo(dAtA []byte) (int, error) {
size := m.Size()
return m.MarshalToSizedBuffer(dAtA[:size])
}
func (m *AuthSigMessage) MarshalToSizedBuffer(dAtA []byte) (int, error) {
i := len(dAtA)
_ = i
var l int
_ = l
if len(m.Sig) > 0 {
i -= len(m.Sig)
copy(dAtA[i:], m.Sig)
i = encodeVarintTypes(dAtA, i, uint64(len(m.Sig)))
i--
dAtA[i] = 0x12
}
{
size, err := m.PubKey.MarshalToSizedBuffer(dAtA[:i])
if err != nil {
return 0, err
}
i -= size
i = encodeVarintTypes(dAtA, i, uint64(size))
}
i--
dAtA[i] = 0xa
return len(dAtA) - i, nil
}
func encodeVarintTypes(dAtA []byte, offset int, v uint64) int {
offset -= sovTypes(v)
base := offset
@ -1503,6 +1601,20 @@ func (m *Message_PingResponse) Size() (n int) {
}
return n
}
func (m *AuthSigMessage) Size() (n int) {
if m == nil {
return 0
}
var l int
_ = l
l = m.PubKey.Size()
n += 1 + l + sovTypes(uint64(l))
l = len(m.Sig)
if l > 0 {
n += 1 + l + sovTypes(uint64(l))
}
return n
}
func sovTypes(x uint64) (n int) {
return (math_bits.Len64(x|1) + 6) / 7
@ -2746,6 +2858,126 @@ func (m *Message) Unmarshal(dAtA []byte) error {
}
return nil
}
func (m *AuthSigMessage) Unmarshal(dAtA []byte) error {
l := len(dAtA)
iNdEx := 0
for iNdEx < l {
preIndex := iNdEx
var wire uint64
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTypes
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
wire |= uint64(b&0x7F) << shift
if b < 0x80 {
break
}
}
fieldNum := int32(wire >> 3)
wireType := int(wire & 0x7)
if wireType == 4 {
return fmt.Errorf("proto: AuthSigMessage: wiretype end group for non-group")
}
if fieldNum <= 0 {
return fmt.Errorf("proto: AuthSigMessage: illegal tag %d (wire type %d)", fieldNum, wire)
}
switch fieldNum {
case 1:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field PubKey", wireType)
}
var msglen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTypes
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
msglen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if msglen < 0 {
return ErrInvalidLengthTypes
}
postIndex := iNdEx + msglen
if postIndex < 0 {
return ErrInvalidLengthTypes
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
if err := m.PubKey.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
return err
}
iNdEx = postIndex
case 2:
if wireType != 2 {
return fmt.Errorf("proto: wrong wireType = %d for field Sig", wireType)
}
var byteLen int
for shift := uint(0); ; shift += 7 {
if shift >= 64 {
return ErrIntOverflowTypes
}
if iNdEx >= l {
return io.ErrUnexpectedEOF
}
b := dAtA[iNdEx]
iNdEx++
byteLen |= int(b&0x7F) << shift
if b < 0x80 {
break
}
}
if byteLen < 0 {
return ErrInvalidLengthTypes
}
postIndex := iNdEx + byteLen
if postIndex < 0 {
return ErrInvalidLengthTypes
}
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Sig = append(m.Sig[:0], dAtA[iNdEx:postIndex]...)
if m.Sig == nil {
m.Sig = []byte{}
}
iNdEx = postIndex
default:
iNdEx = preIndex
skippy, err := skipTypes(dAtA[iNdEx:])
if err != nil {
return err
}
if skippy < 0 {
return ErrInvalidLengthTypes
}
if (iNdEx + skippy) < 0 {
return ErrInvalidLengthTypes
}
if (iNdEx + skippy) > l {
return io.ErrUnexpectedEOF
}
iNdEx += skippy
}
}
if iNdEx > l {
return io.ErrUnexpectedEOF
}
return nil
}
func skipTypes(dAtA []byte) (n int, err error) {
l := len(dAtA)
iNdEx := 0


+ 8
- 0
proto/tendermint/privval/types.proto View File

@ -74,3 +74,11 @@ message Message {
PingResponse ping_response = 8;
}
}
// AuthSigMessage is duplicated from p2p prior to the P2P refactor.
// It is used for the SecretConnection until we migrate privval to gRPC.
// https://github.com/tendermint/tendermint/issues/4698
message AuthSigMessage {
tendermint.crypto.PublicKey pub_key = 1 [(gogoproto.nullable) = false];
bytes sig = 2;
}

Loading…
Cancel
Save