chore: move pk2id and id2pk to primitives (#7382)

This commit is contained in:
Dan Cline
2024-03-29 17:43:52 -04:00
committed by GitHub
parent b1026e0e23
commit c13d7da68d
17 changed files with 89 additions and 97 deletions

View File

@ -201,14 +201,14 @@ impl Discv4 {
/// # use std::io;
/// use rand::thread_rng;
/// use reth_discv4::{Discv4, Discv4Config};
/// use reth_primitives::{NodeRecord, PeerId};
/// use reth_primitives::{pk2id, NodeRecord, PeerId};
/// use secp256k1::SECP256K1;
/// use std::{net::SocketAddr, str::FromStr};
/// # async fn t() -> io::Result<()> {
/// // generate a (random) keypair
/// let mut rng = thread_rng();
/// let (secret_key, pk) = SECP256K1.generate_keypair(&mut rng);
/// let id = PeerId::from_slice(&pk.serialize_uncompressed()[1..]);
/// let id = pk2id(&pk);
///
/// let socket = SocketAddr::from_str("0.0.0.0:0").unwrap();
/// let local_enr =

View File

@ -7,7 +7,7 @@ use alloy_rlp::{
use enr::{Enr, EnrKey};
use reth_primitives::{
bytes::{Buf, BufMut, Bytes, BytesMut},
keccak256, ForkId, NodeRecord, B256,
keccak256, pk2id, ForkId, NodeRecord, B256,
};
use secp256k1::{
ecdsa::{RecoverableSignature, RecoveryId},
@ -161,7 +161,7 @@ impl Message {
let msg = secp256k1::Message::from_slice(keccak256(&packet[97..]).as_slice())?;
let pk = SECP256K1.recover_ecdsa(&msg, &recoverable_sig)?;
let node_id = PeerId::from_slice(&pk.serialize_uncompressed()[1..]);
let node_id = pk2id(&pk);
let msg_type = packet[97];
let payload = &mut &packet[98..];
@ -724,7 +724,7 @@ mod tests {
for _ in 0..100 {
let msg = rng_message(&mut rng);
let (secret_key, pk) = SECP256K1.generate_keypair(&mut rng);
let sender_id = PeerId::from_slice(&pk.serialize_uncompressed()[1..]);
let sender_id = pk2id(&pk);
let (buf, _) = msg.encode(&secret_key);

View File

@ -6,7 +6,7 @@ use crate::{
IngressReceiver, PeerId, SAFE_MAX_DATAGRAM_NEIGHBOUR_RECORDS,
};
use rand::{thread_rng, Rng, RngCore};
use reth_primitives::{hex, ForkHash, ForkId, NodeRecord, B256};
use reth_primitives::{hex, pk2id, ForkHash, ForkId, NodeRecord, B256};
use secp256k1::{SecretKey, SECP256K1};
use std::{
collections::{HashMap, HashSet},
@ -49,7 +49,7 @@ impl MockDiscovery {
let mut rng = thread_rng();
let socket = SocketAddr::from_str("0.0.0.0:0").unwrap();
let (secret_key, pk) = SECP256K1.generate_keypair(&mut rng);
let id = PeerId::from_slice(&pk.serialize_uncompressed()[1..]);
let id = pk2id(&pk);
let socket = Arc::new(UdpSocket::bind(socket).await?);
let local_addr = socket.local_addr()?;
let local_enr = NodeRecord {
@ -241,7 +241,7 @@ pub async fn create_discv4_with_config(config: Discv4Config) -> (Discv4, Discv4S
let mut rng = thread_rng();
let socket = SocketAddr::from_str("0.0.0.0:0").unwrap();
let (secret_key, pk) = SECP256K1.generate_keypair(&mut rng);
let id = PeerId::from_slice(&pk.serialize_uncompressed()[1..]);
let id = pk2id(&pk);
let local_enr =
NodeRecord { address: socket.ip(), tcp_port: socket.port(), udp_port: socket.port(), id };
Discv4::bind(socket, local_enr, secret_key, config).await.unwrap()

View File

@ -22,7 +22,7 @@ use crate::{
pub use config::DnsDiscoveryConfig;
use enr::Enr;
use error::ParseDnsEntryError;
use reth_primitives::{ForkId, NodeRecord, PeerId};
use reth_primitives::{pk2id, ForkId, NodeRecord};
use schnellru::{ByLength, LruMap};
use secp256k1::SecretKey;
use std::{
@ -398,7 +398,7 @@ fn convert_enr_node_record(enr: &Enr<SecretKey>) -> Option<DnsNodeRecordUpdate>
address: enr.ip4().map(IpAddr::from).or_else(|| enr.ip6().map(IpAddr::from))?,
tcp_port: enr.tcp4().or_else(|| enr.tcp6())?,
udp_port: enr.udp4().or_else(|| enr.udp6())?,
id: PeerId::from_slice(&enr.public_key().serialize_uncompressed()[1..]),
id: pk2id(&enr.public_key()),
}
.into_ipv4_mapped();

View File

@ -3,7 +3,7 @@
use crate::{
error::ECIESErrorImpl,
mac::{HeaderBytes, MAC},
util::{hmac_sha256, id2pk, pk2id, sha256},
util::{hmac_sha256, sha256},
ECIESError,
};
use aes::{cipher::StreamCipher, Aes128, Aes256};
@ -15,7 +15,7 @@ use educe::Educe;
use rand::{thread_rng, Rng};
use reth_primitives::{
bytes::{BufMut, Bytes, BytesMut},
B128, B256, B512 as PeerId,
id2pk, pk2id, B128, B256, B512 as PeerId,
};
use secp256k1::{
ecdsa::{RecoverableSignature, RecoveryId},

View File

@ -175,7 +175,7 @@ where
#[cfg(test)]
mod tests {
use super::*;
use crate::util::pk2id;
use reth_primitives::pk2id;
use secp256k1::SECP256K1;
use tokio::net::{TcpListener, TcpStream};

View File

@ -1,8 +1,7 @@
//! Utility functions for hashing and encoding.
use hmac::{Hmac, Mac};
use reth_primitives::{B256, B512 as PeerId};
use secp256k1::PublicKey;
use reth_primitives::B256;
use sha2::{Digest, Sha256};
/// Hashes the input data with SHA256.
@ -21,35 +20,3 @@ pub(crate) fn hmac_sha256(key: &[u8], input: &[&[u8]], auth_data: &[u8]) -> B256
hmac.update(auth_data);
B256::from_slice(&hmac.finalize().into_bytes())
}
/// Converts a [secp256k1::PublicKey] to a [PeerId] by stripping the
/// SECP256K1_TAG_PUBKEY_UNCOMPRESSED tag and storing the rest of the slice in the [PeerId].
pub fn pk2id(pk: &PublicKey) -> PeerId {
PeerId::from_slice(&pk.serialize_uncompressed()[1..])
}
/// Converts a [PeerId] to a [secp256k1::PublicKey] by prepending the [PeerId] bytes with the
/// SECP256K1_TAG_PUBKEY_UNCOMPRESSED tag.
pub(crate) fn id2pk(id: PeerId) -> Result<PublicKey, secp256k1::Error> {
// NOTE: B512 is used as a PeerId not because it represents a hash, but because 512 bits is
// enough to represent an uncompressed public key.
let mut s = [0u8; 65];
// SECP256K1_TAG_PUBKEY_UNCOMPRESSED = 0x04
// see: https://github.com/bitcoin-core/secp256k1/blob/master/include/secp256k1.h#L211
s[0] = 4;
s[1..].copy_from_slice(id.as_slice());
PublicKey::from_slice(&s)
}
#[cfg(test)]
mod tests {
use super::*;
use secp256k1::{SecretKey, SECP256K1};
#[test]
fn pk2id2pk() {
let prikey = SecretKey::new(&mut secp256k1::rand::thread_rng());
let pubkey = PublicKey::from_secret_key(SECP256K1, &prikey);
assert_eq!(pubkey, id2pk(pk2id(&pubkey)).unwrap());
}
}

View File

@ -345,8 +345,6 @@ where
#[cfg(test)]
mod tests {
use std::time::Duration;
use super::UnauthedEthStream;
use crate::{
errors::{EthHandshakeError, EthStreamError},
@ -357,9 +355,10 @@ mod tests {
use alloy_chains::NamedChain;
use futures::{SinkExt, StreamExt};
use reth_discv4::DEFAULT_DISCOVERY_PORT;
use reth_ecies::{stream::ECIESStream, util::pk2id};
use reth_primitives::{ForkFilter, Head, B256, U256};
use reth_ecies::stream::ECIESStream;
use reth_primitives::{pk2id, ForkFilter, Head, B256, U256};
use secp256k1::{SecretKey, SECP256K1};
use std::time::Duration;
use tokio::net::{TcpListener, TcpStream};
use tokio_util::codec::Decoder;

View File

@ -37,8 +37,8 @@ impl HelloMessageWithProtocols {
/// Starts a new `HelloMessageProtocolsBuilder`
///
/// ```
/// use reth_ecies::util::pk2id;
/// use reth_eth_wire::HelloMessageWithProtocols;
/// use reth_primitives::pk2id;
/// use secp256k1::{SecretKey, SECP256K1};
/// let secret_key = SecretKey::new(&mut rand::thread_rng());
/// let id = pk2id(&secret_key.public_key(SECP256K1));
@ -119,8 +119,8 @@ impl HelloMessage {
/// Starts a new `HelloMessageBuilder`
///
/// ```
/// use reth_ecies::util::pk2id;
/// use reth_eth_wire::HelloMessage;
/// use reth_primitives::pk2id;
/// use secp256k1::{SecretKey, SECP256K1};
/// let secret_key = SecretKey::new(&mut rand::thread_rng());
/// let id = pk2id(&secret_key.public_key(SECP256K1));
@ -209,7 +209,7 @@ impl HelloMessageBuilder {
mod tests {
use alloy_rlp::{Decodable, Encodable, EMPTY_STRING_CODE};
use reth_discv4::DEFAULT_DISCOVERY_PORT;
use reth_ecies::util::pk2id;
use reth_primitives::pk2id;
use secp256k1::{SecretKey, SECP256K1};
use crate::{

View File

@ -21,23 +21,21 @@
//! [`StreamClone`] buffers egress bytes for [`MuxDemuxer`] that are read and sent to the network
//! when [`MuxDemuxStream`] is polled.
use std::{
collections::HashMap,
pin::Pin,
task::{ready, Context, Poll},
};
use derive_more::{Deref, DerefMut};
use futures::{Sink, SinkExt, StreamExt};
use reth_primitives::bytes::{Bytes, BytesMut};
use tokio::sync::mpsc;
use tokio_stream::Stream;
use crate::{
capability::{Capability, SharedCapabilities, SharedCapability},
errors::MuxDemuxError,
CanDisconnect, DisconnectP2P, DisconnectReason,
};
use derive_more::{Deref, DerefMut};
use futures::{Sink, SinkExt, StreamExt};
use reth_primitives::bytes::{Bytes, BytesMut};
use std::{
collections::HashMap,
pin::Pin,
task::{ready, Context, Poll},
};
use tokio::sync::mpsc;
use tokio_stream::Stream;
use MuxDemuxError::*;
@ -351,21 +349,6 @@ impl CanDisconnect<Bytes> for StreamClone {
#[cfg(test)]
mod tests {
use std::{net::SocketAddr, pin::Pin};
use futures::{Future, SinkExt, StreamExt};
use reth_ecies::util::pk2id;
use reth_primitives::{
bytes::{BufMut, Bytes, BytesMut},
ForkFilter, Hardfork, MAINNET,
};
use secp256k1::{SecretKey, SECP256K1};
use tokio::{
net::{TcpListener, TcpStream},
task::JoinHandle,
};
use tokio_util::codec::{Decoder, Framed, LengthDelimitedCodec};
use crate::{
capability::{Capability, SharedCapabilities},
muxdemux::MuxDemuxStream,
@ -373,6 +356,18 @@ mod tests {
EthVersion, HelloMessageWithProtocols, Status, StatusBuilder, StreamClone,
UnauthedEthStream, UnauthedP2PStream,
};
use futures::{Future, SinkExt, StreamExt};
use reth_primitives::{
bytes::{BufMut, Bytes, BytesMut},
pk2id, ForkFilter, Hardfork, MAINNET,
};
use secp256k1::{SecretKey, SECP256K1};
use std::{net::SocketAddr, pin::Pin};
use tokio::{
net::{TcpListener, TcpStream},
task::JoinHandle,
};
use tokio_util::codec::{Decoder, Framed, LengthDelimitedCodec};
const ETH_68_CAP: Capability = Capability::eth(EthVersion::Eth68);
const ETH_68_PROTOCOL: Protocol = Protocol::new(ETH_68_CAP, 13);

View File

@ -5,8 +5,7 @@ use crate::{
};
use alloy_chains::Chain;
use reth_discv4::DEFAULT_DISCOVERY_PORT;
use reth_ecies::util::pk2id;
use reth_primitives::{ForkFilter, Head, B256, U256};
use reth_primitives::{pk2id, ForkFilter, Head, B256, U256};
use secp256k1::{SecretKey, SECP256K1};
use std::net::SocketAddr;
use tokio::net::TcpStream;

View File

@ -10,10 +10,9 @@ use crate::{
};
use reth_discv4::{Discv4Config, Discv4ConfigBuilder, DEFAULT_DISCOVERY_ADDRESS};
use reth_dns_discovery::DnsDiscoveryConfig;
use reth_ecies::util::pk2id;
use reth_eth_wire::{HelloMessage, HelloMessageWithProtocols, Status};
use reth_primitives::{
mainnet_nodes, sepolia_nodes, ChainSpec, ForkFilter, Head, NodeRecord, PeerId, MAINNET,
mainnet_nodes, pk2id, sepolia_nodes, ChainSpec, ForkFilter, Head, NodeRecord, PeerId, MAINNET,
};
use reth_provider::{BlockReader, HeaderProvider};
use reth_tasks::{TaskSpawner, TokioTaskExecutor};

View File

@ -20,7 +20,6 @@ use reth_eth_wire::{
};
use reth_interfaces::p2p::error::RequestError;
use reth_metrics::common::mpsc::MeteredPollSender;
use reth_primitives::PeerId;
use std::{
collections::VecDeque,
@ -764,13 +763,13 @@ mod tests {
config::PROTOCOL_BREACH_REQUEST_TIMEOUT, handle::PendingSessionEvent,
start_pending_incoming_session,
};
use reth_ecies::{stream::ECIESStream, util::pk2id};
use reth_ecies::stream::ECIESStream;
use reth_eth_wire::{
EthStream, GetBlockBodies, HelloMessageWithProtocols, P2PStream, Status, StatusBuilder,
UnauthedEthStream, UnauthedP2PStream,
};
use reth_net_common::bandwidth_meter::{BandwidthMeter, MeteredStream};
use reth_primitives::{ForkFilter, Hardfork, MAINNET};
use reth_primitives::{pk2id, ForkFilter, Hardfork, MAINNET};
use secp256k1::{SecretKey, SECP256K1};
use tokio::{
net::{TcpListener, TcpStream},

View File

@ -74,7 +74,7 @@ pub use net::{
goerli_nodes, holesky_nodes, mainnet_nodes, parse_nodes, sepolia_nodes, NodeRecord,
GOERLI_BOOTNODES, HOLESKY_BOOTNODES, MAINNET_BOOTNODES, SEPOLIA_BOOTNODES,
};
pub use peer::{AnyNode, PeerId, WithPeerId};
pub use peer::{id2pk, pk2id, AnyNode, PeerId, WithPeerId};
pub use prune::{
PruneCheckpoint, PruneMode, PruneModes, PruneProgress, PrunePurpose, PruneSegment,
PruneSegmentError, ReceiptsLogPruneConfig, MINIMUM_PRUNING_DISTANCE,

View File

@ -1,10 +1,38 @@
use enr::Enr;
use reth_rpc_types::NodeRecord;
use secp256k1::SecretKey;
use secp256k1::{constants::UNCOMPRESSED_PUBLIC_KEY_SIZE, PublicKey, SecretKey};
use std::{net::IpAddr, str::FromStr};
// Re-export PeerId for ease of use.
pub use reth_rpc_types::PeerId;
/// This tag should be set to indicate to libsecp256k1 that the following bytes denote an
/// uncompressed pubkey.
///
/// `SECP256K1_TAG_PUBKEY_UNCOMPRESSED` = `0x04`
///
/// See: <https://github.com/bitcoin-core/secp256k1/blob/master/include/secp256k1.h#L211>
const SECP256K1_TAG_PUBKEY_UNCOMPRESSED: u8 = 4;
/// Converts a [secp256k1::PublicKey] to a [PeerId] by stripping the
/// `SECP256K1_TAG_PUBKEY_UNCOMPRESSED` tag and storing the rest of the slice in the [PeerId].
#[inline]
pub fn pk2id(pk: &PublicKey) -> PeerId {
PeerId::from_slice(&pk.serialize_uncompressed()[1..])
}
/// Converts a [PeerId] to a [secp256k1::PublicKey] by prepending the [PeerId] bytes with the
/// SECP256K1_TAG_PUBKEY_UNCOMPRESSED tag.
#[inline]
pub fn id2pk(id: PeerId) -> Result<PublicKey, secp256k1::Error> {
// NOTE: B512 is used as a PeerId because 512 bits is enough to represent an uncompressed
// public key.
let mut s = [0u8; UNCOMPRESSED_PUBLIC_KEY_SIZE];
s[0] = SECP256K1_TAG_PUBKEY_UNCOMPRESSED;
s[1..].copy_from_slice(id.as_slice());
PublicKey::from_slice(&s)
}
/// A peer that can come in ENR or [NodeRecord] form.
#[derive(
Debug, Clone, Eq, PartialEq, Hash, serde_with::SerializeDisplay, serde_with::DeserializeFromStr,
@ -23,9 +51,7 @@ impl AnyNode {
pub fn peer_id(&self) -> PeerId {
match self {
AnyNode::NodeRecord(record) => record.id,
AnyNode::Enr(enr) => {
PeerId::from_slice(&enr.public_key().serialize_uncompressed()[1..])
}
AnyNode::Enr(enr) => pk2id(&enr.public_key()),
AnyNode::PeerId(peer_id) => *peer_id,
}
}
@ -39,7 +65,7 @@ impl AnyNode {
address: enr.ip4().map(IpAddr::from).or_else(|| enr.ip6().map(IpAddr::from))?,
tcp_port: enr.tcp4().or_else(|| enr.tcp6())?,
udp_port: enr.udp4().or_else(|| enr.udp6())?,
id: PeerId::from_slice(&enr.public_key().serialize_uncompressed()[1..]),
id: pk2id(&enr.public_key()),
}
.into_ipv4_mapped();
Some(node_record)
@ -151,6 +177,7 @@ impl<T> WithPeerId<Option<T>> {
#[cfg(test)]
mod tests {
use super::*;
use secp256k1::SECP256K1;
#[test]
fn test_node_record_parse() {
@ -190,4 +217,11 @@ mod tests {
);
assert_eq!(node.to_string(), url);
}
#[test]
fn pk2id2pk() {
let prikey = SecretKey::new(&mut secp256k1::rand::thread_rng());
let pubkey = PublicKey::from_secret_key(SECP256K1, &prikey);
assert_eq!(pubkey, id2pk(pk2id(&pubkey)).unwrap());
}
}

View File

@ -304,7 +304,7 @@ mod tests {
let cases = vec![
// IPv4
(
"\"enode://6f8a80d14311c39f35f516fa664deaaaa13e85b2f7493f37f6144d86991ec012937307647bd3b9a82abe2974e1407241d54947bbb39763a4cac9f77166ad92a0@10.3.58.6:30303?discport=30301\"",
"\"enode://6f8a80d14311c39f35f516fa664deaaaa13e85b2f7493f37f6144d86991ec012937307647bd3b9a82abe2974e1407241d54947bbb39763a4cac9f77166ad92a0@10.3.58.6:30303?discport=30301\"",
NodeRecord{
address: IpAddr::V4([10, 3, 58, 6].into()),
tcp_port: 30303u16,