You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

477 lines
14 KiB

crypto: Use a different library for ed25519/sr25519 (#6526) At Oasis we have spend some time writing a new Ed25519/X25519/sr25519 implementation called curve25519-voi. This PR switches the import from ed25519consensus/go-schnorrkel, which should lead to performance gains on most systems. Summary of changes: * curve25519-voi is now used for Ed25519 operations, following the existing ZIP-215 semantics. * curve25519-voi's public key cache is enabled (hardcoded size of 4096 entries, should be tuned, see the code comment) to accelerate repeated Ed25519 verification with the same public key(s). * (BREAKING) curve25519-voi is now used for sr25519 operations. This is a breaking change as the current sr25519 support does something decidedly non-standard when going from a MiniSecretKey to a SecretKey and or PublicKey (The expansion routine is called twice). While I believe the new behavior (that expands once and only once) to be more "correct", this changes the semantics as implemented. * curve25519-voi is now used for merlin since the included STROBE implementation produces much less garbage on the heap. Side issues fixed: * The version of go-schnorrkel that is currently imported by tendermint has a badly broken batch verification implementation. Upstream has fixed the issue after I reported it, so the version should be bumped in the interim. Open design questions/issues: * As noted, the public key cache size should be tuned. It is currently backed by a trivial thread-safe LRU cache, which is not scan-resistant, but replacing it with something better is a matter of implementing an interface. * As far as I can tell, the only reason why serial verification on batch failure is necessary is to provide more detailed error messages (that are only used in some unit tests). If you trust the batch verification to be consistent with serial verification then the fallback can be eliminated entirely (the BatchVerifier provided by the new library supports an option that omits the fallback if this is chosen as the way forward). * curve25519-voi's sr25519 support could use more optimization and more eyes on the code. The algorithm unfortunately is woefully under-specified, and the implementation was done primarily because I got really sad when I actually looked at go-schnorrkel, and we do not use the algorithm at this time.
3 years ago
cleanup: Reduce and normalize import path aliasing. (#6975) The code in the Tendermint repository makes heavy use of import aliasing. This is made necessary by our extensive reuse of common base package names, and by repetition of similar names across different subdirectories. Unfortunately we have not been very consistent about which packages we alias in various circumstances, and the aliases we use vary. In the spirit of the advice in the style guide and https://github.com/golang/go/wiki/CodeReviewComments#imports, his change makes an effort to clean up and normalize import aliasing. This change makes no API or behavioral changes. It is a pure cleanup intended o help make the code more readable to developers (including myself) trying to understand what is being imported where. Only unexported names have been modified, and the changes were generated and applied mechanically with gofmt -r and comby, respecting the lexical and syntactic rules of Go. Even so, I did not fix every inconsistency. Where the changes would be too disruptive, I left it alone. The principles I followed in this cleanup are: - Remove aliases that restate the package name. - Remove aliases where the base package name is unambiguous. - Move overly-terse abbreviations from the import to the usage site. - Fix lexical issues (remove underscores, remove capitalization). - Fix import groupings to more closely match the style guide. - Group blank (side-effecting) imports and ensure they are commented. - Add aliases to multiple imports with the same base package name.
3 years ago
crypto: Use a different library for ed25519/sr25519 (#6526) At Oasis we have spend some time writing a new Ed25519/X25519/sr25519 implementation called curve25519-voi. This PR switches the import from ed25519consensus/go-schnorrkel, which should lead to performance gains on most systems. Summary of changes: * curve25519-voi is now used for Ed25519 operations, following the existing ZIP-215 semantics. * curve25519-voi's public key cache is enabled (hardcoded size of 4096 entries, should be tuned, see the code comment) to accelerate repeated Ed25519 verification with the same public key(s). * (BREAKING) curve25519-voi is now used for sr25519 operations. This is a breaking change as the current sr25519 support does something decidedly non-standard when going from a MiniSecretKey to a SecretKey and or PublicKey (The expansion routine is called twice). While I believe the new behavior (that expands once and only once) to be more "correct", this changes the semantics as implemented. * curve25519-voi is now used for merlin since the included STROBE implementation produces much less garbage on the heap. Side issues fixed: * The version of go-schnorrkel that is currently imported by tendermint has a badly broken batch verification implementation. Upstream has fixed the issue after I reported it, so the version should be bumped in the interim. Open design questions/issues: * As noted, the public key cache size should be tuned. It is currently backed by a trivial thread-safe LRU cache, which is not scan-resistant, but replacing it with something better is a matter of implementing an interface. * As far as I can tell, the only reason why serial verification on batch failure is necessary is to provide more detailed error messages (that are only used in some unit tests). If you trust the batch verification to be consistent with serial verification then the fallback can be eliminated entirely (the BatchVerifier provided by the new library supports an option that omits the fallback if this is chosen as the way forward). * curve25519-voi's sr25519 support could use more optimization and more eyes on the code. The algorithm unfortunately is woefully under-specified, and the implementation was done primarily because I got really sad when I actually looked at go-schnorrkel, and we do not use the algorithm at this time.
3 years ago
crypto: Use a different library for ed25519/sr25519 (#6526) At Oasis we have spend some time writing a new Ed25519/X25519/sr25519 implementation called curve25519-voi. This PR switches the import from ed25519consensus/go-schnorrkel, which should lead to performance gains on most systems. Summary of changes: * curve25519-voi is now used for Ed25519 operations, following the existing ZIP-215 semantics. * curve25519-voi's public key cache is enabled (hardcoded size of 4096 entries, should be tuned, see the code comment) to accelerate repeated Ed25519 verification with the same public key(s). * (BREAKING) curve25519-voi is now used for sr25519 operations. This is a breaking change as the current sr25519 support does something decidedly non-standard when going from a MiniSecretKey to a SecretKey and or PublicKey (The expansion routine is called twice). While I believe the new behavior (that expands once and only once) to be more "correct", this changes the semantics as implemented. * curve25519-voi is now used for merlin since the included STROBE implementation produces much less garbage on the heap. Side issues fixed: * The version of go-schnorrkel that is currently imported by tendermint has a badly broken batch verification implementation. Upstream has fixed the issue after I reported it, so the version should be bumped in the interim. Open design questions/issues: * As noted, the public key cache size should be tuned. It is currently backed by a trivial thread-safe LRU cache, which is not scan-resistant, but replacing it with something better is a matter of implementing an interface. * As far as I can tell, the only reason why serial verification on batch failure is necessary is to provide more detailed error messages (that are only used in some unit tests). If you trust the batch verification to be consistent with serial verification then the fallback can be eliminated entirely (the BatchVerifier provided by the new library supports an option that omits the fallback if this is chosen as the way forward). * curve25519-voi's sr25519 support could use more optimization and more eyes on the code. The algorithm unfortunately is woefully under-specified, and the implementation was done primarily because I got really sad when I actually looked at go-schnorrkel, and we do not use the algorithm at this time.
3 years ago
cleanup: Reduce and normalize import path aliasing. (#6975) The code in the Tendermint repository makes heavy use of import aliasing. This is made necessary by our extensive reuse of common base package names, and by repetition of similar names across different subdirectories. Unfortunately we have not been very consistent about which packages we alias in various circumstances, and the aliases we use vary. In the spirit of the advice in the style guide and https://github.com/golang/go/wiki/CodeReviewComments#imports, his change makes an effort to clean up and normalize import aliasing. This change makes no API or behavioral changes. It is a pure cleanup intended o help make the code more readable to developers (including myself) trying to understand what is being imported where. Only unexported names have been modified, and the changes were generated and applied mechanically with gofmt -r and comby, respecting the lexical and syntactic rules of Go. Even so, I did not fix every inconsistency. Where the changes would be too disruptive, I left it alone. The principles I followed in this cleanup are: - Remove aliases that restate the package name. - Remove aliases where the base package name is unambiguous. - Move overly-terse abbreviations from the import to the usage site. - Fix lexical issues (remove underscores, remove capitalization). - Fix import groupings to more closely match the style guide. - Group blank (side-effecting) imports and ensure they are commented. - Add aliases to multiple imports with the same base package name.
3 years ago
cleanup: Reduce and normalize import path aliasing. (#6975) The code in the Tendermint repository makes heavy use of import aliasing. This is made necessary by our extensive reuse of common base package names, and by repetition of similar names across different subdirectories. Unfortunately we have not been very consistent about which packages we alias in various circumstances, and the aliases we use vary. In the spirit of the advice in the style guide and https://github.com/golang/go/wiki/CodeReviewComments#imports, his change makes an effort to clean up and normalize import aliasing. This change makes no API or behavioral changes. It is a pure cleanup intended o help make the code more readable to developers (including myself) trying to understand what is being imported where. Only unexported names have been modified, and the changes were generated and applied mechanically with gofmt -r and comby, respecting the lexical and syntactic rules of Go. Even so, I did not fix every inconsistency. Where the changes would be too disruptive, I left it alone. The principles I followed in this cleanup are: - Remove aliases that restate the package name. - Remove aliases where the base package name is unambiguous. - Move overly-terse abbreviations from the import to the usage site. - Fix lexical issues (remove underscores, remove capitalization). - Fix import groupings to more closely match the style guide. - Group blank (side-effecting) imports and ensure they are commented. - Add aliases to multiple imports with the same base package name.
3 years ago
  1. package privval
  2. import (
  3. "bytes"
  4. "crypto/cipher"
  5. crand "crypto/rand"
  6. "crypto/sha256"
  7. "encoding/binary"
  8. "errors"
  9. "fmt"
  10. "io"
  11. "math"
  12. "net"
  13. "sync"
  14. "time"
  15. gogotypes "github.com/gogo/protobuf/types"
  16. pool "github.com/libp2p/go-buffer-pool"
  17. "github.com/oasisprotocol/curve25519-voi/primitives/merlin"
  18. "golang.org/x/crypto/chacha20poly1305"
  19. "golang.org/x/crypto/curve25519"
  20. "golang.org/x/crypto/hkdf"
  21. "golang.org/x/crypto/nacl/box"
  22. "github.com/tendermint/tendermint/crypto"
  23. "github.com/tendermint/tendermint/crypto/ed25519"
  24. "github.com/tendermint/tendermint/crypto/encoding"
  25. "github.com/tendermint/tendermint/internal/libs/async"
  26. "github.com/tendermint/tendermint/internal/libs/protoio"
  27. tmprivval "github.com/tendermint/tendermint/proto/tendermint/privval"
  28. )
  29. // This code has been duplicated from p2p/conn prior to the P2P refactor.
  30. // It is left here temporarily until we migrate privval to gRPC.
  31. // https://github.com/tendermint/tendermint/issues/4698
  32. // 4 + 1024 == 1028 total frame size
  33. const (
  34. dataLenSize = 4
  35. dataMaxSize = 1024
  36. totalFrameSize = dataMaxSize + dataLenSize
  37. aeadSizeOverhead = 16 // overhead of poly 1305 authentication tag
  38. aeadKeySize = chacha20poly1305.KeySize
  39. aeadNonceSize = chacha20poly1305.NonceSize
  40. labelEphemeralLowerPublicKey = "EPHEMERAL_LOWER_PUBLIC_KEY"
  41. labelEphemeralUpperPublicKey = "EPHEMERAL_UPPER_PUBLIC_KEY"
  42. labelDHSecret = "DH_SECRET"
  43. labelSecretConnectionMac = "SECRET_CONNECTION_MAC"
  44. )
  45. var (
  46. ErrSmallOrderRemotePubKey = errors.New("detected low order point from remote peer")
  47. secretConnKeyAndChallengeGen = []byte("TENDERMINT_SECRET_CONNECTION_KEY_AND_CHALLENGE_GEN")
  48. )
  49. // SecretConnection implements net.Conn.
  50. // It is an implementation of the STS protocol.
  51. // See https://github.com/tendermint/tendermint/blob/0.1/docs/sts-final.pdf for
  52. // details on the protocol.
  53. //
  54. // Consumers of the SecretConnection are responsible for authenticating
  55. // the remote peer's pubkey against known information, like a nodeID.
  56. // Otherwise they are vulnerable to MITM.
  57. // (TODO(ismail): see also https://github.com/tendermint/tendermint/issues/3010)
  58. type SecretConnection struct {
  59. // immutable
  60. recvAead cipher.AEAD
  61. sendAead cipher.AEAD
  62. remPubKey crypto.PubKey
  63. conn io.ReadWriteCloser
  64. // net.Conn must be thread safe:
  65. // https://golang.org/pkg/net/#Conn.
  66. // Since we have internal mutable state,
  67. // we need mtxs. But recv and send states
  68. // are independent, so we can use two mtxs.
  69. // All .Read are covered by recvMtx,
  70. // all .Write are covered by sendMtx.
  71. recvMtx sync.Mutex
  72. recvBuffer []byte
  73. recvNonce *[aeadNonceSize]byte
  74. sendMtx sync.Mutex
  75. sendNonce *[aeadNonceSize]byte
  76. }
  77. // MakeSecretConnection performs handshake and returns a new authenticated
  78. // SecretConnection.
  79. // Returns nil if there is an error in handshake.
  80. // Caller should call conn.Close()
  81. // See docs/sts-final.pdf for more information.
  82. func MakeSecretConnection(conn io.ReadWriteCloser, locPrivKey crypto.PrivKey) (*SecretConnection, error) {
  83. var (
  84. locPubKey = locPrivKey.PubKey()
  85. )
  86. // Generate ephemeral keys for perfect forward secrecy.
  87. locEphPub, locEphPriv, err := genEphKeys()
  88. if err != nil {
  89. return nil, err
  90. }
  91. // Write local ephemeral pubkey and receive one too.
  92. // NOTE: every 32-byte string is accepted as a Curve25519 public key (see
  93. // DJB's Curve25519 paper: http://cr.yp.to/ecdh/curve25519-20060209.pdf)
  94. remEphPub, err := shareEphPubKey(conn, locEphPub)
  95. if err != nil {
  96. return nil, err
  97. }
  98. // Sort by lexical order.
  99. loEphPub, hiEphPub := sort32(locEphPub, remEphPub)
  100. transcript := merlin.NewTranscript("TENDERMINT_SECRET_CONNECTION_TRANSCRIPT_HASH")
  101. transcript.AppendMessage(labelEphemeralLowerPublicKey, loEphPub[:])
  102. transcript.AppendMessage(labelEphemeralUpperPublicKey, hiEphPub[:])
  103. // Check if the local ephemeral public key was the least, lexicographically
  104. // sorted.
  105. locIsLeast := bytes.Equal(locEphPub[:], loEphPub[:])
  106. // Compute common diffie hellman secret using X25519.
  107. dhSecret, err := computeDHSecret(remEphPub, locEphPriv)
  108. if err != nil {
  109. return nil, err
  110. }
  111. transcript.AppendMessage(labelDHSecret, dhSecret[:])
  112. // Generate the secret used for receiving, sending, challenge via HKDF-SHA2
  113. // on the transcript state (which itself also uses HKDF-SHA2 to derive a key
  114. // from the dhSecret).
  115. recvSecret, sendSecret, err := deriveSecrets(dhSecret, locIsLeast)
  116. if err != nil {
  117. return nil, err
  118. }
  119. const challengeSize = 32
  120. var challenge [challengeSize]byte
  121. transcript.ExtractBytes(challenge[:], labelSecretConnectionMac)
  122. sendAead, err := chacha20poly1305.New(sendSecret[:])
  123. if err != nil {
  124. return nil, errors.New("invalid send SecretConnection Key")
  125. }
  126. recvAead, err := chacha20poly1305.New(recvSecret[:])
  127. if err != nil {
  128. return nil, errors.New("invalid receive SecretConnection Key")
  129. }
  130. sc := &SecretConnection{
  131. conn: conn,
  132. recvBuffer: nil,
  133. recvNonce: new([aeadNonceSize]byte),
  134. sendNonce: new([aeadNonceSize]byte),
  135. recvAead: recvAead,
  136. sendAead: sendAead,
  137. }
  138. // Sign the challenge bytes for authentication.
  139. locSignature, err := signChallenge(&challenge, locPrivKey)
  140. if err != nil {
  141. return nil, err
  142. }
  143. // Share (in secret) each other's pubkey & challenge signature
  144. authSigMsg, err := shareAuthSignature(sc, locPubKey, locSignature)
  145. if err != nil {
  146. return nil, err
  147. }
  148. remPubKey, remSignature := authSigMsg.Key, authSigMsg.Sig
  149. if _, ok := remPubKey.(ed25519.PubKey); !ok {
  150. return nil, fmt.Errorf("expected ed25519 pubkey, got %T", remPubKey)
  151. }
  152. if !remPubKey.VerifySignature(challenge[:], remSignature) {
  153. return nil, errors.New("challenge verification failed")
  154. }
  155. // We've authorized.
  156. sc.remPubKey = remPubKey
  157. return sc, nil
  158. }
  159. // RemotePubKey returns authenticated remote pubkey
  160. func (sc *SecretConnection) RemotePubKey() crypto.PubKey {
  161. return sc.remPubKey
  162. }
  163. // Writes encrypted frames of `totalFrameSize + aeadSizeOverhead`.
  164. // CONTRACT: data smaller than dataMaxSize is written atomically.
  165. func (sc *SecretConnection) Write(data []byte) (n int, err error) {
  166. sc.sendMtx.Lock()
  167. defer sc.sendMtx.Unlock()
  168. for 0 < len(data) {
  169. if err := func() error {
  170. var sealedFrame = pool.Get(aeadSizeOverhead + totalFrameSize)
  171. var frame = pool.Get(totalFrameSize)
  172. defer func() {
  173. pool.Put(sealedFrame)
  174. pool.Put(frame)
  175. }()
  176. var chunk []byte
  177. if dataMaxSize < len(data) {
  178. chunk = data[:dataMaxSize]
  179. data = data[dataMaxSize:]
  180. } else {
  181. chunk = data
  182. data = nil
  183. }
  184. chunkLength := len(chunk)
  185. binary.LittleEndian.PutUint32(frame, uint32(chunkLength))
  186. copy(frame[dataLenSize:], chunk)
  187. // encrypt the frame
  188. sc.sendAead.Seal(sealedFrame[:0], sc.sendNonce[:], frame, nil)
  189. if err := incrNonce(sc.sendNonce); err != nil {
  190. return err
  191. }
  192. // end encryption
  193. _, err = sc.conn.Write(sealedFrame)
  194. if err != nil {
  195. return err
  196. }
  197. n += len(chunk)
  198. return nil
  199. }(); err != nil {
  200. return n, err
  201. }
  202. }
  203. return n, err
  204. }
  205. // CONTRACT: data smaller than dataMaxSize is read atomically.
  206. func (sc *SecretConnection) Read(data []byte) (n int, err error) {
  207. sc.recvMtx.Lock()
  208. defer sc.recvMtx.Unlock()
  209. // read off and update the recvBuffer, if non-empty
  210. if 0 < len(sc.recvBuffer) {
  211. n = copy(data, sc.recvBuffer)
  212. sc.recvBuffer = sc.recvBuffer[n:]
  213. return
  214. }
  215. // read off the conn
  216. var sealedFrame = pool.Get(aeadSizeOverhead + totalFrameSize)
  217. defer pool.Put(sealedFrame)
  218. _, err = io.ReadFull(sc.conn, sealedFrame)
  219. if err != nil {
  220. return
  221. }
  222. // decrypt the frame.
  223. // reads and updates the sc.recvNonce
  224. var frame = pool.Get(totalFrameSize)
  225. defer pool.Put(frame)
  226. _, err = sc.recvAead.Open(frame[:0], sc.recvNonce[:], sealedFrame, nil)
  227. if err != nil {
  228. return n, fmt.Errorf("failed to decrypt SecretConnection: %w", err)
  229. }
  230. if err = incrNonce(sc.recvNonce); err != nil {
  231. return
  232. }
  233. // end decryption
  234. // copy checkLength worth into data,
  235. // set recvBuffer to the rest.
  236. var chunkLength = binary.LittleEndian.Uint32(frame) // read the first four bytes
  237. if chunkLength > dataMaxSize {
  238. return 0, errors.New("chunkLength is greater than dataMaxSize")
  239. }
  240. var chunk = frame[dataLenSize : dataLenSize+chunkLength]
  241. n = copy(data, chunk)
  242. if n < len(chunk) {
  243. sc.recvBuffer = make([]byte, len(chunk)-n)
  244. copy(sc.recvBuffer, chunk[n:])
  245. }
  246. return n, err
  247. }
  248. // Implements net.Conn
  249. func (sc *SecretConnection) Close() error { return sc.conn.Close() }
  250. func (sc *SecretConnection) LocalAddr() net.Addr { return sc.conn.(net.Conn).LocalAddr() }
  251. func (sc *SecretConnection) RemoteAddr() net.Addr { return sc.conn.(net.Conn).RemoteAddr() }
  252. func (sc *SecretConnection) SetDeadline(t time.Time) error { return sc.conn.(net.Conn).SetDeadline(t) }
  253. func (sc *SecretConnection) SetReadDeadline(t time.Time) error {
  254. return sc.conn.(net.Conn).SetReadDeadline(t)
  255. }
  256. func (sc *SecretConnection) SetWriteDeadline(t time.Time) error {
  257. return sc.conn.(net.Conn).SetWriteDeadline(t)
  258. }
  259. func genEphKeys() (ephPub, ephPriv *[32]byte, err error) {
  260. // TODO: Probably not a problem but ask Tony: different from the rust implementation (uses x25519-dalek),
  261. // we do not "clamp" the private key scalar:
  262. // see: https://github.com/dalek-cryptography/x25519-dalek/blob/34676d336049df2bba763cc076a75e47ae1f170f/src/x25519.rs#L56-L74
  263. ephPub, ephPriv, err = box.GenerateKey(crand.Reader)
  264. if err != nil {
  265. return
  266. }
  267. return
  268. }
  269. func shareEphPubKey(conn io.ReadWriter, locEphPub *[32]byte) (remEphPub *[32]byte, err error) {
  270. // Send our pubkey and receive theirs in tandem.
  271. var trs, _ = async.Parallel(
  272. func(_ int) (val interface{}, abort bool, err error) {
  273. lc := *locEphPub
  274. _, err = protoio.NewDelimitedWriter(conn).WriteMsg(&gogotypes.BytesValue{Value: lc[:]})
  275. if err != nil {
  276. return nil, true, err // abort
  277. }
  278. return nil, false, nil
  279. },
  280. func(_ int) (val interface{}, abort bool, err error) {
  281. var bytes gogotypes.BytesValue
  282. _, err = protoio.NewDelimitedReader(conn, 1024*1024).ReadMsg(&bytes)
  283. if err != nil {
  284. return nil, true, err // abort
  285. }
  286. var _remEphPub [32]byte
  287. copy(_remEphPub[:], bytes.Value)
  288. return _remEphPub, false, nil
  289. },
  290. )
  291. // If error:
  292. if trs.FirstError() != nil {
  293. err = trs.FirstError()
  294. return
  295. }
  296. // Otherwise:
  297. var _remEphPub = trs.FirstValue().([32]byte)
  298. return &_remEphPub, nil
  299. }
  300. func deriveSecrets(
  301. dhSecret *[32]byte,
  302. locIsLeast bool,
  303. ) (recvSecret, sendSecret *[aeadKeySize]byte, err error) {
  304. hash := sha256.New
  305. hkdf := hkdf.New(hash, dhSecret[:], nil, secretConnKeyAndChallengeGen)
  306. // get enough data for 2 aead keys, and a 32 byte challenge
  307. res := new([2*aeadKeySize + 32]byte)
  308. _, err = io.ReadFull(hkdf, res[:])
  309. if err != nil {
  310. return nil, nil, err
  311. }
  312. recvSecret = new([aeadKeySize]byte)
  313. sendSecret = new([aeadKeySize]byte)
  314. // bytes 0 through aeadKeySize - 1 are one aead key.
  315. // bytes aeadKeySize through 2*aeadKeySize -1 are another aead key.
  316. // which key corresponds to sending and receiving key depends on whether
  317. // the local key is less than the remote key.
  318. if locIsLeast {
  319. copy(recvSecret[:], res[0:aeadKeySize])
  320. copy(sendSecret[:], res[aeadKeySize:aeadKeySize*2])
  321. } else {
  322. copy(sendSecret[:], res[0:aeadKeySize])
  323. copy(recvSecret[:], res[aeadKeySize:aeadKeySize*2])
  324. }
  325. return
  326. }
  327. // computeDHSecret computes a Diffie-Hellman shared secret key
  328. // from our own local private key and the other's public key.
  329. func computeDHSecret(remPubKey, locPrivKey *[32]byte) (*[32]byte, error) {
  330. shrKey, err := curve25519.X25519(locPrivKey[:], remPubKey[:])
  331. if err != nil {
  332. return nil, err
  333. }
  334. var shrKeyArray [32]byte
  335. copy(shrKeyArray[:], shrKey)
  336. return &shrKeyArray, nil
  337. }
  338. func sort32(foo, bar *[32]byte) (lo, hi *[32]byte) {
  339. if bytes.Compare(foo[:], bar[:]) < 0 {
  340. lo = foo
  341. hi = bar
  342. } else {
  343. lo = bar
  344. hi = foo
  345. }
  346. return
  347. }
  348. func signChallenge(challenge *[32]byte, locPrivKey crypto.PrivKey) ([]byte, error) {
  349. signature, err := locPrivKey.Sign(challenge[:])
  350. if err != nil {
  351. return nil, err
  352. }
  353. return signature, nil
  354. }
  355. type authSigMessage struct {
  356. Key crypto.PubKey
  357. Sig []byte
  358. }
  359. func shareAuthSignature(sc io.ReadWriter, pubKey crypto.PubKey, signature []byte) (recvMsg authSigMessage, err error) {
  360. // Send our info and receive theirs in tandem.
  361. var trs, _ = async.Parallel(
  362. func(_ int) (val interface{}, abort bool, err error) {
  363. pbpk, err := encoding.PubKeyToProto(pubKey)
  364. if err != nil {
  365. return nil, true, err
  366. }
  367. _, err = protoio.NewDelimitedWriter(sc).WriteMsg(&tmprivval.AuthSigMessage{PubKey: pbpk, Sig: signature})
  368. if err != nil {
  369. return nil, true, err // abort
  370. }
  371. return nil, false, nil
  372. },
  373. func(_ int) (val interface{}, abort bool, err error) {
  374. var pba tmprivval.AuthSigMessage
  375. _, err = protoio.NewDelimitedReader(sc, 1024*1024).ReadMsg(&pba)
  376. if err != nil {
  377. return nil, true, err // abort
  378. }
  379. pk, err := encoding.PubKeyFromProto(pba.PubKey)
  380. if err != nil {
  381. return nil, true, err // abort
  382. }
  383. _recvMsg := authSigMessage{
  384. Key: pk,
  385. Sig: pba.Sig,
  386. }
  387. return _recvMsg, false, nil
  388. },
  389. )
  390. // If error:
  391. if trs.FirstError() != nil {
  392. err = trs.FirstError()
  393. return
  394. }
  395. var _recvMsg = trs.FirstValue().(authSigMessage)
  396. return _recvMsg, nil
  397. }
  398. //--------------------------------------------------------------------------------
  399. // Increment nonce little-endian by 1 with wraparound.
  400. // Due to chacha20poly1305 expecting a 12 byte nonce we do not use the first four
  401. // bytes. We only increment a 64 bit unsigned int in the remaining 8 bytes
  402. // (little-endian in nonce[4:]).
  403. func incrNonce(nonce *[aeadNonceSize]byte) error {
  404. counter := binary.LittleEndian.Uint64(nonce[4:])
  405. if counter == math.MaxUint64 {
  406. // Terminates the session and makes sure the nonce would not re-used.
  407. // See https://github.com/tendermint/tendermint/issues/3531
  408. return errors.New("can't increase nonce without overflow")
  409. }
  410. counter++
  411. binary.LittleEndian.PutUint64(nonce[4:], counter)
  412. return nil
  413. }