Skip to content

Commit

Permalink
[PAN-2585] Prefer EnodeURL over Endpoint (PegaSysEng#1378)
Browse files Browse the repository at this point in the history
  • Loading branch information
mbaxter authored and notlesh committed May 14, 2019
1 parent de49103 commit b4dad69
Show file tree
Hide file tree
Showing 45 changed files with 518 additions and 640 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@
import tech.pegasys.pantheon.ethereum.p2p.config.RlpxConfiguration;
import tech.pegasys.pantheon.ethereum.p2p.network.DefaultP2PNetwork;
import tech.pegasys.pantheon.ethereum.p2p.peers.DefaultPeer;
import tech.pegasys.pantheon.ethereum.p2p.peers.Endpoint;
import tech.pegasys.pantheon.ethereum.p2p.peers.Peer;
import tech.pegasys.pantheon.ethereum.p2p.peers.PeerBlacklist;
import tech.pegasys.pantheon.ethereum.p2p.wire.messages.DisconnectMessage.DisconnectReason;
Expand All @@ -55,10 +54,8 @@

import java.io.Closeable;
import java.io.IOException;
import java.net.InetAddress;
import java.util.HashMap;
import java.util.Map;
import java.util.OptionalInt;
import java.util.concurrent.CompletableFuture;

import io.vertx.core.Vertx;
Expand All @@ -70,7 +67,6 @@ public class TestNode implements Closeable {
private static final Logger LOG = LogManager.getLogger();
private static final MetricsSystem metricsSystem = new NoOpMetricsSystem();

protected final Integer port;
protected final SECP256K1.KeyPair kp;
protected final P2PNetwork network;
protected final Peer selfPeer;
Expand Down Expand Up @@ -156,8 +152,7 @@ public TestNode(
PendingTransactions.DEFAULT_TX_RETENTION_HOURS);

networkRunner.start();
this.port = network.getLocalEnode().get().getListeningPort();
selfPeer = new DefaultPeer(id(), endpoint());
selfPeer = DefaultPeer.fromEnodeURL(network.getLocalEnode().get());
}

public BytesValue id() {
Expand All @@ -172,13 +167,6 @@ public String shortId() {
return shortId(id());
}

public Endpoint endpoint() {
checkNotNull(
port, "Must either pass port to ctor, or call createNetwork() first to set the port");
return new Endpoint(
InetAddress.getLoopbackAddress().getHostAddress(), port, OptionalInt.of(port));
}

public Peer selfPeer() {
return selfPeer;
}
Expand Down Expand Up @@ -209,9 +197,9 @@ public void close() throws IOException {
public String toString() {
return shortId()
+ "@"
+ selfPeer.getEndpoint().getHost()
+ selfPeer.getEnodeURL().getIpAsString()
+ ':'
+ selfPeer.getEndpoint().getTcpPort();
+ selfPeer.getEnodeURL().getListeningPort();
}

public void receiveRemoteTransaction(final Transaction transaction) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,8 +73,8 @@ public JsonRpcResponse response(final JsonRpcRequest req) {
final BytesValue nodeId = enode.getNodeId();
response.put("enode", enode.toString());
ports.put("discovery", enode.getEffectiveDiscoveryPort());
response.put("ip", enode.getIp());
response.put("listenAddr", enode.getIp() + ":" + enode.getListeningPort());
response.put("ip", enode.getIpAsString());
response.put("listenAddr", enode.getIpAsString() + ":" + enode.getListeningPort());
response.put("id", nodeId.toUnprefixedString());
response.put("name", clientVersion);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,8 @@ public JsonRpcResponse response(final JsonRpcRequest req) {
.ifPresent(
enode ->
servicesMapBuilder.put(
"p2p", createServiceDetailsMap(enode.getIp(), enode.getListeningPort())));
"p2p",
createServiceDetailsMap(enode.getIpAsString(), enode.getListeningPort())));
}
if (metricsConfiguration.isEnabled()) {
servicesMapBuilder.put(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import tech.pegasys.pantheon.ethereum.p2p.api.P2PNetwork;
import tech.pegasys.pantheon.ethereum.p2p.peers.DefaultPeer;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import tech.pegasys.pantheon.util.enode.EnodeURL;
import tech.pegasys.pantheon.util.uint.UInt256;

import java.math.BigInteger;
Expand Down Expand Up @@ -59,7 +60,14 @@ public class AdminNodeInfoTest {
private final ChainHead testChainHead = new ChainHead(Hash.EMPTY, UInt256.ONE);
private final GenesisConfigOptions genesisConfigOptions =
new StubGenesisConfigOptions().chainId(BigInteger.valueOf(2019));
private final DefaultPeer defaultPeer = new DefaultPeer(nodeId, "1.2.3.4", 7890, 30303);
private final DefaultPeer defaultPeer =
DefaultPeer.fromEnodeURL(
EnodeURL.builder()
.nodeId(nodeId)
.ipAddress("1.2.3.4")
.discoveryPort(7890)
.listeningPort(30303)
.build());

@Before
public void setup() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,15 @@ public class NetEnodeTest {
BytesValue.fromHexString(
"0x0f1b319e32017c3fcb221841f0f978701b4e9513fe6a567a2db43d43381a9c7e3dfe7cae13cbc2f56943400bacaf9082576ab087cd51983b17d729ae796f6807");

private final DefaultPeer defaultPeer = new DefaultPeer(nodeId, "1.2.3.4", 7890, 30303);
private final DefaultPeer defaultPeer =
DefaultPeer.fromEnodeURL(
EnodeURL.builder()
.nodeId(nodeId)
.ipAddress("1.2.3.4")
.discoveryPort(7890)
.listeningPort(30303)
.build());

private final Optional<EnodeURL> enodeURL = Optional.of(defaultPeer.getEnodeURL());

@Mock private P2PNetwork p2PNetwork;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -262,11 +262,7 @@ public Set<Capability> getAgreedCapabilities() {
@Override
public PeerInfo getPeerInfo() {
return new PeerInfo(
5,
"mock-network-client",
capabilities,
to.getEndpoint().getTcpPort().getAsInt(),
to.getId());
5, "mock-network-client", capabilities, to.getEnodeURL().getListeningPort(), to.getId());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import tech.pegasys.pantheon.ethereum.p2p.wire.RawMessage;
import tech.pegasys.pantheon.ethereum.p2p.wire.messages.DisconnectMessage.DisconnectReason;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import tech.pegasys.pantheon.util.enode.EnodeURL;

import java.util.Arrays;
import java.util.Optional;
Expand All @@ -39,8 +40,22 @@ public final class MockNetworkTest {
public void exchangeMessages() throws Exception {
final Capability cap = Capability.create("eth", 63);
final MockNetwork network = new MockNetwork(Arrays.asList(cap));
final Peer one = new DefaultPeer(randomId(), "192.168.1.2", 1234, 4321);
final Peer two = new DefaultPeer(randomId(), "192.168.1.3", 1234, 4321);
final Peer one =
DefaultPeer.fromEnodeURL(
EnodeURL.builder()
.nodeId(randomId())
.ipAddress("192.168.1.2")
.discoveryPort(1234)
.listeningPort(4321)
.build());
final Peer two =
DefaultPeer.fromEnodeURL(
EnodeURL.builder()
.nodeId(randomId())
.ipAddress("192.168.1.3")
.discoveryPort(1234)
.listeningPort(4321)
.build());
try (final P2PNetwork network1 = network.setup(one);
final P2PNetwork network2 = network.setup(two)) {
final CompletableFuture<Message> messageFuture = new CompletableFuture<>();
Expand Down Expand Up @@ -102,8 +117,6 @@ public void exchangeMessages() throws Exception {
}

private static BytesValue randomId() {
final byte[] raw = new byte[DefaultPeer.PEER_ID_SIZE];
ThreadLocalRandom.current().nextBytes(raw);
return BytesValue.wrap(raw);
return Peer.randomId();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,44 +13,54 @@
package tech.pegasys.pantheon.ethereum.p2p.discovery;

import tech.pegasys.pantheon.ethereum.p2p.peers.DefaultPeer;
import tech.pegasys.pantheon.ethereum.p2p.peers.Endpoint;
import tech.pegasys.pantheon.ethereum.p2p.peers.Peer;
import tech.pegasys.pantheon.ethereum.p2p.peers.PeerId;
import tech.pegasys.pantheon.ethereum.rlp.RLPInput;
import tech.pegasys.pantheon.ethereum.rlp.RLPOutput;
import tech.pegasys.pantheon.util.bytes.BytesValue;

import java.util.OptionalInt;
import tech.pegasys.pantheon.util.enode.EnodeURL;

/**
* Represents an Ethereum node that we interacting with through the discovery and wire protocols.
*/
public class DiscoveryPeer extends DefaultPeer {
private PeerDiscoveryStatus status = PeerDiscoveryStatus.KNOWN;
// Endpoint is a datastructure used in discovery messages
private final Endpoint endpoint;

// Timestamps.
private long firstDiscovered = 0;
private long lastContacted = 0;
private long lastSeen = 0;

public DiscoveryPeer(
final BytesValue id, final String host, final int udpPort, final int tcpPort) {
super(id, host, udpPort, tcpPort);
private DiscoveryPeer(final EnodeURL enode, final Endpoint endpoint) {
super(enode);
this.endpoint = endpoint;
}

public DiscoveryPeer(
final BytesValue id, final String host, final int udpPort, final OptionalInt tcpPort) {
super(id, host, udpPort, tcpPort);
public static DiscoveryPeer fromEnode(final EnodeURL enode) {
return new DiscoveryPeer(enode, Endpoint.fromEnode(enode));
}

public DiscoveryPeer(final BytesValue id, final String host, final int udpPort) {
super(id, host, udpPort);
public static DiscoveryPeer fromIdAndEndpoint(final BytesValue id, final Endpoint endpoint) {
return new DiscoveryPeer(endpoint.toEnode(id), endpoint);
}

public DiscoveryPeer(final BytesValue id, final Endpoint endpoint) {
super(id, endpoint);
public static DiscoveryPeer readFrom(final RLPInput in) {
final int size = in.enterList();

// The last list item will be the id, pass size - 1 to Endpoint
final Endpoint endpoint = Endpoint.decodeInline(in, size - 1);
final BytesValue id = in.readBytesValue();
in.leaveList();

return DiscoveryPeer.fromIdAndEndpoint(id, endpoint);
}

public DiscoveryPeer(final Peer peer) {
super(peer.getId(), peer.getEndpoint());
public void writeTo(final RLPOutput out) {
out.startList();
endpoint.encodeInline(out);
out.writeBytesValue(getId());
out.endList();
}

public PeerDiscoveryStatus getStatus() {
Expand Down Expand Up @@ -86,11 +96,15 @@ public void setLastSeen(final long lastSeen) {
this.lastSeen = lastSeen;
}

public Endpoint getEndpoint() {
return endpoint;
}

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("DiscoveryPeer{");
sb.append("status=").append(status);
sb.append(", endPoint=").append(this.getEndpoint());
sb.append(", enode=").append(this.getEnodeURL());
sb.append(", firstDiscovered=").append(firstDiscovered);
sb.append(", lastContacted=").append(lastContacted);
sb.append(", lastSeen=").append(lastSeen);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,23 +10,27 @@
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
*/
package tech.pegasys.pantheon.ethereum.p2p.peers;
package tech.pegasys.pantheon.ethereum.p2p.discovery;

import static com.google.common.base.Preconditions.checkArgument;
import static tech.pegasys.pantheon.util.Preconditions.checkGuard;

import tech.pegasys.pantheon.ethereum.p2p.discovery.PeerDiscoveryPacketDecodingException;
import tech.pegasys.pantheon.ethereum.rlp.RLPInput;
import tech.pegasys.pantheon.ethereum.rlp.RLPOutput;
import tech.pegasys.pantheon.util.NetworkUtility;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import tech.pegasys.pantheon.util.enode.EnodeURL;

import java.net.InetAddress;
import java.util.Objects;
import java.util.OptionalInt;

import com.google.common.net.InetAddresses;

/** Encapsulates the network coordinates of a {@link Peer}. */
/**
* Encapsulates the network coordinates of a {@link DiscoveryPeer} as well as serialization logic
* used in various Discovery messages.
*/
public class Endpoint {
private final String host;
private final int udpPort;
Expand All @@ -47,6 +51,23 @@ public Endpoint(final String host, final int udpPort, final OptionalInt tcpPort)
this.tcpPort = tcpPort;
}

public static Endpoint fromEnode(final EnodeURL enode) {
final OptionalInt tcpPort =
enode.getDiscoveryPort().isPresent()
? OptionalInt.of(enode.getListeningPort())
: OptionalInt.empty();
return new Endpoint(enode.getIp().getHostAddress(), enode.getEffectiveDiscoveryPort(), tcpPort);
}

public EnodeURL toEnode(final BytesValue nodeId) {
return EnodeURL.builder()
.nodeId(nodeId)
.ipAddress(host)
.listeningPort(tcpPort.orElse(udpPort))
.discoveryPort(udpPort)
.build();
}

public String getHost() {
return host;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@
import tech.pegasys.pantheon.ethereum.p2p.discovery.internal.PingPacketData;
import tech.pegasys.pantheon.ethereum.p2p.discovery.internal.TimerUtil;
import tech.pegasys.pantheon.ethereum.p2p.peers.DefaultPeerId;
import tech.pegasys.pantheon.ethereum.p2p.peers.Endpoint;
import tech.pegasys.pantheon.ethereum.p2p.peers.Peer;
import tech.pegasys.pantheon.ethereum.p2p.peers.PeerBlacklist;
import tech.pegasys.pantheon.ethereum.p2p.wire.messages.DisconnectMessage;
import tech.pegasys.pantheon.ethereum.permissioning.NodeLocalConfigPermissioningController;
Expand All @@ -40,6 +40,7 @@
import tech.pegasys.pantheon.util.NetworkUtility;
import tech.pegasys.pantheon.util.Subscribers;
import tech.pegasys.pantheon.util.bytes.BytesValue;
import tech.pegasys.pantheon.util.enode.EnodeURL;

import java.net.InetSocketAddress;
import java.net.SocketException;
Expand Down Expand Up @@ -110,7 +111,10 @@ public PeerDiscoveryAgent(
this.nodeWhitelistController = nodeWhitelistController;
this.nodePermissioningController = nodePermissioningController;
this.bootstrapPeers =
config.getBootstrapPeers().stream().map(DiscoveryPeer::new).collect(Collectors.toList());
config.getBootstrapPeers().stream()
.map(Peer::getEnodeURL)
.map(DiscoveryPeer::fromEnode)
.collect(Collectors.toList());

this.config = config;
this.keyPair = keyPair;
Expand Down Expand Up @@ -141,8 +145,13 @@ public CompletableFuture<?> start(final int tcpPort) {
(InetSocketAddress localAddress) -> {
// Once listener is set up, finish initializing
advertisedPeer =
new DiscoveryPeer(
id, config.getAdvertisedHost(), localAddress.getPort(), tcpPort);
DiscoveryPeer.fromEnode(
EnodeURL.builder()
.nodeId(id)
.ipAddress(config.getAdvertisedHost())
.listeningPort(tcpPort)
.discoveryPort(localAddress.getPort())
.build());
isActive = true;
startController();
});
Expand Down Expand Up @@ -197,7 +206,15 @@ protected void handleIncomingPacket(final Endpoint sourceEndpoint, final Packet
// Notify the peer controller.
String host = sourceEndpoint.getHost();
int port = sourceEndpoint.getUdpPort();
final DiscoveryPeer peer = new DiscoveryPeer(packet.getNodeId(), host, port, tcpPort);
final DiscoveryPeer peer =
DiscoveryPeer.fromEnode(
EnodeURL.builder()
.nodeId(packet.getNodeId())
.ipAddress(host)
.listeningPort(tcpPort.orElse(port))
.discoveryPort(port)
.build());

controller.ifPresent(c -> c.onMessage(packet, peer));
}

Expand Down
Loading

0 comments on commit b4dad69

Please sign in to comment.