feat: improve peer manager and re-integrate to light push (#2191)

* up lock

* make ConnectionManager use ctor

* reform connection manager configurations

* remove log param from peerManager

* make PeerManager use only ConnectionManager, move getPeers to ConnectionManager, remove not needed code

* remove allPeers and connectedPeers from BaseProtocolCore, update tests, add getPeers for IWaku

* use only one peerManager from Waku object

* remove IBaseProtocolSDK and merge with PeerManager

* re-implement peerManager, remove ProtocolUseOptions

* remove not needed test, up lock

* update deps and lock

* remove old test for peerManager, fix check and spell

* rename to getConnectedPeers

* feat: improve filter subscriptions (#2193)

* add message cache to Filter

* remove WakuOptions and use only ProtocolCreateOptions

* move subscribe options to createLightNode Fitler protocol options

* rename SubscriptionManager to Subscription

* rename to CreateNodeOptions

* add warning

* feat: introduce subscription manager (#2202)

* feat: inroduce subscription manager

* fix: make pipeline succeed (#2238)

* fix test

* use hardcoded value

* update playwright

* fix test:browser

* up lock

* make peer retrieval probabilistic

* add comments

* up lightpush tests

* add tests for peer_manager, improve folder structure

* create named files for protocols

* create named files, simplify project structure

* remove only
This commit is contained in:
Sasha 2025-01-31 00:16:00 +01:00 committed by GitHub
parent d706fe146c
commit 62f93dc842
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
70 changed files with 6156 additions and 6851 deletions

View File

@ -116,6 +116,7 @@
"upgrader",
"vacp",
"varint",
"weboko",
"waku",
"wakuconnect",
"wakunode",

View File

@ -57,7 +57,7 @@ jobs:
browser:
runs-on: ubuntu-latest
container:
image: mcr.microsoft.com/playwright:v1.48.0-jammy
image: mcr.microsoft.com/playwright:v1.50.0-jammy
env:
HOME: "/root"
steps:

View File

@ -20,7 +20,7 @@ jobs:
timeout-minutes: 60
runs-on: ubuntu-latest
container:
image: mcr.microsoft.com/playwright:v1.48.0-jammy
image: mcr.microsoft.com/playwright:v1.50.0-jammy
steps:
- uses: actions/checkout@v3
- uses: actions/setup-node@v3

7605
package-lock.json generated

File diff suppressed because it is too large Load Diff

View File

@ -11,7 +11,7 @@
"test": "npx playwright test"
},
"devDependencies": {
"@playwright/test": "^1.48.1",
"@playwright/test": "^1.50.0",
"@waku/create-app": "^0.1.1-504bcd4",
"dotenv-flow": "^4.1.0",
"serve": "^14.2.3"

View File

@ -82,6 +82,7 @@
},
"devDependencies": {
"@libp2p/peer-id": "^5.0.1",
"@libp2p/interface": "^2.1.3",
"@multiformats/multiaddr": "^12.0.0",
"@rollup/plugin-commonjs": "^25.0.7",
"@rollup/plugin-json": "^6.0.0",

View File

@ -15,11 +15,10 @@ export { LightPushCodec, LightPushCore } from "./lib/light_push/index.js";
export * as waku_store from "./lib/store/index.js";
export { StoreCore, StoreCodec } from "./lib/store/index.js";
export { ConnectionManager } from "./lib/connection_manager.js";
export { ConnectionManager } from "./lib/connection_manager/index.js";
export { getHealthManager } from "./lib/health_manager.js";
export { KeepAliveManager } from "./lib/keep_alive_manager.js";
export { StreamManager } from "./lib/stream_manager/index.js";
export { MetadataCodec, wakuMetadata } from "./lib/metadata/index.js";

View File

@ -5,10 +5,7 @@ import type {
Libp2pComponents,
PubsubTopic
} from "@waku/interfaces";
import { Logger } from "@waku/utils";
import { getPeersForProtocol, sortPeersByLatency } from "@waku/utils/libp2p";
import { filterPeersByDiscovery } from "./filterPeers.js";
import { StreamManager } from "./stream_manager/index.js";
/**
@ -23,7 +20,6 @@ export class BaseProtocol implements IBaseProtocolCore {
protected constructor(
public multicodec: string,
protected components: Libp2pComponents,
private log: Logger,
public readonly pubsubTopics: PubsubTopic[]
) {
this.addLibp2pEventListener = components.events.addEventListener.bind(
@ -45,73 +41,4 @@ export class BaseProtocol implements IBaseProtocolCore {
protected async getStream(peer: Peer): Promise<Stream> {
return this.streamManager.getStream(peer);
}
/**
* Returns known peers from the address book (`libp2p.peerStore`) that support
* the class protocol. Waku may or may not be currently connected to these
* peers.
*/
public async allPeers(): Promise<Peer[]> {
return getPeersForProtocol(this.components.peerStore, [this.multicodec]);
}
public async connectedPeers(): Promise<Peer[]> {
const peers = await this.allPeers();
return peers.filter((peer) => {
const connections = this.components.connectionManager.getConnections(
peer.id
);
return connections.length > 0;
});
}
/**
* Retrieves a list of connected peers that support the protocol. The list is sorted by latency.
*
* @param numPeers - The total number of peers to retrieve. If 0, all peers are returned.
* @param maxBootstrapPeers - The maximum number of bootstrap peers to retrieve.
* @returns A list of peers that support the protocol sorted by latency. By default, returns all peers available, including bootstrap.
*/
public async getPeers(
{
numPeers,
maxBootstrapPeers
}: {
numPeers: number;
maxBootstrapPeers: number;
} = {
maxBootstrapPeers: 0,
numPeers: 0
}
): Promise<Peer[]> {
// Retrieve all connected peers that support the protocol & shard (if configured)
const allAvailableConnectedPeers = await this.connectedPeers();
// Filter the peers based on discovery & number of peers requested
const filteredPeers = filterPeersByDiscovery(
allAvailableConnectedPeers,
numPeers,
maxBootstrapPeers
);
// Sort the peers by latency
const sortedFilteredPeers = await sortPeersByLatency(
this.components.peerStore,
filteredPeers
);
if (sortedFilteredPeers.length === 0) {
this.log.warn(
"No peers found. Ensure you have a connection to the network."
);
}
if (sortedFilteredPeers.length < numPeers) {
this.log.warn(
`Only ${sortedFilteredPeers.length} peers found. Requested ${numPeers}.`
);
}
return sortedFilteredPeers;
}
}

View File

@ -19,7 +19,6 @@ import {
IConnectionStateEvents,
IPeersByDiscoveryEvents,
IRelay,
KeepAliveOptions,
PeersByDiscoveryResult,
PubsubTopic,
ShardInfo
@ -29,18 +28,31 @@ import { decodeRelayShard, shardInfoToPubsubTopics } from "@waku/utils";
import { Logger } from "@waku/utils";
import { KeepAliveManager } from "./keep_alive_manager.js";
import { getPeerPing } from "./utils.js";
const log = new Logger("connection-manager");
export const DEFAULT_MAX_BOOTSTRAP_PEERS_ALLOWED = 1;
export const DEFAULT_MAX_DIAL_ATTEMPTS_FOR_PEER = 3;
export const DEFAULT_MAX_PARALLEL_DIALS = 3;
const DEFAULT_MAX_BOOTSTRAP_PEERS_ALLOWED = 1;
const DEFAULT_MAX_DIAL_ATTEMPTS_FOR_PEER = 3;
const DEFAULT_MAX_PARALLEL_DIALS = 3;
const DEFAULT_PING_KEEP_ALIVE_SEC = 5 * 60;
const DEFAULT_RELAY_KEEP_ALIVE_SEC = 5 * 60;
type ConnectionManagerConstructorOptions = {
libp2p: Libp2p;
pubsubTopics: PubsubTopic[];
relay?: IRelay;
config?: Partial<ConnectionManagerOptions>;
};
export class ConnectionManager
extends TypedEventEmitter<IPeersByDiscoveryEvents & IConnectionStateEvents>
implements IConnectionManager
{
private static instances = new Map<string, ConnectionManager>();
// TODO(weboko): make it private
public readonly pubsubTopics: PubsubTopic[];
private keepAliveManager: KeepAliveManager;
private options: ConnectionManagerOptions;
private libp2p: Libp2p;
@ -60,29 +72,6 @@ export class ConnectionManager
return this.isP2PNetworkConnected;
}
public static create(
peerId: string,
libp2p: Libp2p,
keepAliveOptions: KeepAliveOptions,
pubsubTopics: PubsubTopic[],
relay?: IRelay,
options?: ConnectionManagerOptions
): ConnectionManager {
let instance = ConnectionManager.instances.get(peerId);
if (!instance) {
instance = new ConnectionManager(
libp2p,
keepAliveOptions,
pubsubTopics,
relay,
options
);
ConnectionManager.instances.set(peerId, instance);
}
return instance;
}
public stop(): void {
this.keepAliveManager.stopAll();
this.libp2p.removeEventListener(
@ -165,27 +154,26 @@ export class ConnectionManager
};
}
private constructor(
libp2p: Libp2p,
keepAliveOptions: KeepAliveOptions,
public readonly configuredPubsubTopics: PubsubTopic[],
relay?: IRelay,
options?: Partial<ConnectionManagerOptions>
) {
public constructor(options: ConnectionManagerConstructorOptions) {
super();
this.libp2p = libp2p;
this.configuredPubsubTopics = configuredPubsubTopics;
this.libp2p = options.libp2p;
this.pubsubTopics = options.pubsubTopics;
this.options = {
maxDialAttemptsForPeer: DEFAULT_MAX_DIAL_ATTEMPTS_FOR_PEER,
maxBootstrapPeersAllowed: DEFAULT_MAX_BOOTSTRAP_PEERS_ALLOWED,
maxParallelDials: DEFAULT_MAX_PARALLEL_DIALS,
...options
pingKeepAlive: DEFAULT_PING_KEEP_ALIVE_SEC,
relayKeepAlive: DEFAULT_RELAY_KEEP_ALIVE_SEC,
...options.config
};
this.keepAliveManager = new KeepAliveManager({
relay,
libp2p,
options: keepAliveOptions
relay: options.relay,
libp2p: options.libp2p,
options: {
pingKeepAlive: this.options.pingKeepAlive,
relayKeepAlive: this.options.relayKeepAlive
}
});
this.startEventListeners()
@ -202,6 +190,29 @@ export class ConnectionManager
);
}
public async getConnectedPeers(codec?: string): Promise<Peer[]> {
const peerIDs = this.libp2p.getPeers();
if (peerIDs.length === 0) {
return [];
}
const peers = await Promise.all(
peerIDs.map(async (id) => {
try {
return await this.libp2p.peerStore.get(id);
} catch (e) {
return null;
}
})
);
return peers
.filter((p) => !!p)
.filter((p) => (codec ? (p as Peer).protocols.includes(codec) : true))
.sort((left, right) => getPeerPing(left) - getPeerPing(right)) as Peer[];
}
private async dialPeerStorePeers(): Promise<void> {
const peerInfos = await this.libp2p.peerStore.all();
const dialPromises = [];
@ -572,7 +583,7 @@ export class ConnectionManager
log.warn(
`Discovered peer ${peerId.toString()} with ShardInfo ${shardInfo} is not part of any of the configured pubsub topics (${
this.configuredPubsubTopics
this.pubsubTopics
}).
Not dialing.`
);
@ -667,7 +678,7 @@ export class ConnectionManager
const pubsubTopics = shardInfoToPubsubTopics(shardInfo);
const isTopicConfigured = pubsubTopics.some((topic) =>
this.configuredPubsubTopics.includes(topic)
this.pubsubTopics.includes(topic)
);
return isTopicConfigured;
}

View File

@ -0,0 +1 @@
export { ConnectionManager } from "./connection_manager.js";

View File

@ -1,14 +1,18 @@
import type { PeerId } from "@libp2p/interface";
import type { IRelay, Libp2p, PeerIdStr } from "@waku/interfaces";
import type { KeepAliveOptions } from "@waku/interfaces";
import { Logger, pubsubTopicToSingleShardInfo } from "@waku/utils";
import { utf8ToBytes } from "@waku/utils/bytes";
import { createEncoder } from "./message/version_0.js";
import { createEncoder } from "../message/version_0.js";
export const RelayPingContentTopic = "/relay-ping/1/ping/null";
const RelayPingContentTopic = "/relay-ping/1/ping/null";
const log = new Logger("keep-alive");
type KeepAliveOptions = {
pingKeepAlive: number;
relayKeepAlive: number;
};
type CreateKeepAliveManagerOptions = {
options: KeepAliveOptions;
libp2p: Libp2p;

View File

@ -0,0 +1,25 @@
import type { Peer } from "@libp2p/interface";
import { bytesToUtf8 } from "@waku/utils/bytes";
/**
* Reads peer's metadata and retrieves ping value.
* @param peer Peer or null
* @returns -1 if no ping attached, otherwise returns ping value
*/
export const getPeerPing = (peer: Peer | null): number => {
if (!peer) {
return -1;
}
try {
const bytes = peer.metadata.get("ping");
if (!bytes) {
return -1;
}
return Number(bytesToUtf8(bytes));
} catch (e) {
return -1;
}
};

View File

@ -0,0 +1,315 @@
import type { Peer, Stream } from "@libp2p/interface";
import type { IncomingStreamData } from "@libp2p/interface-internal";
import {
type ContentTopic,
type CoreProtocolResult,
type IBaseProtocolCore,
type Libp2p,
ProtocolError,
type PubsubTopic
} from "@waku/interfaces";
import { WakuMessage } from "@waku/proto";
import { Logger } from "@waku/utils";
import all from "it-all";
import * as lp from "it-length-prefixed";
import { pipe } from "it-pipe";
import { Uint8ArrayList } from "uint8arraylist";
import { BaseProtocol } from "../base_protocol.js";
import {
FilterPushRpc,
FilterSubscribeResponse,
FilterSubscribeRpc
} from "./filter_rpc.js";
const log = new Logger("filter:v2");
export const FilterCodecs = {
SUBSCRIBE: "/vac/waku/filter-subscribe/2.0.0-beta1",
PUSH: "/vac/waku/filter-push/2.0.0-beta1"
};
export class FilterCore extends BaseProtocol implements IBaseProtocolCore {
public constructor(
private handleIncomingMessage: (
pubsubTopic: PubsubTopic,
wakuMessage: WakuMessage,
peerIdStr: string
) => Promise<void>,
public readonly pubsubTopics: PubsubTopic[],
libp2p: Libp2p
) {
super(FilterCodecs.SUBSCRIBE, libp2p.components, pubsubTopics);
libp2p
.handle(FilterCodecs.PUSH, this.onRequest.bind(this), {
maxInboundStreams: 100
})
.catch((e) => {
log.error("Failed to register ", FilterCodecs.PUSH, e);
});
}
public async subscribe(
pubsubTopic: PubsubTopic,
peer: Peer,
contentTopics: ContentTopic[]
): Promise<CoreProtocolResult> {
const stream = await this.getStream(peer);
const request = FilterSubscribeRpc.createSubscribeRequest(
pubsubTopic,
contentTopics
);
let res: Uint8ArrayList[] | undefined;
try {
res = await pipe(
[request.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
} catch (error) {
log.error("Failed to send subscribe request", error);
return {
success: null,
failure: {
error: ProtocolError.GENERIC_FAIL,
peerId: peer.id
}
};
}
const { statusCode, requestId, statusDesc } =
FilterSubscribeResponse.decode(res[0].slice());
if (statusCode < 200 || statusCode >= 300) {
log.error(
`Filter subscribe request ${requestId} failed with status code ${statusCode}: ${statusDesc}`
);
return {
failure: {
error: ProtocolError.REMOTE_PEER_REJECTED,
peerId: peer.id
},
success: null
};
}
return {
failure: null,
success: peer.id
};
}
public async unsubscribe(
pubsubTopic: PubsubTopic,
peer: Peer,
contentTopics: ContentTopic[]
): Promise<CoreProtocolResult> {
let stream: Stream | undefined;
try {
stream = await this.getStream(peer);
} catch (error) {
log.error(
`Failed to get a stream for remote peer${peer.id.toString()}`,
error
);
return {
success: null,
failure: {
error: ProtocolError.NO_STREAM_AVAILABLE,
peerId: peer.id
}
};
}
const unsubscribeRequest = FilterSubscribeRpc.createUnsubscribeRequest(
pubsubTopic,
contentTopics
);
try {
await pipe([unsubscribeRequest.encode()], lp.encode, stream.sink);
} catch (error) {
log.error("Failed to send unsubscribe request", error);
return {
success: null,
failure: {
error: ProtocolError.GENERIC_FAIL,
peerId: peer.id
}
};
}
return {
success: peer.id,
failure: null
};
}
public async unsubscribeAll(
pubsubTopic: PubsubTopic,
peer: Peer
): Promise<CoreProtocolResult> {
const stream = await this.getStream(peer);
const request = FilterSubscribeRpc.createUnsubscribeAllRequest(pubsubTopic);
const res = await pipe(
[request.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
if (!res || !res.length) {
return {
failure: {
error: ProtocolError.NO_RESPONSE,
peerId: peer.id
},
success: null
};
}
const { statusCode, requestId, statusDesc } =
FilterSubscribeResponse.decode(res[0].slice());
if (statusCode < 200 || statusCode >= 300) {
log.error(
`Filter unsubscribe all request ${requestId} failed with status code ${statusCode}: ${statusDesc}`
);
return {
failure: {
error: ProtocolError.REMOTE_PEER_REJECTED,
peerId: peer.id
},
success: null
};
}
return {
failure: null,
success: peer.id
};
}
public async ping(peer: Peer): Promise<CoreProtocolResult> {
let stream: Stream | undefined;
try {
stream = await this.getStream(peer);
} catch (error) {
log.error(
`Failed to get a stream for remote peer${peer.id.toString()}`,
error
);
return {
success: null,
failure: {
error: ProtocolError.NO_STREAM_AVAILABLE,
peerId: peer.id
}
};
}
const request = FilterSubscribeRpc.createSubscriberPingRequest();
let res: Uint8ArrayList[] | undefined;
try {
res = await pipe(
[request.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
} catch (error) {
log.error("Failed to send ping request", error);
return {
success: null,
failure: {
error: ProtocolError.GENERIC_FAIL,
peerId: peer.id
}
};
}
if (!res || !res.length) {
return {
success: null,
failure: {
error: ProtocolError.NO_RESPONSE,
peerId: peer.id
}
};
}
const { statusCode, requestId, statusDesc } =
FilterSubscribeResponse.decode(res[0].slice());
if (statusCode < 200 || statusCode >= 300) {
log.error(
`Filter ping request ${requestId} failed with status code ${statusCode}: ${statusDesc}`
);
return {
success: null,
failure: {
error: ProtocolError.REMOTE_PEER_REJECTED,
peerId: peer.id
}
};
}
return {
success: peer.id,
failure: null
};
}
private onRequest(streamData: IncomingStreamData): void {
const { connection, stream } = streamData;
const { remotePeer } = connection;
log.info(`Received message from ${remotePeer.toString()}`);
try {
pipe(stream, lp.decode, async (source) => {
for await (const bytes of source) {
const response = FilterPushRpc.decode(bytes.slice());
const { pubsubTopic, wakuMessage } = response;
if (!wakuMessage) {
log.error("Received empty message");
return;
}
if (!pubsubTopic) {
log.error("Pubsub topic missing from push message");
return;
}
await this.handleIncomingMessage(
pubsubTopic,
wakuMessage,
connection.remotePeer.toString()
);
}
}).then(
() => {
log.info("Receiving pipe closed.");
},
async (e) => {
log.error(
`Error with receiving pipe on peer:${connection.remotePeer.toString()} -- stream:${stream.id} -- protocol:${stream.protocol}: `,
e
);
}
);
} catch (e) {
log.error("Error decoding message", e);
}
}
}

View File

@ -1,315 +1 @@
import type { Peer, Stream } from "@libp2p/interface";
import type { IncomingStreamData } from "@libp2p/interface-internal";
import {
type ContentTopic,
type CoreProtocolResult,
type IBaseProtocolCore,
type Libp2p,
ProtocolError,
type PubsubTopic
} from "@waku/interfaces";
import { WakuMessage } from "@waku/proto";
import { Logger } from "@waku/utils";
import all from "it-all";
import * as lp from "it-length-prefixed";
import { pipe } from "it-pipe";
import { Uint8ArrayList } from "uint8arraylist";
import { BaseProtocol } from "../base_protocol.js";
import {
FilterPushRpc,
FilterSubscribeResponse,
FilterSubscribeRpc
} from "./filter_rpc.js";
const log = new Logger("filter:v2");
export const FilterCodecs = {
SUBSCRIBE: "/vac/waku/filter-subscribe/2.0.0-beta1",
PUSH: "/vac/waku/filter-push/2.0.0-beta1"
};
export class FilterCore extends BaseProtocol implements IBaseProtocolCore {
public constructor(
private handleIncomingMessage: (
pubsubTopic: PubsubTopic,
wakuMessage: WakuMessage,
peerIdStr: string
) => Promise<void>,
public readonly pubsubTopics: PubsubTopic[],
libp2p: Libp2p
) {
super(FilterCodecs.SUBSCRIBE, libp2p.components, log, pubsubTopics);
libp2p
.handle(FilterCodecs.PUSH, this.onRequest.bind(this), {
maxInboundStreams: 100
})
.catch((e) => {
log.error("Failed to register ", FilterCodecs.PUSH, e);
});
}
public async subscribe(
pubsubTopic: PubsubTopic,
peer: Peer,
contentTopics: ContentTopic[]
): Promise<CoreProtocolResult> {
const stream = await this.getStream(peer);
const request = FilterSubscribeRpc.createSubscribeRequest(
pubsubTopic,
contentTopics
);
let res: Uint8ArrayList[] | undefined;
try {
res = await pipe(
[request.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
} catch (error) {
log.error("Failed to send subscribe request", error);
return {
success: null,
failure: {
error: ProtocolError.GENERIC_FAIL,
peerId: peer.id
}
};
}
const { statusCode, requestId, statusDesc } =
FilterSubscribeResponse.decode(res[0].slice());
if (statusCode < 200 || statusCode >= 300) {
log.error(
`Filter subscribe request ${requestId} failed with status code ${statusCode}: ${statusDesc}`
);
return {
failure: {
error: ProtocolError.REMOTE_PEER_REJECTED,
peerId: peer.id
},
success: null
};
}
return {
failure: null,
success: peer.id
};
}
public async unsubscribe(
pubsubTopic: PubsubTopic,
peer: Peer,
contentTopics: ContentTopic[]
): Promise<CoreProtocolResult> {
let stream: Stream | undefined;
try {
stream = await this.getStream(peer);
} catch (error) {
log.error(
`Failed to get a stream for remote peer${peer.id.toString()}`,
error
);
return {
success: null,
failure: {
error: ProtocolError.NO_STREAM_AVAILABLE,
peerId: peer.id
}
};
}
const unsubscribeRequest = FilterSubscribeRpc.createUnsubscribeRequest(
pubsubTopic,
contentTopics
);
try {
await pipe([unsubscribeRequest.encode()], lp.encode, stream.sink);
} catch (error) {
log.error("Failed to send unsubscribe request", error);
return {
success: null,
failure: {
error: ProtocolError.GENERIC_FAIL,
peerId: peer.id
}
};
}
return {
success: peer.id,
failure: null
};
}
public async unsubscribeAll(
pubsubTopic: PubsubTopic,
peer: Peer
): Promise<CoreProtocolResult> {
const stream = await this.getStream(peer);
const request = FilterSubscribeRpc.createUnsubscribeAllRequest(pubsubTopic);
const res = await pipe(
[request.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
if (!res || !res.length) {
return {
failure: {
error: ProtocolError.NO_RESPONSE,
peerId: peer.id
},
success: null
};
}
const { statusCode, requestId, statusDesc } =
FilterSubscribeResponse.decode(res[0].slice());
if (statusCode < 200 || statusCode >= 300) {
log.error(
`Filter unsubscribe all request ${requestId} failed with status code ${statusCode}: ${statusDesc}`
);
return {
failure: {
error: ProtocolError.REMOTE_PEER_REJECTED,
peerId: peer.id
},
success: null
};
}
return {
failure: null,
success: peer.id
};
}
public async ping(peer: Peer): Promise<CoreProtocolResult> {
let stream: Stream | undefined;
try {
stream = await this.getStream(peer);
} catch (error) {
log.error(
`Failed to get a stream for remote peer${peer.id.toString()}`,
error
);
return {
success: null,
failure: {
error: ProtocolError.NO_STREAM_AVAILABLE,
peerId: peer.id
}
};
}
const request = FilterSubscribeRpc.createSubscriberPingRequest();
let res: Uint8ArrayList[] | undefined;
try {
res = await pipe(
[request.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
} catch (error) {
log.error("Failed to send ping request", error);
return {
success: null,
failure: {
error: ProtocolError.GENERIC_FAIL,
peerId: peer.id
}
};
}
if (!res || !res.length) {
return {
success: null,
failure: {
error: ProtocolError.NO_RESPONSE,
peerId: peer.id
}
};
}
const { statusCode, requestId, statusDesc } =
FilterSubscribeResponse.decode(res[0].slice());
if (statusCode < 200 || statusCode >= 300) {
log.error(
`Filter ping request ${requestId} failed with status code ${statusCode}: ${statusDesc}`
);
return {
success: null,
failure: {
error: ProtocolError.REMOTE_PEER_REJECTED,
peerId: peer.id
}
};
}
return {
success: peer.id,
failure: null
};
}
private onRequest(streamData: IncomingStreamData): void {
const { connection, stream } = streamData;
const { remotePeer } = connection;
log.info(`Received message from ${remotePeer.toString()}`);
try {
pipe(stream, lp.decode, async (source) => {
for await (const bytes of source) {
const response = FilterPushRpc.decode(bytes.slice());
const { pubsubTopic, wakuMessage } = response;
if (!wakuMessage) {
log.error("Received empty message");
return;
}
if (!pubsubTopic) {
log.error("Pubsub topic missing from push message");
return;
}
await this.handleIncomingMessage(
pubsubTopic,
wakuMessage,
connection.remotePeer.toString()
);
}
}).then(
() => {
log.info("Receiving pipe closed.");
},
async (e) => {
log.error(
`Error with receiving pipe on peer:${connection.remotePeer.toString()} -- stream:${stream.id} -- protocol:${stream.protocol}: `,
e
);
}
);
} catch (e) {
log.error("Error decoding message", e);
}
}
}
export { FilterCodecs, FilterCore } from "./filter.js";

View File

@ -1,153 +0,0 @@
import { generateKeyPair } from "@libp2p/crypto/keys";
import { Peer } from "@libp2p/interface";
import type { Tag } from "@libp2p/interface";
import { peerIdFromPrivateKey } from "@libp2p/peer-id";
import { Tags } from "@waku/interfaces";
import { expect } from "chai";
import { filterPeersByDiscovery } from "./filterPeers.js";
describe("filterPeersByDiscovery function", function () {
it("should return all peers when numPeers is 0", async function () {
const [peer1, peer2, peer3] = await Promise.all([
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey)
]);
const mockPeers = [
{
id: peer1,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer2,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer3,
tags: new Map<string, Tag>([[Tags.PEER_EXCHANGE, { value: 100 }]])
}
] as unknown as Peer[];
const result = filterPeersByDiscovery(mockPeers, 0, 10);
expect(result.length).to.deep.equal(mockPeers.length);
});
it("should return all non-bootstrap peers and no bootstrap peer when numPeers is 0 and maxBootstrapPeers is 0", async function () {
const [peer1, peer2, peer3, peer4] = await Promise.all([
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey)
]);
const mockPeers = [
{
id: peer1,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer2,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer3,
tags: new Map<string, Tag>([[Tags.PEER_EXCHANGE, { value: 100 }]])
},
{
id: peer4,
tags: new Map<string, Tag>([[Tags.PEER_EXCHANGE, { value: 100 }]])
}
] as unknown as Peer[];
const result = filterPeersByDiscovery(mockPeers, 0, 0);
// result should have no bootstrap peers, and a total of 2 peers
expect(result.length).to.equal(2);
expect(
result.filter((peer: Peer) => peer.tags.has(Tags.BOOTSTRAP)).length
).to.equal(0);
});
it("should return one bootstrap peer, and all non-boostrap peers, when numPeers is 0 & maxBootstrap is 1", async function () {
const [peer1, peer2, peer3, peer4, peer5] = await Promise.all([
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey)
]);
const mockPeers = [
{
id: peer1,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer2,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer3,
tags: new Map<string, Tag>([[Tags.PEER_EXCHANGE, { value: 100 }]])
},
{
id: peer4,
tags: new Map<string, Tag>([[Tags.PEER_EXCHANGE, { value: 100 }]])
},
{
id: peer5,
tags: new Map<string, Tag>([[Tags.PEER_EXCHANGE, { value: 100 }]])
}
] as unknown as Peer[];
const result = filterPeersByDiscovery(mockPeers, 0, 1);
// result should have 1 bootstrap peers, and a total of 4 peers
expect(result.length).to.equal(4);
expect(
result.filter((peer: Peer) => peer.tags.has(Tags.BOOTSTRAP)).length
).to.equal(1);
});
it("should return only bootstrap peers up to maxBootstrapPeers", async function () {
const [peer1, peer2, peer3, peer4, peer5] = await Promise.all([
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey)
]);
const mockPeers = [
{
id: peer1,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer2,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer3,
tags: new Map<string, Tag>([[Tags.BOOTSTRAP, { value: 100 }]])
},
{
id: peer4,
tags: new Map<string, Tag>([[Tags.PEER_EXCHANGE, { value: 100 }]])
},
{
id: peer5,
tags: new Map<string, Tag>([[Tags.PEER_EXCHANGE, { value: 100 }]])
}
] as unknown as Peer[];
const result = filterPeersByDiscovery(mockPeers, 5, 2);
// check that result has at least 2 bootstrap peers and no more than 5 peers
expect(result.length).to.be.at.least(2);
expect(result.length).to.be.at.most(5);
expect(result.filter((peer: Peer) => peer.tags.has(Tags.BOOTSTRAP)).length);
});
});

View File

@ -1,51 +0,0 @@
import { Peer } from "@libp2p/interface";
import { Tags } from "@waku/interfaces";
/**
* Retrieves a list of peers based on the specified criteria:
* 1. If numPeers is 0, return all peers
* 2. Bootstrap peers are prioritized
* 3. Non-bootstrap peers are randomly selected to fill up to numPeers
*
* @param peers - The list of peers to filter from.
* @param numPeers - The total number of peers to retrieve. If 0, all peers are returned, irrespective of `maxBootstrapPeers`.
* @param maxBootstrapPeers - The maximum number of bootstrap peers to retrieve.
* @returns An array of peers based on the specified criteria.
*/
export function filterPeersByDiscovery(
peers: Peer[],
numPeers: number,
maxBootstrapPeers: number
): Peer[] {
// Collect the bootstrap peers up to the specified maximum
let bootstrapPeers = peers
.filter((peer) => peer.tags.has(Tags.BOOTSTRAP))
.slice(0, maxBootstrapPeers);
// If numPeers is less than the number of bootstrap peers, adjust the bootstrapPeers array
if (numPeers > 0 && numPeers < bootstrapPeers.length) {
bootstrapPeers = bootstrapPeers.slice(0, numPeers);
}
// Collect non-bootstrap peers
const nonBootstrapPeers = peers.filter(
(peer) => !peer.tags.has(Tags.BOOTSTRAP)
);
// If numPeers is 0, return all peers
if (numPeers === 0) {
return [...bootstrapPeers, ...nonBootstrapPeers];
}
// Initialize the list of selected peers with the bootstrap peers
const selectedPeers: Peer[] = [...bootstrapPeers];
// Fill up to numPeers with remaining random peers if needed
while (selectedPeers.length < numPeers && nonBootstrapPeers.length > 0) {
const randomIndex = Math.floor(Math.random() * nonBootstrapPeers.length);
const randomPeer = nonBootstrapPeers.splice(randomIndex, 1)[0];
selectedPeers.push(randomPeer);
}
return selectedPeers;
}

View File

@ -1,188 +1 @@
import type { Peer, Stream } from "@libp2p/interface";
import {
type CoreProtocolResult,
type IBaseProtocolCore,
type IEncoder,
type IMessage,
type Libp2p,
ProtocolError,
PubsubTopic,
type ThisOrThat
} from "@waku/interfaces";
import { PushResponse } from "@waku/proto";
import { isMessageSizeUnderCap } from "@waku/utils";
import { Logger } from "@waku/utils";
import all from "it-all";
import * as lp from "it-length-prefixed";
import { pipe } from "it-pipe";
import { Uint8ArrayList } from "uint8arraylist";
import { BaseProtocol } from "../base_protocol.js";
import { PushRpc } from "./push_rpc.js";
import { isRLNResponseError, matchRLNErrorMessage } from "./utils.js";
const log = new Logger("light-push");
export const LightPushCodec = "/vac/waku/lightpush/2.0.0-beta1";
export { PushResponse };
type PreparePushMessageResult = ThisOrThat<"query", PushRpc>;
/**
* Implements the [Waku v2 Light Push protocol](https://rfc.vac.dev/spec/19/).
*/
export class LightPushCore extends BaseProtocol implements IBaseProtocolCore {
public constructor(
public readonly pubsubTopics: PubsubTopic[],
libp2p: Libp2p
) {
super(LightPushCodec, libp2p.components, log, pubsubTopics);
}
private async preparePushMessage(
encoder: IEncoder,
message: IMessage
): Promise<PreparePushMessageResult> {
try {
if (!message.payload || message.payload.length === 0) {
log.error("Failed to send waku light push: payload is empty");
return { query: null, error: ProtocolError.EMPTY_PAYLOAD };
}
if (!(await isMessageSizeUnderCap(encoder, message))) {
log.error("Failed to send waku light push: message is bigger than 1MB");
return { query: null, error: ProtocolError.SIZE_TOO_BIG };
}
const protoMessage = await encoder.toProtoObj(message);
if (!protoMessage) {
log.error("Failed to encode to protoMessage, aborting push");
return {
query: null,
error: ProtocolError.ENCODE_FAILED
};
}
const query = PushRpc.createRequest(protoMessage, encoder.pubsubTopic);
return { query, error: null };
} catch (error) {
log.error("Failed to prepare push message", error);
return {
query: null,
error: ProtocolError.GENERIC_FAIL
};
}
}
public async send(
encoder: IEncoder,
message: IMessage,
peer: Peer
): Promise<CoreProtocolResult> {
const { query, error: preparationError } = await this.preparePushMessage(
encoder,
message
);
if (preparationError || !query) {
return {
success: null,
failure: {
error: preparationError,
peerId: peer.id
}
};
}
let stream: Stream;
try {
stream = await this.getStream(peer);
} catch (error) {
log.error("Failed to get stream", error);
return {
success: null,
failure: {
error: ProtocolError.NO_STREAM_AVAILABLE,
peerId: peer.id
}
};
}
let res: Uint8ArrayList[] | undefined;
try {
res = await pipe(
[query.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
} catch (err) {
log.error("Failed to send waku light push request", err);
return {
success: null,
failure: {
error: ProtocolError.GENERIC_FAIL,
peerId: peer.id
}
};
}
const bytes = new Uint8ArrayList();
res.forEach((chunk) => {
bytes.append(chunk);
});
let response: PushResponse | undefined;
try {
response = PushRpc.decode(bytes).response;
} catch (err) {
log.error("Failed to decode push reply", err);
return {
success: null,
failure: {
error: ProtocolError.DECODE_FAILED,
peerId: peer.id
}
};
}
if (!response) {
log.error("Remote peer fault: No response in PushRPC");
return {
success: null,
failure: {
error: ProtocolError.NO_RESPONSE,
peerId: peer.id
}
};
}
if (isRLNResponseError(response.info)) {
const rlnErrorCase = matchRLNErrorMessage(response.info!);
log.error("Remote peer rejected the message: ", rlnErrorCase);
return {
success: null,
failure: {
error: rlnErrorCase,
peerId: peer.id
}
};
}
if (!response.isSuccess) {
log.error("Remote peer rejected the message: ", response.info);
return {
success: null,
failure: {
error: ProtocolError.REMOTE_PEER_REJECTED,
peerId: peer.id
}
};
}
return { success: peer.id, failure: null };
}
}
export { LightPushCore, LightPushCodec, PushResponse } from "./light_push.js";

View File

@ -0,0 +1,189 @@
import type { Peer, Stream } from "@libp2p/interface";
import {
type CoreProtocolResult,
type IBaseProtocolCore,
type IEncoder,
type IMessage,
type Libp2p,
ProtocolError,
PubsubTopic,
type ThisOrThat
} from "@waku/interfaces";
import { PushResponse } from "@waku/proto";
import { isMessageSizeUnderCap } from "@waku/utils";
import { Logger } from "@waku/utils";
import all from "it-all";
import * as lp from "it-length-prefixed";
import { pipe } from "it-pipe";
import { Uint8ArrayList } from "uint8arraylist";
import { BaseProtocol } from "../base_protocol.js";
import { PushRpc } from "./push_rpc.js";
import { isRLNResponseError, matchRLNErrorMessage } from "./utils.js";
const log = new Logger("light-push");
export const LightPushCodec = "/vac/waku/lightpush/2.0.0-beta1";
export { PushResponse };
type PreparePushMessageResult = ThisOrThat<"query", PushRpc>;
/**
* Implements the [Waku v2 Light Push protocol](https://rfc.vac.dev/spec/19/).
*/
export class LightPushCore extends BaseProtocol implements IBaseProtocolCore {
public constructor(
public readonly pubsubTopics: PubsubTopic[],
libp2p: Libp2p
) {
super(LightPushCodec, libp2p.components, pubsubTopics);
}
private async preparePushMessage(
encoder: IEncoder,
message: IMessage
): Promise<PreparePushMessageResult> {
try {
if (!message.payload || message.payload.length === 0) {
log.error("Failed to send waku light push: payload is empty");
return { query: null, error: ProtocolError.EMPTY_PAYLOAD };
}
if (!(await isMessageSizeUnderCap(encoder, message))) {
log.error("Failed to send waku light push: message is bigger than 1MB");
return { query: null, error: ProtocolError.SIZE_TOO_BIG };
}
const protoMessage = await encoder.toProtoObj(message);
if (!protoMessage) {
log.error("Failed to encode to protoMessage, aborting push");
return {
query: null,
error: ProtocolError.ENCODE_FAILED
};
}
const query = PushRpc.createRequest(protoMessage, encoder.pubsubTopic);
return { query, error: null };
} catch (error) {
log.error("Failed to prepare push message", error);
return {
query: null,
error: ProtocolError.GENERIC_FAIL
};
}
}
// TODO(weboko): use peer.id as parameter instead
public async send(
encoder: IEncoder,
message: IMessage,
peer: Peer
): Promise<CoreProtocolResult> {
const { query, error: preparationError } = await this.preparePushMessage(
encoder,
message
);
if (preparationError || !query) {
return {
success: null,
failure: {
error: preparationError,
peerId: peer.id
}
};
}
let stream: Stream;
try {
stream = await this.getStream(peer);
} catch (error) {
log.error("Failed to get stream", error);
return {
success: null,
failure: {
error: ProtocolError.NO_STREAM_AVAILABLE,
peerId: peer.id
}
};
}
let res: Uint8ArrayList[] | undefined;
try {
res = await pipe(
[query.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
} catch (err) {
log.error("Failed to send waku light push request", err);
return {
success: null,
failure: {
error: ProtocolError.GENERIC_FAIL,
peerId: peer.id
}
};
}
const bytes = new Uint8ArrayList();
res.forEach((chunk) => {
bytes.append(chunk);
});
let response: PushResponse | undefined;
try {
response = PushRpc.decode(bytes).response;
} catch (err) {
log.error("Failed to decode push reply", err);
return {
success: null,
failure: {
error: ProtocolError.DECODE_FAILED,
peerId: peer.id
}
};
}
if (!response) {
log.error("Remote peer fault: No response in PushRPC");
return {
success: null,
failure: {
error: ProtocolError.NO_RESPONSE,
peerId: peer.id
}
};
}
if (isRLNResponseError(response.info)) {
const rlnErrorCase = matchRLNErrorMessage(response.info!);
log.error("Remote peer rejected the message: ", rlnErrorCase);
return {
success: null,
failure: {
error: rlnErrorCase,
peerId: peer.id
}
};
}
if (!response.isSuccess) {
log.error("Remote peer rejected the message: ", response.info);
return {
success: null,
failure: {
error: ProtocolError.REMOTE_PEER_REJECTED,
peerId: peer.id
}
};
}
return { success: peer.id, failure: null };
}
}

View File

@ -1,182 +1 @@
import type { PeerId } from "@libp2p/interface";
import { IncomingStreamData } from "@libp2p/interface";
import {
type IMetadata,
type Libp2pComponents,
type MetadataQueryResult,
type PeerIdStr,
ProtocolError,
PubsubTopic,
type ShardInfo
} from "@waku/interfaces";
import { proto_metadata } from "@waku/proto";
import { encodeRelayShard, Logger, pubsubTopicsToShardInfo } from "@waku/utils";
import all from "it-all";
import * as lp from "it-length-prefixed";
import { pipe } from "it-pipe";
import { Uint8ArrayList } from "uint8arraylist";
import { BaseProtocol } from "../base_protocol.js";
const log = new Logger("metadata");
export const MetadataCodec = "/vac/waku/metadata/1.0.0";
class Metadata extends BaseProtocol implements IMetadata {
private libp2pComponents: Libp2pComponents;
protected handshakesConfirmed: Map<PeerIdStr, ShardInfo> = new Map();
public constructor(
public pubsubTopics: PubsubTopic[],
libp2p: Libp2pComponents
) {
super(MetadataCodec, libp2p.components, log, pubsubTopics);
this.libp2pComponents = libp2p;
void libp2p.registrar.handle(MetadataCodec, (streamData) => {
void this.onRequest(streamData);
});
}
/**
* Make a metadata query to a peer
*/
public async query(peerId: PeerId): Promise<MetadataQueryResult> {
const request = proto_metadata.WakuMetadataRequest.encode(
pubsubTopicsToShardInfo(this.pubsubTopics)
);
const peer = await this.libp2pComponents.peerStore.get(peerId);
if (!peer) {
return {
shardInfo: null,
error: ProtocolError.NO_PEER_AVAILABLE
};
}
let stream;
try {
stream = await this.getStream(peer);
} catch (error) {
log.error("Failed to get stream", error);
return {
shardInfo: null,
error: ProtocolError.NO_STREAM_AVAILABLE
};
}
const encodedResponse = await pipe(
[request],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
const { error, shardInfo } = this.decodeMetadataResponse(encodedResponse);
if (error) {
return {
shardInfo: null,
error
};
}
await this.savePeerShardInfo(peerId, shardInfo);
return {
shardInfo,
error: null
};
}
public async confirmOrAttemptHandshake(
peerId: PeerId
): Promise<MetadataQueryResult> {
const shardInfo = this.handshakesConfirmed.get(peerId.toString());
if (shardInfo) {
return {
shardInfo,
error: null
};
}
return await this.query(peerId);
}
/**
* Handle an incoming metadata request
*/
private async onRequest(streamData: IncomingStreamData): Promise<void> {
try {
const { stream, connection } = streamData;
const encodedShardInfo = proto_metadata.WakuMetadataResponse.encode(
pubsubTopicsToShardInfo(this.pubsubTopics)
);
const encodedResponse = await pipe(
[encodedShardInfo],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
const { error, shardInfo } = this.decodeMetadataResponse(encodedResponse);
if (error) {
return;
}
await this.savePeerShardInfo(connection.remotePeer, shardInfo);
} catch (error) {
log.error("Error handling metadata request", error);
}
}
private decodeMetadataResponse(
encodedResponse: Uint8ArrayList[]
): MetadataQueryResult {
const bytes = new Uint8ArrayList();
encodedResponse.forEach((chunk) => {
bytes.append(chunk);
});
const response = proto_metadata.WakuMetadataResponse.decode(
bytes
) as ShardInfo;
if (!response) {
log.error("Error decoding metadata response");
return {
shardInfo: null,
error: ProtocolError.DECODE_FAILED
};
}
return {
shardInfo: response,
error: null
};
}
private async savePeerShardInfo(
peerId: PeerId,
shardInfo: ShardInfo
): Promise<void> {
// add or update the shardInfo to peer store
await this.libp2pComponents.peerStore.merge(peerId, {
metadata: {
shardInfo: encodeRelayShard(shardInfo)
}
});
this.handshakesConfirmed.set(peerId.toString(), shardInfo);
}
}
export function wakuMetadata(
pubsubTopics: PubsubTopic[]
): (components: Libp2pComponents) => IMetadata {
return (components: Libp2pComponents) =>
new Metadata(pubsubTopics, components);
}
export { wakuMetadata, MetadataCodec } from "./metadata.js";

View File

@ -0,0 +1,182 @@
import type { PeerId } from "@libp2p/interface";
import { IncomingStreamData } from "@libp2p/interface";
import {
type IMetadata,
type Libp2pComponents,
type MetadataQueryResult,
type PeerIdStr,
ProtocolError,
PubsubTopic,
type ShardInfo
} from "@waku/interfaces";
import { proto_metadata } from "@waku/proto";
import { encodeRelayShard, Logger, pubsubTopicsToShardInfo } from "@waku/utils";
import all from "it-all";
import * as lp from "it-length-prefixed";
import { pipe } from "it-pipe";
import { Uint8ArrayList } from "uint8arraylist";
import { BaseProtocol } from "../base_protocol.js";
const log = new Logger("metadata");
export const MetadataCodec = "/vac/waku/metadata/1.0.0";
class Metadata extends BaseProtocol implements IMetadata {
private libp2pComponents: Libp2pComponents;
protected handshakesConfirmed: Map<PeerIdStr, ShardInfo> = new Map();
public constructor(
public pubsubTopics: PubsubTopic[],
libp2p: Libp2pComponents
) {
super(MetadataCodec, libp2p.components, pubsubTopics);
this.libp2pComponents = libp2p;
void libp2p.registrar.handle(MetadataCodec, (streamData) => {
void this.onRequest(streamData);
});
}
/**
* Make a metadata query to a peer
*/
public async query(peerId: PeerId): Promise<MetadataQueryResult> {
const request = proto_metadata.WakuMetadataRequest.encode(
pubsubTopicsToShardInfo(this.pubsubTopics)
);
const peer = await this.libp2pComponents.peerStore.get(peerId);
if (!peer) {
return {
shardInfo: null,
error: ProtocolError.NO_PEER_AVAILABLE
};
}
let stream;
try {
stream = await this.getStream(peer);
} catch (error) {
log.error("Failed to get stream", error);
return {
shardInfo: null,
error: ProtocolError.NO_STREAM_AVAILABLE
};
}
const encodedResponse = await pipe(
[request],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
const { error, shardInfo } = this.decodeMetadataResponse(encodedResponse);
if (error) {
return {
shardInfo: null,
error
};
}
await this.savePeerShardInfo(peerId, shardInfo);
return {
shardInfo,
error: null
};
}
public async confirmOrAttemptHandshake(
peerId: PeerId
): Promise<MetadataQueryResult> {
const shardInfo = this.handshakesConfirmed.get(peerId.toString());
if (shardInfo) {
return {
shardInfo,
error: null
};
}
return await this.query(peerId);
}
/**
* Handle an incoming metadata request
*/
private async onRequest(streamData: IncomingStreamData): Promise<void> {
try {
const { stream, connection } = streamData;
const encodedShardInfo = proto_metadata.WakuMetadataResponse.encode(
pubsubTopicsToShardInfo(this.pubsubTopics)
);
const encodedResponse = await pipe(
[encodedShardInfo],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
const { error, shardInfo } = this.decodeMetadataResponse(encodedResponse);
if (error) {
return;
}
await this.savePeerShardInfo(connection.remotePeer, shardInfo);
} catch (error) {
log.error("Error handling metadata request", error);
}
}
private decodeMetadataResponse(
encodedResponse: Uint8ArrayList[]
): MetadataQueryResult {
const bytes = new Uint8ArrayList();
encodedResponse.forEach((chunk) => {
bytes.append(chunk);
});
const response = proto_metadata.WakuMetadataResponse.decode(
bytes
) as ShardInfo;
if (!response) {
log.error("Error decoding metadata response");
return {
shardInfo: null,
error: ProtocolError.DECODE_FAILED
};
}
return {
shardInfo: response,
error: null
};
}
private async savePeerShardInfo(
peerId: PeerId,
shardInfo: ShardInfo
): Promise<void> {
// add or update the shardInfo to peer store
await this.libp2pComponents.peerStore.merge(peerId, {
metadata: {
shardInfo: encodeRelayShard(shardInfo)
}
});
this.handshakesConfirmed.set(peerId.toString(), shardInfo);
}
}
export function wakuMetadata(
pubsubTopics: PubsubTopic[]
): (components: Libp2pComponents) => IMetadata {
return (components: Libp2pComponents) =>
new Metadata(pubsubTopics, components);
}

View File

@ -1,136 +1 @@
import type { Peer } from "@libp2p/interface";
import {
IDecodedMessage,
IDecoder,
IStoreCore,
Libp2p,
PubsubTopic,
QueryRequestParams
} from "@waku/interfaces";
import { Logger } from "@waku/utils";
import all from "it-all";
import * as lp from "it-length-prefixed";
import { pipe } from "it-pipe";
import { Uint8ArrayList } from "uint8arraylist";
import { BaseProtocol } from "../base_protocol.js";
import { toProtoMessage } from "../to_proto_message.js";
import {
DEFAULT_PAGE_SIZE,
MAX_PAGE_SIZE,
StoreQueryRequest,
StoreQueryResponse
} from "./rpc.js";
const log = new Logger("store");
export const StoreCodec = "/vac/waku/store-query/3.0.0";
export class StoreCore extends BaseProtocol implements IStoreCore {
public constructor(
public readonly pubsubTopics: PubsubTopic[],
libp2p: Libp2p
) {
super(StoreCodec, libp2p.components, log, pubsubTopics);
}
public async *queryPerPage<T extends IDecodedMessage>(
queryOpts: QueryRequestParams,
decoders: Map<string, IDecoder<T>>,
peer: Peer
): AsyncGenerator<Promise<T | undefined>[]> {
if (
queryOpts.contentTopics.toString() !==
Array.from(decoders.keys()).toString()
) {
throw new Error(
"Internal error, the decoders should match the query's content topics"
);
}
let currentCursor = queryOpts.paginationCursor;
while (true) {
const storeQueryRequest = StoreQueryRequest.create({
...queryOpts,
paginationCursor: currentCursor
});
let stream;
try {
stream = await this.getStream(peer);
} catch (e) {
log.error("Failed to get stream", e);
break;
}
const res = await pipe(
[storeQueryRequest.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
const bytes = new Uint8ArrayList();
res.forEach((chunk) => {
bytes.append(chunk);
});
const storeQueryResponse = StoreQueryResponse.decode(bytes);
if (
!storeQueryResponse.statusCode ||
storeQueryResponse.statusCode >= 300
) {
const errorMessage = `Store query failed with status code: ${storeQueryResponse.statusCode}, description: ${storeQueryResponse.statusDesc}`;
log.error(errorMessage);
throw new Error(errorMessage);
}
if (!storeQueryResponse.messages || !storeQueryResponse.messages.length) {
log.warn("Stopping pagination due to empty messages in response");
break;
}
log.info(
`${storeQueryResponse.messages.length} messages retrieved from store`
);
const decodedMessages = storeQueryResponse.messages.map((protoMsg) => {
if (!protoMsg.message) {
return Promise.resolve(undefined);
}
const contentTopic = protoMsg.message.contentTopic;
if (contentTopic) {
const decoder = decoders.get(contentTopic);
if (decoder) {
return decoder.fromProtoObj(
protoMsg.pubsubTopic || "",
toProtoMessage(protoMsg.message)
);
}
}
return Promise.resolve(undefined);
});
yield decodedMessages;
if (queryOpts.paginationForward) {
currentCursor =
storeQueryResponse.messages[storeQueryResponse.messages.length - 1]
.messageHash;
} else {
currentCursor = storeQueryResponse.messages[0].messageHash;
}
if (
storeQueryResponse.messages.length > MAX_PAGE_SIZE &&
storeQueryResponse.messages.length <
(queryOpts.paginationLimit || DEFAULT_PAGE_SIZE)
) {
break;
}
}
}
}
export { StoreCore, StoreCodec } from "./store.js";

View File

@ -0,0 +1,136 @@
import type { Peer } from "@libp2p/interface";
import {
IDecodedMessage,
IDecoder,
IStoreCore,
Libp2p,
PubsubTopic,
QueryRequestParams
} from "@waku/interfaces";
import { Logger } from "@waku/utils";
import all from "it-all";
import * as lp from "it-length-prefixed";
import { pipe } from "it-pipe";
import { Uint8ArrayList } from "uint8arraylist";
import { BaseProtocol } from "../base_protocol.js";
import { toProtoMessage } from "../to_proto_message.js";
import {
DEFAULT_PAGE_SIZE,
MAX_PAGE_SIZE,
StoreQueryRequest,
StoreQueryResponse
} from "./rpc.js";
const log = new Logger("store");
export const StoreCodec = "/vac/waku/store-query/3.0.0";
export class StoreCore extends BaseProtocol implements IStoreCore {
public constructor(
public readonly pubsubTopics: PubsubTopic[],
libp2p: Libp2p
) {
super(StoreCodec, libp2p.components, pubsubTopics);
}
public async *queryPerPage<T extends IDecodedMessage>(
queryOpts: QueryRequestParams,
decoders: Map<string, IDecoder<T>>,
peer: Peer
): AsyncGenerator<Promise<T | undefined>[]> {
if (
queryOpts.contentTopics.toString() !==
Array.from(decoders.keys()).toString()
) {
throw new Error(
"Internal error, the decoders should match the query's content topics"
);
}
let currentCursor = queryOpts.paginationCursor;
while (true) {
const storeQueryRequest = StoreQueryRequest.create({
...queryOpts,
paginationCursor: currentCursor
});
let stream;
try {
stream = await this.getStream(peer);
} catch (e) {
log.error("Failed to get stream", e);
break;
}
const res = await pipe(
[storeQueryRequest.encode()],
lp.encode,
stream,
lp.decode,
async (source) => await all(source)
);
const bytes = new Uint8ArrayList();
res.forEach((chunk) => {
bytes.append(chunk);
});
const storeQueryResponse = StoreQueryResponse.decode(bytes);
if (
!storeQueryResponse.statusCode ||
storeQueryResponse.statusCode >= 300
) {
const errorMessage = `Store query failed with status code: ${storeQueryResponse.statusCode}, description: ${storeQueryResponse.statusDesc}`;
log.error(errorMessage);
throw new Error(errorMessage);
}
if (!storeQueryResponse.messages || !storeQueryResponse.messages.length) {
log.warn("Stopping pagination due to empty messages in response");
break;
}
log.info(
`${storeQueryResponse.messages.length} messages retrieved from store`
);
const decodedMessages = storeQueryResponse.messages.map((protoMsg) => {
if (!protoMsg.message) {
return Promise.resolve(undefined);
}
const contentTopic = protoMsg.message.contentTopic;
if (contentTopic) {
const decoder = decoders.get(contentTopic);
if (decoder) {
return decoder.fromProtoObj(
protoMsg.pubsubTopic || "",
toProtoMessage(protoMsg.message)
);
}
}
return Promise.resolve(undefined);
});
yield decodedMessages;
if (queryOpts.paginationForward) {
currentCursor =
storeQueryResponse.messages[storeQueryResponse.messages.length - 1]
.messageHash;
} else {
currentCursor = storeQueryResponse.messages[0].messageHash;
}
if (
storeQueryResponse.messages.length > MAX_PAGE_SIZE &&
storeQueryResponse.messages.length <
(queryOpts.paginationLimit || DEFAULT_PAGE_SIZE)
) {
break;
}
}
}
}

View File

@ -32,7 +32,7 @@ export class WakuPeerExchange extends BaseProtocol implements IPeerExchange {
components: Libp2pComponents,
pubsubTopics: PubsubTopic[]
) {
super(PeerExchangeCodec, components, log, pubsubTopics);
super(PeerExchangeCodec, components, pubsubTopics);
}
/**

View File

@ -8,22 +8,44 @@ export enum Tags {
LOCAL = "local-peer-cache"
}
export interface ConnectionManagerOptions {
export type ConnectionManagerOptions = {
/**
* Number of attempts before a peer is considered non-dialable
* This is used to not spam a peer with dial attempts when it is not dialable
* Number of attempts before a peer is considered non-dialable.
* This is used to not spam a peer with dial attempts when it is not dialable.
*
* @default 3
*/
maxDialAttemptsForPeer: number;
/**
* Max number of bootstrap peers allowed to be connected to, initially
* This is used to increase intention of dialing non-bootstrap peers, found using other discovery mechanisms (like Peer Exchange)
* Max number of bootstrap peers allowed to be connected to initially.
* This is used to increase intention of dialing non-bootstrap peers, found using other discovery mechanisms (like Peer Exchange).
*
* @default 1
*/
maxBootstrapPeersAllowed: number;
/**
* Max number of parallel dials allowed
* Max number of parallel dials allowed.
*
* @default 3
*/
maxParallelDials: number;
}
/**
* Keep alive libp2p pings interval in seconds.
*
* @default 300 seconds
*/
pingKeepAlive: number;
/**
* Gossip sub specific keep alive interval in seconds.
*
* @default 300 seconds
*/
relayKeepAlive: number;
};
export enum EPeersByDiscoveryEvents {
PEER_DISCOVERY_BOOTSTRAP = "peer:discovery:bootstrap",
@ -63,7 +85,8 @@ export interface IConnectionStateEvents {
export interface IConnectionManager
extends TypedEventEmitter<IPeersByDiscoveryEvents & IConnectionStateEvents> {
configuredPubsubTopics: PubsubTopic[];
pubsubTopics: PubsubTopic[];
getConnectedPeers(codec?: string): Promise<Peer[]>;
dropConnection(peerId: PeerId): Promise<void>;
getPeersByDiscovery(): Promise<PeersByDiscoveryResult>;
stop(): void;

View File

@ -5,9 +5,7 @@ import type { ContentTopic, ThisOrThat } from "./misc.js";
import type {
Callback,
IBaseProtocolCore,
IBaseProtocolSDK,
ProtocolError,
ProtocolUseOptions,
SDKProtocolResult
} from "./protocols.js";
import type { IReceiver } from "./receiver.js";
@ -17,17 +15,34 @@ export type SubscriptionCallback<T extends IDecodedMessage> = {
callback: Callback<T>;
};
export type SubscribeOptions = {
keepAlive?: number;
pingsBeforePeerRenewed?: number;
enableLightPushFilterCheck?: boolean;
export type FilterProtocolOptions = {
/**
* Interval with which Filter subscription will attempt to send ping requests to subscribed peers.
*
* @default 60_000
*/
keepAliveIntervalMs: number;
/**
* Number of failed pings allowed to make to a remote peer before attempting to subscribe to a new one.
*
* @default 3
*/
pingsBeforePeerRenewed: number;
/**
* Enables js-waku to send probe LightPush message over subscribed pubsubTopics on created subscription.
* In case message won't be received back through Filter - js-waku will attempt to subscribe to another peer.
*
* @default false
*/
enableLightPushFilterCheck: boolean;
};
export interface ISubscription {
subscribe<T extends IDecodedMessage>(
decoders: IDecoder<T> | IDecoder<T>[],
callback: Callback<T>,
options?: SubscribeOptions
callback: Callback<T>
): Promise<SDKProtocolResult>;
unsubscribe(contentTopics: ContentTopic[]): Promise<SDKProtocolResult>;
@ -37,15 +52,12 @@ export interface ISubscription {
unsubscribeAll(): Promise<SDKProtocolResult>;
}
export type IFilter = IReceiver &
IBaseProtocolSDK & { protocol: IBaseProtocolCore } & {
subscribe<T extends IDecodedMessage>(
decoders: IDecoder<T> | IDecoder<T>[],
callback: Callback<T>,
protocolUseOptions?: ProtocolUseOptions,
subscribeOptions?: SubscribeOptions
): Promise<SubscribeResult>;
};
export type IFilter = IReceiver & { protocol: IBaseProtocolCore } & {
subscribe<T extends IDecodedMessage>(
decoders: IDecoder<T> | IDecoder<T>[],
callback: Callback<T>
): Promise<SubscribeResult>;
};
export type SubscribeResult = SubscriptionSuccess | SubscriptionError;

View File

@ -12,7 +12,6 @@ export * from "./sender.js";
export * from "./receiver.js";
export * from "./misc.js";
export * from "./libp2p.js";
export * from "./keep_alive_manager.js";
export * from "./dns_discovery.js";
export * from "./metadata.js";
export * from "./constants.js";

View File

@ -1,4 +0,0 @@
export interface KeepAliveOptions {
pingKeepAlive: number;
relayKeepAlive: number;
}

View File

@ -1,11 +1,13 @@
import type { Libp2p } from "@libp2p/interface";
import type { PeerId } from "@libp2p/interface";
import type { Peer } from "@libp2p/interface";
import type { ConnectionManagerOptions } from "./connection_manager.js";
import type { FilterProtocolOptions } from "./filter.js";
import type { CreateLibp2pOptions } from "./libp2p.js";
import type { IDecodedMessage } from "./message.js";
import { ThisAndThat, ThisOrThat } from "./misc.js";
import { AutoSharding, StaticSharding } from "./sharding.js";
import type { StoreProtocolOptions } from "./store.js";
export enum Protocols {
Relay = "relay",
@ -16,52 +18,20 @@ export enum Protocols {
export type IBaseProtocolCore = {
multicodec: string;
allPeers: () => Promise<Peer[]>;
connectedPeers: () => Promise<Peer[]>;
addLibp2pEventListener: Libp2p["addEventListener"];
removeLibp2pEventListener: Libp2p["removeEventListener"];
};
export type IBaseProtocolSDK = {
readonly connectedPeers: Peer[];
renewPeer: (peerToDisconnect: PeerId) => Promise<Peer | undefined>;
readonly numPeersToUse: number;
};
export type StoreProtocolOptions = {
peer: string;
};
export type NetworkConfig = StaticSharding | AutoSharding;
//TODO: merge this with ProtocolCreateOptions or establish distinction: https://github.com/waku-org/js-waku/issues/2048
/**
* Options for using LightPush and Filter
*/
export type ProtocolUseOptions = {
export type CreateNodeOptions = {
/**
* Optional flag to force using all available peers
* Set the user agent string to be used in identification of the node.
*
* @default "js-waku"
*/
forceUseAllPeers?: boolean;
/**
* Optional maximum number of attempts for exponential backoff
*/
maxAttempts?: number;
};
userAgent?: string;
export type ProtocolCreateOptions = {
/**
* Configuration for determining the network in use.
*
* If using Static Sharding:
* Default value is configured for The Waku Network.
* The format to specify a shard is: clusterId: number, shards: number[]
* To learn more about the sharding specification, see [Relay Sharding](https://rfc.vac.dev/spec/51/).
*
* If using Auto Sharding:
* See [Waku v2 Topic Usage Recommendations](https://github.com/vacp2p/rfc-index/blob/main/waku/informational/23/topics.md#content-topics) for details.
* You cannot add or remove content topics after initialization of the node.
*/
/**
* Configuration for determining the network in use.
* Network configuration refers to the shards and clusters used in the network.
@ -80,6 +50,7 @@ export type ProtocolCreateOptions = {
* @default { clusterId: 1, shards: [0, 1, 2, 3, 4, 5, 6, 7] }
*/
networkConfig?: NetworkConfig;
/**
* You can pass options to the `Libp2p` instance used by {@link @waku/sdk!WakuNode} using the `libp2p` property.
* This property is the same type as the one passed to [`Libp2p.create`](https://github.com/libp2p/js-libp2p/blob/master/doc/API.md#create)
@ -88,28 +59,46 @@ export type ProtocolCreateOptions = {
* Notes that some values are overridden by {@link @waku/sdk!WakuNode} to ensure it implements the Waku protocol.
*/
libp2p?: Partial<CreateLibp2pOptions>;
/**
* Number of peers to connect to, for the usage of the protocol.
* This is used by:
* - Light Push to send messages,
* - Filter to retrieve messages.
* Defaults to 2.
*
* @default 2.
*/
numPeersToUse?: number;
/**
* Byte array used as key for the noise protocol used for connection encryption
* by [`Libp2p.create`](https://github.com/libp2p/js-libp2p/blob/master/doc/API.md#create)
* This is only used for test purposes to not run out of entropy during CI runs.
*/
staticNoiseKey?: Uint8Array;
/**
* Use recommended bootstrap method to discovery and connect to new nodes.
*/
defaultBootstrap?: boolean;
/**
* List of peers to use to bootstrap the node. Ignored if defaultBootstrap is set to true.
*/
bootstrapPeers?: string[];
/**
* Configuration for connection manager.
* If not specified - default values are applied.
*/
connectionManager?: Partial<ConnectionManagerOptions>;
/**
* Configuration for Filter protocol.
* If not specified - default values are applied.
*/
filter?: Partial<FilterProtocolOptions>;
/**
* Options for the Store protocol.
*/

View File

@ -1,5 +1,5 @@
import type { IDecodedMessage, IDecoder } from "./message.js";
import type { IBaseProtocolCore, IBaseProtocolSDK } from "./protocols.js";
import type { IBaseProtocolCore } from "./protocols.js";
export type StoreCursor = Uint8Array;
@ -78,7 +78,7 @@ export type QueryRequestParams = {
export type IStoreCore = IBaseProtocolCore;
export type IStore = IBaseProtocolSDK & {
export type IStore = {
protocol: IBaseProtocolCore;
createCursor(message: IDecodedMessage): StoreCursor;
queryGenerator: <T extends IDecodedMessage>(
@ -99,3 +99,7 @@ export type IStore = IBaseProtocolSDK & {
options?: Partial<QueryRequestParams>
) => Promise<void>;
};
export type StoreProtocolOptions = {
peer: string;
};

View File

@ -1,4 +1,4 @@
import type { PeerId, Stream } from "@libp2p/interface";
import type { Peer, PeerId, Stream } from "@libp2p/interface";
import type { MultiaddrInput } from "@multiformats/multiaddr";
import { IConnectionManager } from "./connection_manager.js";
@ -121,6 +121,11 @@ export interface IWaku {
* @returns {boolean} `true` if the node has working connection and `false` otherwise
*/
isConnected(): boolean;
/**
* @returns {Peer[]} an array of all connected peers
*/
getConnectedPeers(): Promise<Peer[]>;
}
export interface LightNode extends IWaku {

View File

@ -1,10 +1,5 @@
import type { RelayNode } from "@waku/interfaces";
import {
createLibp2pAndUpdateOptions,
CreateWakuNodeOptions,
WakuNode,
WakuOptions
} from "@waku/sdk";
import type { CreateNodeOptions, RelayNode } from "@waku/interfaces";
import { createLibp2pAndUpdateOptions, WakuNode } from "@waku/sdk";
import { RelayCreateOptions, wakuGossipSub, wakuRelay } from "./relay.js";
@ -19,7 +14,7 @@ import { RelayCreateOptions, wakuGossipSub, wakuRelay } from "./relay.js";
* or use this function with caution.
*/
export async function createRelayNode(
options: CreateWakuNodeOptions & Partial<RelayCreateOptions>
options: CreateNodeOptions & Partial<RelayCreateOptions>
): Promise<RelayNode> {
options = {
...options,
@ -36,7 +31,7 @@ export async function createRelayNode(
return new WakuNode(
pubsubTopics,
options as WakuOptions,
options as CreateNodeOptions,
libp2p,
{},
relay

View File

@ -11,6 +11,7 @@ import { sha256 } from "@noble/hashes/sha256";
import {
ActiveSubscriptions,
Callback,
CreateNodeOptions,
IAsyncIterator,
IDecodedMessage,
IDecoder,
@ -18,7 +19,6 @@ import {
IMessage,
IRelay,
Libp2p,
ProtocolCreateOptions,
ProtocolError,
PubsubTopic,
SDKProtocolResult
@ -39,7 +39,7 @@ export type Observer<T extends IDecodedMessage> = {
callback: Callback<T>;
};
export type RelayCreateOptions = ProtocolCreateOptions & GossipsubOpts;
export type RelayCreateOptions = CreateNodeOptions & GossipsubOpts;
export type ContentTopic = string;
/**

View File

@ -73,10 +73,10 @@
"@waku/proto": "^0.0.8",
"@waku/utils": "0.0.21",
"@waku/message-hash": "0.1.17",
"async-mutex": "^0.5.0",
"libp2p": "2.1.8"
},
"devDependencies": {
"@libp2p/interface": "2.1.3",
"@types/chai": "^4.3.11",
"@rollup/plugin-commonjs": "^25.0.7",
"@rollup/plugin-json": "^6.0.0",

View File

@ -1,6 +1,6 @@
import { type LightNode } from "@waku/interfaces";
import type { CreateNodeOptions, LightNode } from "@waku/interfaces";
import { CreateWakuNodeOptions, WakuNode } from "../waku/index.js";
import { WakuNode } from "../waku/index.js";
import { createLibp2pAndUpdateOptions } from "./libp2p.js";
@ -10,7 +10,7 @@ import { createLibp2pAndUpdateOptions } from "./libp2p.js";
* Uses Waku Filter V2 by default.
*/
export async function createLightNode(
options: CreateWakuNodeOptions = {}
options: CreateNodeOptions = {}
): Promise<LightNode> {
const { libp2p, pubsubTopics } = await createLibp2pAndUpdateOptions(options);

View File

@ -8,6 +8,7 @@ import { all as filterAll, wss } from "@libp2p/websockets/filters";
import { wakuMetadata } from "@waku/core";
import {
type CreateLibp2pOptions,
type CreateNodeOptions,
DefaultNetworkConfig,
type IMetadata,
type Libp2p,
@ -18,11 +19,6 @@ import { derivePubsubTopicsFromNetworkConfig, Logger } from "@waku/utils";
import { createLibp2p } from "libp2p";
import { isTestEnvironment } from "../env.js";
import {
CreateWakuNodeOptions,
DefaultPingMaxInboundStreams,
DefaultUserAgent
} from "../waku/index.js";
import { defaultPeerDiscoveries } from "./discovery.js";
@ -32,6 +28,9 @@ type MetadataService = {
const log = new Logger("sdk:create");
const DefaultUserAgent = "js-waku";
const DefaultPingMaxInboundStreams = 10;
export async function defaultLibp2p(
pubsubTopics: PubsubTopic[],
options?: Partial<CreateLibp2pOptions>,
@ -79,7 +78,7 @@ export async function defaultLibp2p(
}
export async function createLibp2pAndUpdateOptions(
options: CreateWakuNodeOptions
options: CreateNodeOptions
): Promise<{ libp2p: Libp2p; pubsubTopics: PubsubTopic[] }> {
const { networkConfig } = options;
const pubsubTopics = derivePubsubTopicsFromNetworkConfig(

View File

@ -0,0 +1,3 @@
export const DEFAULT_KEEP_ALIVE = 60_000;
export const DEFAULT_MAX_PINGS = 3;
export const DEFAULT_LIGHT_PUSH_FILTER_CHECK = false;

View File

@ -1,22 +1,19 @@
import { ConnectionManager, FilterCore } from "@waku/core";
import {
type Callback,
type CreateSubscriptionResult,
type IAsyncIterator,
type IDecodedMessage,
type IDecoder,
type IFilter,
type ILightPush,
type Libp2p,
NetworkConfig,
type ProtocolCreateOptions,
ProtocolError,
type ProtocolUseOptions,
type PubsubTopic,
type SubscribeOptions,
import type {
Callback,
CreateSubscriptionResult,
FilterProtocolOptions,
IAsyncIterator,
IDecodedMessage,
IDecoder,
IFilter,
ILightPush,
Libp2p,
PubsubTopic,
SubscribeResult,
type Unsubscribe
Unsubscribe
} from "@waku/interfaces";
import { NetworkConfig, ProtocolError } from "@waku/interfaces";
import {
ensurePubsubTopicIsConfigured,
groupByContentTopic,
@ -25,46 +22,45 @@ import {
toAsyncIterator
} from "@waku/utils";
import { BaseProtocolSDK } from "../base_protocol.js";
import { PeerManager } from "../peer_manager/index.js";
import { DEFAULT_SUBSCRIBE_OPTIONS } from "./constants.js";
import { SubscriptionManager } from "./subscription_manager.js";
import { Subscription } from "./subscription.js";
import { buildConfig } from "./utils.js";
const log = new Logger("sdk:filter");
class Filter extends BaseProtocolSDK implements IFilter {
class Filter implements IFilter {
public readonly protocol: FilterCore;
private activeSubscriptions = new Map<string, SubscriptionManager>();
private readonly config: FilterProtocolOptions;
private activeSubscriptions = new Map<string, Subscription>();
public constructor(
connectionManager: ConnectionManager,
private connectionManager: ConnectionManager,
private libp2p: Libp2p,
private peerManager: PeerManager,
private lightPush?: ILightPush,
options?: ProtocolCreateOptions
config?: Partial<FilterProtocolOptions>
) {
super(
new FilterCore(
async (pubsubTopic, wakuMessage, peerIdStr) => {
const subscription = this.getActiveSubscription(pubsubTopic);
if (!subscription) {
log.error(
`No subscription locally registered for topic ${pubsubTopic}`
);
return;
}
await subscription.processIncomingMessage(wakuMessage, peerIdStr);
},
this.config = buildConfig(config);
connectionManager.configuredPubsubTopics,
libp2p
),
connectionManager,
{ numPeersToUse: options?.numPeersToUse }
this.protocol = new FilterCore(
async (pubsubTopic, wakuMessage, peerIdStr) => {
const subscription = this.getActiveSubscription(pubsubTopic);
if (!subscription) {
log.error(
`No subscription locally registered for topic ${pubsubTopic}`
);
return;
}
await subscription.processIncomingMessage(wakuMessage, peerIdStr);
},
connectionManager.pubsubTopics,
libp2p
);
this.protocol = this.core as FilterCore;
this.activeSubscriptions = new Map();
}
@ -74,8 +70,6 @@ class Filter extends BaseProtocolSDK implements IFilter {
*
* @param {IDecoder<T> | IDecoder<T>[]} decoders - A single decoder or an array of decoders to use for decoding messages.
* @param {Callback<T>} callback - The callback function to be invoked with decoded messages.
* @param {ProtocolUseOptions} [protocolUseOptions] - Optional settings for using the protocol.
* @param {SubscribeOptions} [subscribeOptions=DEFAULT_SUBSCRIBE_OPTIONS] - Options for the subscription.
*
* @returns {Promise<SubscribeResult>} A promise that resolves to an object containing:
* - subscription: The created subscription object if successful, or null if failed.
@ -109,9 +103,7 @@ class Filter extends BaseProtocolSDK implements IFilter {
*/
public async subscribe<T extends IDecodedMessage>(
decoders: IDecoder<T> | IDecoder<T>[],
callback: Callback<T>,
protocolUseOptions?: ProtocolUseOptions,
subscribeOptions: SubscribeOptions = DEFAULT_SUBSCRIBE_OPTIONS
callback: Callback<T>
): Promise<SubscribeResult> {
const uniquePubsubTopics = this.getUniquePubsubTopics(decoders);
@ -125,10 +117,7 @@ class Filter extends BaseProtocolSDK implements IFilter {
const pubsubTopic = uniquePubsubTopics[0];
const { subscription, error } = await this.createSubscription(
pubsubTopic,
protocolUseOptions
);
const { subscription, error } = await this.createSubscription(pubsubTopic);
if (error) {
return {
@ -140,8 +129,7 @@ class Filter extends BaseProtocolSDK implements IFilter {
const { failures, successes } = await subscription.subscribe(
decoders,
callback,
subscribeOptions
callback
);
return {
subscription,
@ -160,14 +148,8 @@ class Filter extends BaseProtocolSDK implements IFilter {
* @returns The subscription object.
*/
private async createSubscription(
pubsubTopicShardInfo: NetworkConfig | PubsubTopic,
options?: ProtocolUseOptions
pubsubTopicShardInfo: NetworkConfig | PubsubTopic
): Promise<CreateSubscriptionResult> {
options = {
autoRetry: true,
...options
} as ProtocolUseOptions;
const pubsubTopic =
typeof pubsubTopicShardInfo == "string"
? pubsubTopicShardInfo
@ -175,8 +157,8 @@ class Filter extends BaseProtocolSDK implements IFilter {
ensurePubsubTopicIsConfigured(pubsubTopic, this.protocol.pubsubTopics);
const hasPeers = await this.hasPeers(options);
if (!hasPeers) {
const peers = await this.peerManager.getPeers();
if (peers.length === 0) {
return {
error: ProtocolError.NO_PEER_AVAILABLE,
subscription: null
@ -184,21 +166,21 @@ class Filter extends BaseProtocolSDK implements IFilter {
}
log.info(
`Creating filter subscription with ${this.connectedPeers.length} peers: `,
this.connectedPeers.map((peer) => peer.id.toString())
`Creating filter subscription with ${peers.length} peers: `,
peers.map((peer) => peer.id.toString())
);
const subscription =
this.getActiveSubscription(pubsubTopic) ??
this.setActiveSubscription(
pubsubTopic,
new SubscriptionManager(
new Subscription(
pubsubTopic,
this.protocol,
this.connectionManager,
() => this.connectedPeers,
this.renewPeer.bind(this),
this.peerManager,
this.libp2p,
this.config,
this.lightPush
)
);
@ -226,8 +208,7 @@ class Filter extends BaseProtocolSDK implements IFilter {
*/
public async subscribeWithUnsubscribe<T extends IDecodedMessage>(
decoders: IDecoder<T> | IDecoder<T>[],
callback: Callback<T>,
options: SubscribeOptions = DEFAULT_SUBSCRIBE_OPTIONS
callback: Callback<T>
): Promise<Unsubscribe> {
const uniquePubsubTopics = this.getUniquePubsubTopics<T>(decoders);
@ -251,7 +232,7 @@ class Filter extends BaseProtocolSDK implements IFilter {
throw Error(`Failed to create subscription: ${error}`);
}
await subscription.subscribe(decoders, callback, options);
await subscription.subscribe(decoders, callback);
const contentTopics = Array.from(
groupByContentTopic(
@ -270,17 +251,16 @@ class Filter extends BaseProtocolSDK implements IFilter {
return toAsyncIterator(this, decoders);
}
//TODO: move to SubscriptionManager
private getActiveSubscription(
pubsubTopic: PubsubTopic
): SubscriptionManager | undefined {
): Subscription | undefined {
return this.activeSubscriptions.get(pubsubTopic);
}
private setActiveSubscription(
pubsubTopic: PubsubTopic,
subscription: SubscriptionManager
): SubscriptionManager {
subscription: Subscription
): Subscription {
this.activeSubscriptions.set(pubsubTopic, subscription);
return subscription;
}
@ -304,9 +284,10 @@ class Filter extends BaseProtocolSDK implements IFilter {
export function wakuFilter(
connectionManager: ConnectionManager,
peerManager: PeerManager,
lightPush?: ILightPush,
init?: ProtocolCreateOptions
config?: Partial<FilterProtocolOptions>
): (libp2p: Libp2p) => IFilter {
return (libp2p: Libp2p) =>
new Filter(connectionManager, libp2p, lightPush, init);
new Filter(connectionManager, libp2p, peerManager, lightPush, config);
}

View File

@ -0,0 +1 @@
export { wakuFilter } from "./filter.js";

View File

@ -0,0 +1,260 @@
import { ConnectionManager, createDecoder, FilterCore } from "@waku/core";
import {
type Callback,
type ContentTopic,
type CoreProtocolResult,
FilterProtocolOptions,
type IDecodedMessage,
type IDecoder,
type ILightPush,
type IProtoMessage,
type ISubscription,
type Libp2p,
type PeerIdStr,
ProtocolError,
type PubsubTopic,
type SDKProtocolResult,
SubscriptionCallback
} from "@waku/interfaces";
import { WakuMessage } from "@waku/proto";
import { groupByContentTopic, Logger } from "@waku/utils";
import { PeerManager } from "../peer_manager/index.js";
import { SubscriptionMonitor } from "./subscription_monitor.js";
const log = new Logger("sdk:filter:subscription");
export class Subscription implements ISubscription {
private readonly monitor: SubscriptionMonitor;
private subscriptionCallbacks: Map<
ContentTopic,
SubscriptionCallback<IDecodedMessage>
> = new Map();
public constructor(
private readonly pubsubTopic: PubsubTopic,
private readonly protocol: FilterCore,
connectionManager: ConnectionManager,
peerManager: PeerManager,
libp2p: Libp2p,
private readonly config: FilterProtocolOptions,
lightPush?: ILightPush
) {
this.pubsubTopic = pubsubTopic;
this.monitor = new SubscriptionMonitor({
pubsubTopic,
config,
libp2p,
connectionManager,
filter: protocol,
peerManager,
lightPush,
activeSubscriptions: this.subscriptionCallbacks
});
}
public async subscribe<T extends IDecodedMessage>(
decoders: IDecoder<T> | IDecoder<T>[],
callback: Callback<T>
): Promise<SDKProtocolResult> {
const decodersArray = Array.isArray(decoders) ? decoders : [decoders];
// check that all decoders are configured for the same pubsub topic as this subscription
for (const decoder of decodersArray) {
if (decoder.pubsubTopic !== this.pubsubTopic) {
return {
failures: [
{
error: ProtocolError.TOPIC_DECODER_MISMATCH
}
],
successes: []
};
}
}
if (this.config.enableLightPushFilterCheck) {
decodersArray.push(
createDecoder(
this.monitor.reservedContentTopic,
this.pubsubTopic
) as IDecoder<T>
);
}
const decodersGroupedByCT = groupByContentTopic(decodersArray);
const contentTopics = Array.from(decodersGroupedByCT.keys());
const peers = await this.monitor.getPeers();
const promises = peers.map(async (peer) => {
return this.protocol.subscribe(this.pubsubTopic, peer, contentTopics);
});
const results = await Promise.allSettled(promises);
const finalResult = this.handleResult(results, "subscribe");
// Save the callback functions by content topics so they
// can easily be removed (reciprocally replaced) if `unsubscribe` (reciprocally `subscribe`)
// is called for those content topics
decodersGroupedByCT.forEach((decoders, contentTopic) => {
// Cast the type because a given `subscriptionCallbacks` map may hold
// Decoder that decode to different implementations of `IDecodedMessage`
const subscriptionCallback = {
decoders,
callback
} as unknown as SubscriptionCallback<IDecodedMessage>;
// don't handle case of internal content topic
if (contentTopic === this.monitor.reservedContentTopic) {
return;
}
// The callback and decoder may override previous values, this is on
// purpose as the user may call `subscribe` to refresh the subscription
this.subscriptionCallbacks.set(contentTopic, subscriptionCallback);
});
this.monitor.start();
return finalResult;
}
public async unsubscribe(
contentTopics: ContentTopic[]
): Promise<SDKProtocolResult> {
const peers = await this.monitor.getPeers();
const promises = peers.map(async (peer) => {
const response = await this.protocol.unsubscribe(
this.pubsubTopic,
peer,
contentTopics
);
contentTopics.forEach((contentTopic: string) => {
this.subscriptionCallbacks.delete(contentTopic);
});
return response;
});
const results = await Promise.allSettled(promises);
const finalResult = this.handleResult(results, "unsubscribe");
if (this.subscriptionCallbacks.size === 0) {
this.monitor.stop();
}
return finalResult;
}
public async ping(): Promise<SDKProtocolResult> {
const peers = await this.monitor.getPeers();
const promises = peers.map((peer) => this.protocol.ping(peer));
const results = await Promise.allSettled(promises);
return this.handleResult(results, "ping");
}
public async unsubscribeAll(): Promise<SDKProtocolResult> {
const peers = await this.monitor.getPeers();
const promises = peers.map(async (peer) =>
this.protocol.unsubscribeAll(this.pubsubTopic, peer)
);
const results = await Promise.allSettled(promises);
this.subscriptionCallbacks.clear();
const finalResult = this.handleResult(results, "unsubscribeAll");
this.monitor.stop();
return finalResult;
}
public async processIncomingMessage(
message: WakuMessage,
peerIdStr: PeerIdStr
): Promise<void> {
const received = this.monitor.notifyMessageReceived(
peerIdStr,
message as IProtoMessage
);
if (received) {
log.info("Message already received, skipping");
return;
}
const { contentTopic } = message;
const subscriptionCallback = this.subscriptionCallbacks.get(contentTopic);
if (!subscriptionCallback) {
log.error("No subscription callback available for ", contentTopic);
return;
}
log.info(
"Processing message with content topic ",
contentTopic,
" on pubsub topic ",
this.pubsubTopic
);
await pushMessage(subscriptionCallback, this.pubsubTopic, message);
}
private handleResult(
results: PromiseSettledResult<CoreProtocolResult>[],
type: "ping" | "subscribe" | "unsubscribe" | "unsubscribeAll"
): SDKProtocolResult {
const result: SDKProtocolResult = { failures: [], successes: [] };
for (const promiseResult of results) {
if (promiseResult.status === "rejected") {
log.error(
`Failed to resolve ${type} promise successfully: `,
promiseResult.reason
);
result.failures.push({ error: ProtocolError.GENERIC_FAIL });
} else {
const coreResult = promiseResult.value;
if (coreResult.failure) {
result.failures.push(coreResult.failure);
} else {
result.successes.push(coreResult.success);
}
}
}
return result;
}
}
async function pushMessage<T extends IDecodedMessage>(
subscriptionCallback: SubscriptionCallback<T>,
pubsubTopic: PubsubTopic,
message: WakuMessage
): Promise<void> {
const { decoders, callback } = subscriptionCallback;
const { contentTopic } = message;
if (!contentTopic) {
log.warn("Message has no content topic, skipping");
return;
}
try {
const decodePromises = decoders.map((dec) =>
dec
.fromProtoObj(pubsubTopic, message as IProtoMessage)
.then((decoded) => decoded || Promise.reject("Decoding failed"))
);
const decodedMessage = await Promise.any(decodePromises);
await callback(decodedMessage);
} catch (e) {
log.error("Error decoding message", e);
}
}

View File

@ -0,0 +1,293 @@
import type { EventHandler, Peer, PeerId } from "@libp2p/interface";
import { FilterCore } from "@waku/core";
import type {
FilterProtocolOptions,
IConnectionManager,
ILightPush,
IProtoMessage,
Libp2p
} from "@waku/interfaces";
import { EConnectionStateEvents } from "@waku/interfaces";
import { messageHashStr } from "@waku/message-hash";
import { PeerManager } from "../peer_manager/index.js";
// TODO(weboko): consider adding as config property or combine with maxAllowedPings
const MAX_SUBSCRIBE_ATTEMPTS = 3;
type SubscriptionMonitorConstructorOptions = {
pubsubTopic: string;
config: FilterProtocolOptions;
libp2p: Libp2p;
connectionManager: IConnectionManager;
filter: FilterCore;
peerManager: PeerManager;
lightPush?: ILightPush;
activeSubscriptions: Map<string, unknown>;
};
export class SubscriptionMonitor {
/**
* Cached peers that are in use by subscription.
* Needed to understand if they disconnect later or not.
*/
public peers: Peer[] = [];
private isStarted: boolean = false;
private readonly pubsubTopic: string;
private readonly config: FilterProtocolOptions;
private readonly libp2p: Libp2p;
private readonly filter: FilterCore;
private readonly peerManager: PeerManager;
private readonly connectionManager: IConnectionManager;
private readonly activeSubscriptions: Map<string, unknown>;
private keepAliveIntervalId: number | undefined;
private pingFailedAttempts = new Map<string, number>();
private receivedMessagesFormPeer = new Set<string>();
private receivedMessages = new Set<string>();
private verifiedPeers = new Set<string>();
public constructor(options: SubscriptionMonitorConstructorOptions) {
this.config = options.config;
this.connectionManager = options.connectionManager;
this.filter = options.filter;
this.peerManager = options.peerManager;
this.libp2p = options.libp2p;
this.activeSubscriptions = options.activeSubscriptions;
this.pubsubTopic = options.pubsubTopic;
this.onConnectionChange = this.onConnectionChange.bind(this);
this.onPeerConnected = this.onPeerConnected.bind(this);
this.onPeerDisconnected = this.onPeerDisconnected.bind(this);
}
/**
* @returns content topic used for Filter verification
*/
public get reservedContentTopic(): string {
return `/js-waku-subscription-ping/1/${this.libp2p.peerId.toString()}/utf8`;
}
/**
* Starts:
* - recurring ping queries;
* - connection event observers;
*/
public start(): void {
if (this.isStarted) {
return;
}
this.isStarted = true;
this.startKeepAlive();
this.startConnectionListener();
this.startPeerConnectionListener();
}
/**
* Stops all recurring queries, event listeners or timers.
*/
public stop(): void {
if (!this.isStarted) {
return;
}
this.isStarted = false;
this.stopKeepAlive();
this.stopConnectionListener();
this.stopPeerConnectionListener();
}
/**
* Method to get peers that are used by particular subscription or, if initially called, peers that can be used by subscription.
* @returns array of peers
*/
public async getPeers(): Promise<Peer[]> {
if (!this.isStarted) {
this.peers = await this.peerManager.getPeers();
}
return this.peers;
}
/**
* Notifies monitor if message was received.
*
* @param peerId peer from which message is received
* @param message received message
*
* @returns true if message was received from peer
*/
public notifyMessageReceived(
peerId: string,
message: IProtoMessage
): boolean {
const hash = this.buildMessageHash(message);
this.verifiedPeers.add(peerId);
this.receivedMessagesFormPeer.add(`${peerId}-${hash}`);
if (this.receivedMessages.has(hash)) {
return true;
}
this.receivedMessages.add(hash);
return false;
}
private buildMessageHash(message: IProtoMessage): string {
return messageHashStr(this.pubsubTopic, message);
}
private startConnectionListener(): void {
this.connectionManager.addEventListener(
EConnectionStateEvents.CONNECTION_STATUS,
this.onConnectionChange as (v: CustomEvent<boolean>) => void
);
}
private stopConnectionListener(): void {
this.connectionManager.removeEventListener(
EConnectionStateEvents.CONNECTION_STATUS,
this.onConnectionChange as (v: CustomEvent<boolean>) => void
);
}
private async onConnectionChange({
detail: isConnected
}: CustomEvent<boolean>): Promise<void> {
if (!isConnected) {
this.stopKeepAlive();
return;
}
await Promise.all(this.peers.map((peer) => this.ping(peer, true)));
this.startKeepAlive();
}
private startKeepAlive(): void {
if (this.keepAliveIntervalId) {
return;
}
this.keepAliveIntervalId = setInterval(() => {
void this.peers.map((peer) => this.ping(peer));
}, this.config.keepAliveIntervalMs) as unknown as number;
}
private stopKeepAlive(): void {
if (!this.keepAliveIntervalId) {
return;
}
clearInterval(this.keepAliveIntervalId);
this.keepAliveIntervalId = undefined;
}
private startPeerConnectionListener(): void {
this.libp2p.addEventListener(
"peer:connect",
this.onPeerConnected as EventHandler<CustomEvent<PeerId | undefined>>
);
this.libp2p.addEventListener(
"peer:disconnect",
this.onPeerDisconnected as EventHandler<CustomEvent<PeerId | undefined>>
);
}
private stopPeerConnectionListener(): void {
this.libp2p.removeEventListener(
"peer:connect",
this.onPeerConnected as EventHandler<CustomEvent<PeerId | undefined>>
);
this.libp2p.removeEventListener(
"peer:disconnect",
this.onPeerDisconnected as EventHandler<CustomEvent<PeerId | undefined>>
);
}
// this method keeps track of new connections and will trigger subscribe request if needed
private async onPeerConnected(_event: CustomEvent<PeerId>): Promise<void> {
// TODO(weboko): use config.numOfUsedPeers instead of this.peers
const hasSomePeers = this.peers.length > 0;
if (hasSomePeers) {
return;
}
this.peers = await this.peerManager.getPeers();
await Promise.all(this.peers.map((peer) => this.subscribe(peer)));
}
// this method keeps track of disconnects and will trigger subscribe request if needed
private async onPeerDisconnected(event: CustomEvent<PeerId>): Promise<void> {
const hasNotBeenUsed = !this.peers.find((p) => p.id.equals(event.detail));
if (hasNotBeenUsed) {
return;
}
this.peers = await this.peerManager.getPeers();
// we trigger subscribe for peer that was used before
// it will expectedly fail and we will initiate addition of a new peer
await Promise.all(this.peers.map((peer) => this.subscribe(peer)));
}
private async subscribe(_peer: Peer | undefined): Promise<void> {
let peer: Peer | undefined = _peer;
for (let i = 0; i < MAX_SUBSCRIBE_ATTEMPTS; i++) {
if (!peer) {
return;
}
const response = await this.filter.subscribe(
this.pubsubTopic,
peer,
Array.from(this.activeSubscriptions.keys())
);
if (response.success) {
return;
}
peer = await this.peerManager.requestRenew(peer.id);
}
}
private async ping(
peer: Peer,
renewOnFirstFail: boolean = false
): Promise<void> {
const peerIdStr = peer.id.toString();
const response = await this.filter.ping(peer);
if (response.failure && renewOnFirstFail) {
const newPeer = await this.peerManager.requestRenew(peer.id);
await this.subscribe(newPeer);
return;
}
if (response.failure) {
const prev = this.pingFailedAttempts.get(peerIdStr) || 0;
this.pingFailedAttempts.set(peerIdStr, prev + 1);
}
if (response.success) {
this.pingFailedAttempts.set(peerIdStr, 0);
}
const madeAttempts = this.pingFailedAttempts.get(peerIdStr) || 0;
if (madeAttempts >= this.config.pingsBeforePeerRenewed) {
const newPeer = await this.peerManager.requestRenew(peer.id);
await this.subscribe(newPeer);
}
}
}

View File

@ -0,0 +1,15 @@
import { FilterProtocolOptions } from "@waku/interfaces";
import * as C from "./constants.js";
export const buildConfig = (
config?: Partial<FilterProtocolOptions>
): FilterProtocolOptions => {
return {
keepAliveIntervalMs: config?.keepAliveIntervalMs || C.DEFAULT_KEEP_ALIVE,
pingsBeforePeerRenewed:
config?.pingsBeforePeerRenewed || C.DEFAULT_MAX_PINGS,
enableLightPushFilterCheck:
config?.enableLightPushFilterCheck || C.DEFAULT_LIGHT_PUSH_FILTER_CHECK
};
};

View File

@ -14,9 +14,9 @@ export {
defaultLibp2p,
createLibp2pAndUpdateOptions
} from "./create/index.js";
export { wakuLightPush } from "./protocols/light_push/index.js";
export { wakuFilter } from "./protocols/filter/index.js";
export { wakuStore } from "./protocols/store/index.js";
export { wakuLightPush } from "./light_push/index.js";
export { wakuFilter } from "./filter/index.js";
export { wakuStore } from "./store/index.js";
export * as waku from "@waku/core";
export * as utils from "@waku/utils";

View File

@ -0,0 +1 @@
export { LightPush, wakuLightPush } from "./light_push.js";

View File

@ -10,6 +10,8 @@ import { utf8ToBytes } from "@waku/utils/bytes";
import { expect } from "chai";
import sinon from "sinon";
import { PeerManager } from "../peer_manager/index.js";
import { LightPush } from "./light_push.js";
const PUBSUB_TOPIC = "/waku/2/rs/1/4";
@ -55,8 +57,7 @@ describe("LightPush SDK", () => {
peers: [mockPeer("1"), mockPeer("2"), mockPeer("3"), mockPeer("4")]
});
// check default value that should be 2
lightPush = mockLightPush({ libp2p });
lightPush = mockLightPush({ libp2p, numPeersToUse: 2 });
let sendSpy = sinon.spy(
(_encoder: any, _message: any, peer: Peer) =>
({ success: peer.id }) as any
@ -155,10 +156,16 @@ type MockLightPushOptions = {
function mockLightPush(options: MockLightPushOptions): LightPush {
return new LightPush(
{
configuredPubsubTopics: options.pubsubTopics || [PUBSUB_TOPIC]
pubsubTopics: options.pubsubTopics || [PUBSUB_TOPIC]
} as ConnectionManager,
options.libp2p,
{ numPeersToUse: options.numPeersToUse }
{
getPeers: () =>
options.libp2p
.getPeers()
.map((id) => mockPeer(id.toString()))
.slice(0, options.numPeersToUse || options.libp2p.getPeers().length)
} as unknown as PeerManager,
options.libp2p
);
}

View File

@ -1,10 +1,5 @@
import type { Peer, PeerId } from "@libp2p/interface";
import {
ConnectionManager,
getHealthManager,
LightPushCodec,
LightPushCore
} from "@waku/core";
import { ConnectionManager, getHealthManager, LightPushCore } from "@waku/core";
import {
type CoreProtocolResult,
Failure,
@ -13,13 +8,12 @@ import {
type IMessage,
type ISenderOptions,
type Libp2p,
type ProtocolCreateOptions,
ProtocolError,
SDKProtocolResult
} from "@waku/interfaces";
import { ensurePubsubTopicIsConfigured, Logger } from "@waku/utils";
import { DEFAULT_NUM_PEERS_TO_USE } from "../base_protocol.js";
import { PeerManager } from "../peer_manager/index.js";
const log = new Logger("sdk:light-push");
@ -32,19 +26,14 @@ const DEFAULT_SEND_OPTIONS: ISenderOptions = {
type RetryCallback = (peer: Peer) => Promise<CoreProtocolResult>;
export class LightPush implements ILightPush {
private numPeersToUse: number = DEFAULT_NUM_PEERS_TO_USE;
public readonly protocol: LightPushCore;
public constructor(
connectionManager: ConnectionManager,
private libp2p: Libp2p,
options?: ProtocolCreateOptions
private peerManager: PeerManager,
libp2p: Libp2p
) {
this.numPeersToUse = options?.numPeersToUse ?? DEFAULT_NUM_PEERS_TO_USE;
this.protocol = new LightPushCore(
connectionManager.configuredPubsubTopics,
libp2p
);
this.protocol = new LightPushCore(connectionManager.pubsubTopics, libp2p);
}
public async send(
@ -70,7 +59,7 @@ export class LightPush implements ILightPush {
};
}
const peers = await this.getConnectedPeers();
const peers = await this.peerManager.getPeers();
if (peers.length === 0) {
return {
successes,
@ -128,7 +117,7 @@ export class LightPush implements ILightPush {
maxAttempts?: number
): Promise<void> {
maxAttempts = maxAttempts || DEFAULT_MAX_ATTEMPTS;
const connectedPeers = await this.getConnectedPeers();
const connectedPeers = await this.peerManager.getPeers();
if (connectedPeers.length === 0) {
log.warn("Cannot retry with no connected peers.");
@ -148,34 +137,12 @@ export class LightPush implements ILightPush {
);
}
}
private async getConnectedPeers(): Promise<Peer[]> {
const peerIDs = this.libp2p.getPeers();
if (peerIDs.length === 0) {
return [];
}
const peers = await Promise.all(
peerIDs.map(async (id) => {
try {
return await this.libp2p.peerStore.get(id);
} catch (e) {
return null;
}
})
);
return peers
.filter((p) => !!p)
.filter((p) => (p as Peer).protocols.includes(LightPushCodec))
.slice(0, this.numPeersToUse) as Peer[];
}
}
export function wakuLightPush(
connectionManager: ConnectionManager,
init: Partial<ProtocolCreateOptions> = {}
peerManager: PeerManager
): (libp2p: Libp2p) => ILightPush {
return (libp2p: Libp2p) => new LightPush(connectionManager, libp2p, init);
return (libp2p: Libp2p) =>
new LightPush(connectionManager, peerManager, libp2p);
}

View File

@ -0,0 +1 @@
export { PeerManager } from "./peer_manager.js";

View File

@ -0,0 +1,109 @@
import { Connection, Peer, PeerId } from "@libp2p/interface";
import { Libp2p } from "@waku/interfaces";
import { expect } from "chai";
import sinon from "sinon";
import { PeerManager } from "./peer_manager.js";
describe("PeerManager", () => {
let libp2p: Libp2p;
let peerManager: PeerManager;
beforeEach(() => {
libp2p = mockLibp2p();
peerManager = new PeerManager({ libp2p });
});
afterEach(() => {
peerManager.stop();
sinon.restore();
});
it("should initialize with default number of peers", () => {
expect(peerManager["numPeersToUse"]).to.equal(2);
});
it("should initialize with custom number of peers", () => {
peerManager = new PeerManager({ libp2p, config: { numPeersToUse: 3 } });
expect(peerManager["numPeersToUse"]).to.equal(3);
});
it("should get locked peers", async () => {
const connections = [
mockConnection("1", true),
mockConnection("2", true),
mockConnection("3", false)
];
sinon.stub(libp2p, "getConnections").returns(connections);
const peers = await peerManager.getPeers();
expect(peers.length).to.equal(2);
});
it("should request renew when peer disconnects", async () => {
const connections = [
mockConnection("1", true),
mockConnection("2", false),
mockConnection("3", false)
];
sinon.stub(libp2p, "getConnections").returns(connections);
const peer = await peerManager.requestRenew("1");
expect(peer).to.not.be.undefined;
expect(peer?.id).to.not.equal("1");
});
it("should handle connection events", () => {
const connectSpy = sinon.spy(peerManager["lockPeerIfNeeded"]);
const disconnectSpy = sinon.spy(peerManager["requestRenew"]);
peerManager["lockPeerIfNeeded"] = connectSpy;
peerManager["requestRenew"] = disconnectSpy;
libp2p.dispatchEvent(new CustomEvent("peer:connect", { detail: "1" }));
libp2p.dispatchEvent(new CustomEvent("peer:disconnect", { detail: "1" }));
expect(connectSpy.calledOnce).to.be.true;
expect(disconnectSpy.calledOnce).to.be.true;
});
});
function mockLibp2p(): Libp2p {
const peerStore = {
get: (id: any) => Promise.resolve(mockPeer(id.toString()))
};
const events = new EventTarget();
return {
peerStore,
addEventListener: (event: string, handler: EventListener) =>
events.addEventListener(event, handler),
removeEventListener: (event: string, handler: EventListener) =>
events.removeEventListener(event, handler),
dispatchEvent: (event: Event) => events.dispatchEvent(event),
getConnections: () => [],
components: {
events,
peerStore
}
} as unknown as Libp2p;
}
function mockPeer(id: string): Peer {
return {
id,
protocols: []
} as unknown as Peer;
}
function mockConnection(id: string, locked: boolean): Connection {
return {
remotePeer: {
toString: () => id,
equals: (other: string | PeerId) =>
(typeof other === "string" ? other.toString() : other) === id
},
status: "open",
tags: locked ? ["peer-manager-lock"] : []
} as unknown as Connection;
}

View File

@ -0,0 +1,141 @@
import { Connection, Peer, PeerId } from "@libp2p/interface";
import { Libp2p } from "@waku/interfaces";
import { Logger } from "@waku/utils";
const log = new Logger("peer-manager");
const DEFAULT_NUM_PEERS_TO_USE = 2;
const CONNECTION_LOCK_TAG = "peer-manager-lock";
type PeerManagerConfig = {
numPeersToUse?: number;
};
type PeerManagerParams = {
libp2p: Libp2p;
config?: PeerManagerConfig;
};
export class PeerManager {
private readonly numPeersToUse: number;
private readonly libp2p: Libp2p;
public constructor(params: PeerManagerParams) {
this.onConnected = this.onConnected.bind(this);
this.onDisconnected = this.onDisconnected.bind(this);
this.numPeersToUse =
params?.config?.numPeersToUse || DEFAULT_NUM_PEERS_TO_USE;
this.libp2p = params.libp2p;
this.startConnectionListener();
}
public stop(): void {
this.stopConnectionListener();
}
public async getPeers(): Promise<Peer[]> {
return Promise.all(
this.getLockedConnections().map((c) => this.mapConnectionToPeer(c))
);
}
public async requestRenew(
peerId: PeerId | string
): Promise<Peer | undefined> {
const lockedConnections = this.getLockedConnections();
const neededPeers = this.numPeersToUse - lockedConnections.length;
if (neededPeers === 0) {
return;
}
const result = await Promise.all(
this.getUnlockedConnections()
.filter((c) => !c.remotePeer.equals(peerId))
.slice(0, neededPeers)
.map((c) => this.lockConnection(c))
.map((c) => this.mapConnectionToPeer(c))
);
const newPeer = result[0];
if (!newPeer) {
log.warn(
`requestRenew: Couldn't renew peer ${peerId.toString()} - no peers.`
);
return;
}
log.info(
`requestRenew: Renewed peer ${peerId.toString()} to ${newPeer.id.toString()}`
);
return newPeer;
}
private startConnectionListener(): void {
this.libp2p.addEventListener("peer:connect", this.onConnected);
this.libp2p.addEventListener("peer:disconnect", this.onDisconnected);
}
private stopConnectionListener(): void {
this.libp2p.removeEventListener("peer:connect", this.onConnected);
this.libp2p.removeEventListener("peer:disconnect", this.onDisconnected);
}
private onConnected(event: CustomEvent<PeerId>): void {
const peerId = event.detail;
void this.lockPeerIfNeeded(peerId);
}
private onDisconnected(event: CustomEvent<PeerId>): void {
const peerId = event.detail;
void this.requestRenew(peerId);
}
private async lockPeerIfNeeded(peerId: PeerId): Promise<void> {
const lockedConnections = this.getLockedConnections();
const neededPeers = this.numPeersToUse - lockedConnections.length;
if (neededPeers === 0) {
return;
}
this.getUnlockedConnections()
.filter((c) => c.remotePeer.equals(peerId))
.map((c) => this.lockConnection(c));
}
private getLockedConnections(): Connection[] {
return this.libp2p
.getConnections()
.filter((c) => c.status === "open" && this.isConnectionLocked(c));
}
private getUnlockedConnections(): Connection[] {
return this.libp2p
.getConnections()
.filter((c) => c.status === "open" && !this.isConnectionLocked(c));
}
private lockConnection(c: Connection): Connection {
log.info(
`requestRenew: Locking connection for peerId=${c.remotePeer.toString()}`
);
c.tags.push(CONNECTION_LOCK_TAG);
return c;
}
private isConnectionLocked(c: Connection): boolean {
return c.tags.includes(CONNECTION_LOCK_TAG);
}
private async mapConnectionToPeer(c: Connection): Promise<Peer> {
const peerId = c.remotePeer;
return this.libp2p.peerStore.get(peerId);
}
}

View File

@ -1,191 +0,0 @@
import type { Peer, PeerId } from "@libp2p/interface";
import { ConnectionManager } from "@waku/core";
import { BaseProtocol } from "@waku/core/lib/base_protocol";
import { IBaseProtocolSDK, ProtocolUseOptions } from "@waku/interfaces";
import { Logger } from "@waku/utils";
import { PeerManager } from "./peer_manager.js";
interface Options {
numPeersToUse?: number;
maintainPeersInterval?: number;
}
export const DEFAULT_NUM_PEERS_TO_USE = 2;
const DEFAULT_MAINTAIN_PEERS_INTERVAL = 30_000;
export class BaseProtocolSDK implements IBaseProtocolSDK {
private peerManager: PeerManager;
public readonly numPeersToUse: number;
private maintainPeersIntervalId: ReturnType<
typeof window.setInterval
> | null = null;
private log: Logger;
public constructor(
protected core: BaseProtocol,
protected connectionManager: ConnectionManager,
options: Options
) {
this.log = new Logger(`sdk:${core.multicodec}`);
this.numPeersToUse = options?.numPeersToUse ?? DEFAULT_NUM_PEERS_TO_USE;
const maintainPeersInterval =
options?.maintainPeersInterval ?? DEFAULT_MAINTAIN_PEERS_INTERVAL;
this.peerManager = new PeerManager(connectionManager, core, this.log);
this.log.info(
`Initializing BaseProtocolSDK with numPeersToUse: ${this.numPeersToUse}, maintainPeersInterval: ${maintainPeersInterval}ms`
);
void this.startMaintainPeersInterval(maintainPeersInterval);
}
public get connectedPeers(): Peer[] {
return this.peerManager.getPeers().slice(0, this.numPeersToUse);
}
/**
* Disconnects from a peer and tries to find a new one to replace it.
* @param peerToDisconnect The peer to disconnect from.
* @returns The new peer that was found and connected to.
*/
public async renewPeer(peerToDisconnect: PeerId): Promise<Peer | undefined> {
this.log.info(`Attempting to renew peer ${peerToDisconnect}`);
const newPeer = await this.peerManager.findPeers(1);
if (newPeer.length === 0) {
this.log.error(
"Failed to find a new peer to replace the disconnected one"
);
return undefined;
}
await this.peerManager.removePeer(peerToDisconnect);
await this.peerManager.addPeer(newPeer[0]);
this.log.info(`Successfully renewed peer. New peer: ${newPeer[0].id}`);
return newPeer[0];
}
/**
* Stops the maintain peers interval.
*/
public stopMaintainPeersInterval(): void {
if (this.maintainPeersIntervalId) {
clearInterval(this.maintainPeersIntervalId);
this.maintainPeersIntervalId = null;
this.log.info("Maintain peers interval stopped");
} else {
this.log.info("Maintain peers interval was not running");
}
}
/**
* Checks if there are sufficient peers to send a message to.
* If `forceUseAllPeers` is `false` (default), returns `true` if there are any connected peers.
* If `forceUseAllPeers` is `true`, attempts to connect to `numPeersToUse` peers.
* @param options Optional options object
* @param options.forceUseAllPeers Optional flag to force connecting to `numPeersToUse` peers (default: false)
* @param options.maxAttempts Optional maximum number of attempts to reach the required number of peers (default: 3)
* @returns `true` if the required number of peers are connected, `false` otherwise
*/
protected async hasPeers(
options: Partial<ProtocolUseOptions> = {}
): Promise<boolean> {
const { forceUseAllPeers = false, maxAttempts = 3 } = options;
this.log.info(
`Checking for peers. forceUseAllPeers: ${forceUseAllPeers}, maxAttempts: ${maxAttempts}`
);
for (let attempts = 0; attempts < maxAttempts; attempts++) {
this.log.info(
`Attempt ${attempts + 1}/${maxAttempts} to reach required number of peers`
);
await this.maintainPeers();
if (!forceUseAllPeers && this.connectedPeers.length > 0) {
this.log.info(
`At least one peer connected (${this.connectedPeers.length}), not forcing use of all peers`
);
return true;
}
if (this.connectedPeers.length >= this.numPeersToUse) {
this.log.info(
`Required number of peers (${this.numPeersToUse}) reached`
);
return true;
}
this.log.warn(
`Found only ${this.connectedPeers.length}/${this.numPeersToUse} required peers. Retrying...`
);
}
this.log.error(
`Failed to find required number of peers (${this.numPeersToUse}) after ${maxAttempts} attempts`
);
return false;
}
/**
* Starts an interval to maintain the peers list to `numPeersToUse`.
* @param interval The interval in milliseconds to maintain the peers.
*/
private async startMaintainPeersInterval(interval: number): Promise<void> {
this.log.info(
`Starting maintain peers interval with ${interval}ms interval`
);
try {
this.maintainPeersIntervalId = setInterval(() => {
this.log.info("Running scheduled peer maintenance");
this.maintainPeers().catch((error) => {
this.log.error("Error during scheduled peer maintenance:", error);
});
}, interval);
this.log.info("Maintain peers interval started successfully");
} catch (error) {
this.log.error("Error starting maintain peers interval:", error);
throw error;
}
}
/**
* Maintains the peers list to `numPeersToUse`.
*/
private async maintainPeers(): Promise<void> {
try {
const currentPeerCount = await this.peerManager.getPeerCount();
const numPeersToAdd = this.numPeersToUse - currentPeerCount;
this.log.info(
`Current peer count: ${currentPeerCount}, target: ${this.numPeersToUse}`
);
if (numPeersToAdd === 0) {
this.log.info("Peer count is at target, no maintenance required");
return;
}
if (numPeersToAdd > 0) {
this.log.info(`Attempting to add ${numPeersToAdd} peer(s)`);
await this.peerManager.findAndAddPeers(numPeersToAdd);
} else {
this.log.info(
`Attempting to remove ${Math.abs(numPeersToAdd)} excess peer(s)`
);
await this.peerManager.removeExcessPeers(Math.abs(numPeersToAdd));
}
const finalPeerCount = await this.peerManager.getPeerCount();
this.log.info(
`Peer maintenance completed. Initial count: ${currentPeerCount}, Final count: ${finalPeerCount}`
);
} catch (error) {
this.log.error("Error during peer maintenance", { error });
}
}
}

View File

@ -1,8 +0,0 @@
export const DEFAULT_KEEP_ALIVE = 60_000;
export const DEFAULT_LIGHT_PUSH_FILTER_CHECK = false;
export const DEFAULT_LIGHT_PUSH_FILTER_CHECK_INTERVAL = 10_000;
export const DEFAULT_SUBSCRIBE_OPTIONS = {
keepAlive: DEFAULT_KEEP_ALIVE,
enableLightPushFilterCheck: DEFAULT_LIGHT_PUSH_FILTER_CHECK
};

View File

@ -1,438 +0,0 @@
import type { Peer } from "@libp2p/interface";
import type { PeerId } from "@libp2p/interface";
import {
ConnectionManager,
createDecoder,
createEncoder,
FilterCore,
LightPushCore
} from "@waku/core";
import {
type Callback,
type ContentTopic,
type CoreProtocolResult,
EConnectionStateEvents,
type IDecodedMessage,
type IDecoder,
type ILightPush,
type IProtoMessage,
type ISubscription,
type Libp2p,
type PeerIdStr,
ProtocolError,
type PubsubTopic,
type SDKProtocolResult,
type SubscribeOptions,
SubscriptionCallback
} from "@waku/interfaces";
import { WakuMessage } from "@waku/proto";
import { groupByContentTopic, Logger } from "@waku/utils";
import { ReliabilityMonitorManager } from "../../reliability_monitor/index.js";
import { ReceiverReliabilityMonitor } from "../../reliability_monitor/receiver.js";
import {
DEFAULT_KEEP_ALIVE,
DEFAULT_LIGHT_PUSH_FILTER_CHECK,
DEFAULT_LIGHT_PUSH_FILTER_CHECK_INTERVAL,
DEFAULT_SUBSCRIBE_OPTIONS
} from "./constants.js";
const log = new Logger("sdk:filter:subscription_manager");
export class SubscriptionManager implements ISubscription {
private reliabilityMonitor: ReceiverReliabilityMonitor;
private keepAliveTimeout: number = DEFAULT_KEEP_ALIVE;
private keepAliveInterval: ReturnType<typeof setInterval> | null = null;
private enableLightPushFilterCheck = DEFAULT_LIGHT_PUSH_FILTER_CHECK;
private subscriptionCallbacks: Map<
ContentTopic,
SubscriptionCallback<IDecodedMessage>
>;
public constructor(
private readonly pubsubTopic: PubsubTopic,
private readonly protocol: FilterCore,
private readonly connectionManager: ConnectionManager,
private readonly getPeers: () => Peer[],
private readonly renewPeer: (
peerToDisconnect: PeerId
) => Promise<Peer | undefined>,
private readonly libp2p: Libp2p,
private readonly lightPush?: ILightPush
) {
this.pubsubTopic = pubsubTopic;
this.subscriptionCallbacks = new Map();
this.reliabilityMonitor = ReliabilityMonitorManager.createReceiverMonitor(
this.pubsubTopic,
this.getPeers.bind(this),
this.renewPeer.bind(this),
() => Array.from(this.subscriptionCallbacks.keys()),
this.protocol.subscribe.bind(this.protocol),
this.protocol.addLibp2pEventListener.bind(this.protocol),
this.sendLightPushCheckMessage.bind(this)
);
}
public async subscribe<T extends IDecodedMessage>(
decoders: IDecoder<T> | IDecoder<T>[],
callback: Callback<T>,
options: SubscribeOptions = DEFAULT_SUBSCRIBE_OPTIONS
): Promise<SDKProtocolResult> {
this.reliabilityMonitor.setMaxPingFailures(options.pingsBeforePeerRenewed);
this.keepAliveTimeout = options.keepAlive || DEFAULT_KEEP_ALIVE;
this.enableLightPushFilterCheck =
options?.enableLightPushFilterCheck || DEFAULT_LIGHT_PUSH_FILTER_CHECK;
const decodersArray = Array.isArray(decoders) ? decoders : [decoders];
// check that all decoders are configured for the same pubsub topic as this subscription
for (const decoder of decodersArray) {
if (decoder.pubsubTopic !== this.pubsubTopic) {
return {
failures: [
{
error: ProtocolError.TOPIC_DECODER_MISMATCH
}
],
successes: []
};
}
}
if (this.enableLightPushFilterCheck) {
decodersArray.push(
createDecoder(
this.buildLightPushContentTopic(),
this.pubsubTopic
) as IDecoder<T>
);
}
const decodersGroupedByCT = groupByContentTopic(decodersArray);
const contentTopics = Array.from(decodersGroupedByCT.keys());
const promises = this.getPeers().map(async (peer) =>
this.subscribeWithPeerVerification(peer, contentTopics)
);
const results = await Promise.allSettled(promises);
const finalResult = this.handleResult(results, "subscribe");
// Save the callback functions by content topics so they
// can easily be removed (reciprocally replaced) if `unsubscribe` (reciprocally `subscribe`)
// is called for those content topics
decodersGroupedByCT.forEach((decoders, contentTopic) => {
// Cast the type because a given `subscriptionCallbacks` map may hold
// Decoder that decode to different implementations of `IDecodedMessage`
const subscriptionCallback = {
decoders,
callback
} as unknown as SubscriptionCallback<IDecodedMessage>;
// don't handle case of internal content topic
if (contentTopic === this.buildLightPushContentTopic()) {
return;
}
// The callback and decoder may override previous values, this is on
// purpose as the user may call `subscribe` to refresh the subscription
this.subscriptionCallbacks.set(contentTopic, subscriptionCallback);
});
this.startSubscriptionsMaintenance(this.keepAliveTimeout);
return finalResult;
}
public async unsubscribe(
contentTopics: ContentTopic[]
): Promise<SDKProtocolResult> {
const promises = this.getPeers().map(async (peer) => {
const response = await this.protocol.unsubscribe(
this.pubsubTopic,
peer,
contentTopics
);
contentTopics.forEach((contentTopic: string) => {
this.subscriptionCallbacks.delete(contentTopic);
});
return response;
});
const results = await Promise.allSettled(promises);
const finalResult = this.handleResult(results, "unsubscribe");
if (this.subscriptionCallbacks.size === 0) {
this.stopSubscriptionsMaintenance();
}
return finalResult;
}
public async ping(peerId?: PeerId): Promise<SDKProtocolResult> {
log.info("Sending keep-alive ping");
const peers = peerId ? [peerId] : this.getPeers().map((peer) => peer.id);
const promises = peers.map((peerId) => this.pingSpecificPeer(peerId));
const results = await Promise.allSettled(promises);
return this.handleResult(results, "ping");
}
public async unsubscribeAll(): Promise<SDKProtocolResult> {
const promises = this.getPeers().map(async (peer) =>
this.protocol.unsubscribeAll(this.pubsubTopic, peer)
);
const results = await Promise.allSettled(promises);
this.subscriptionCallbacks.clear();
const finalResult = this.handleResult(results, "unsubscribeAll");
this.stopSubscriptionsMaintenance();
return finalResult;
}
public async processIncomingMessage(
message: WakuMessage,
peerIdStr: PeerIdStr
): Promise<void> {
const alreadyReceived = this.reliabilityMonitor.notifyMessageReceived(
peerIdStr,
message as IProtoMessage
);
if (alreadyReceived) {
log.info("Message already received, skipping");
return;
}
const { contentTopic } = message;
const subscriptionCallback = this.subscriptionCallbacks.get(contentTopic);
if (!subscriptionCallback) {
log.error("No subscription callback available for ", contentTopic);
return;
}
log.info(
"Processing message with content topic ",
contentTopic,
" on pubsub topic ",
this.pubsubTopic
);
await pushMessage(subscriptionCallback, this.pubsubTopic, message);
}
private async subscribeWithPeerVerification(
peer: Peer,
contentTopics: string[]
): Promise<CoreProtocolResult> {
const result = await this.protocol.subscribe(
this.pubsubTopic,
peer,
contentTopics
);
await this.sendLightPushCheckMessage(peer);
return result;
}
private handleResult(
results: PromiseSettledResult<CoreProtocolResult>[],
type: "ping" | "subscribe" | "unsubscribe" | "unsubscribeAll"
): SDKProtocolResult {
const result: SDKProtocolResult = { failures: [], successes: [] };
for (const promiseResult of results) {
if (promiseResult.status === "rejected") {
log.error(
`Failed to resolve ${type} promise successfully: `,
promiseResult.reason
);
result.failures.push({ error: ProtocolError.GENERIC_FAIL });
} else {
const coreResult = promiseResult.value;
if (coreResult.failure) {
result.failures.push(coreResult.failure);
} else {
result.successes.push(coreResult.success);
}
}
}
return result;
}
private async pingSpecificPeer(peerId: PeerId): Promise<CoreProtocolResult> {
const peer = this.getPeers().find((p) => p.id.equals(peerId));
if (!peer) {
return {
success: null,
failure: {
peerId,
error: ProtocolError.NO_PEER_AVAILABLE
}
};
}
let result;
try {
result = await this.protocol.ping(peer);
} catch (error) {
result = {
success: null,
failure: {
peerId,
error: ProtocolError.GENERIC_FAIL
}
};
}
log.info(
`Received result from filter ping peerId:${peerId.toString()}\tsuccess:${result.success?.toString()}\tfailure:${result.failure?.error}`
);
await this.reliabilityMonitor.handlePingResult(peerId, result);
return result;
}
private startSubscriptionsMaintenance(timeout: number): void {
log.info("Starting subscriptions maintenance");
this.startKeepAlivePings(timeout);
this.startConnectionListener();
}
private stopSubscriptionsMaintenance(): void {
log.info("Stopping subscriptions maintenance");
this.stopKeepAlivePings();
this.stopConnectionListener();
}
private startConnectionListener(): void {
this.connectionManager.addEventListener(
EConnectionStateEvents.CONNECTION_STATUS,
this.connectionListener.bind(this) as (v: CustomEvent<boolean>) => void
);
}
private stopConnectionListener(): void {
this.connectionManager.removeEventListener(
EConnectionStateEvents.CONNECTION_STATUS,
this.connectionListener.bind(this) as (v: CustomEvent<boolean>) => void
);
}
private async connectionListener({
detail: isConnected
}: CustomEvent<boolean>): Promise<void> {
if (!isConnected) {
this.stopKeepAlivePings();
return;
}
try {
// we do nothing here, as the renewal process is managed internally by `this.ping()`
await this.ping();
} catch (err) {
log.error(`networkStateListener failed to recover: ${err}`);
}
this.startKeepAlivePings(this.keepAliveTimeout);
}
private startKeepAlivePings(timeout: number): void {
if (this.keepAliveInterval) {
log.info("Recurring pings already set up.");
return;
}
this.keepAliveInterval = setInterval(() => {
void this.ping();
}, timeout);
}
private stopKeepAlivePings(): void {
if (!this.keepAliveInterval) {
log.info("Already stopped recurring pings.");
return;
}
log.info("Stopping recurring pings.");
clearInterval(this.keepAliveInterval);
this.keepAliveInterval = null;
}
private async sendLightPushCheckMessage(peer: Peer): Promise<void> {
if (
this.lightPush &&
this.libp2p &&
this.reliabilityMonitor.shouldVerifyPeer(peer.id)
) {
const encoder = createEncoder({
contentTopic: this.buildLightPushContentTopic(),
pubsubTopic: this.pubsubTopic,
ephemeral: true
});
const message = { payload: new Uint8Array(1) };
const protoMessage = await encoder.toProtoObj(message);
// make a delay to be sure message is send when subscription is in place
setTimeout(
(async () => {
const result = await (this.lightPush!.protocol as LightPushCore).send(
encoder,
message,
peer
);
this.reliabilityMonitor.notifyMessageSent(peer.id, protoMessage);
if (result.failure) {
log.error(
`failed to send lightPush ping message to peer:${peer.id.toString()}\t${result.failure.error}`
);
return;
}
}) as () => void,
DEFAULT_LIGHT_PUSH_FILTER_CHECK_INTERVAL
);
}
}
private buildLightPushContentTopic(): string {
return `/js-waku-subscription-ping/1/${this.libp2p.peerId.toString()}/utf8`;
}
}
async function pushMessage<T extends IDecodedMessage>(
subscriptionCallback: SubscriptionCallback<T>,
pubsubTopic: PubsubTopic,
message: WakuMessage
): Promise<void> {
const { decoders, callback } = subscriptionCallback;
const { contentTopic } = message;
if (!contentTopic) {
log.warn("Message has no content topic, skipping");
return;
}
try {
const decodePromises = decoders.map((dec) =>
dec
.fromProtoObj(pubsubTopic, message as IProtoMessage)
.then((decoded) => decoded || Promise.reject("Decoding failed"))
);
const decodedMessage = await Promise.any(decodePromises);
await callback(decodedMessage);
} catch (e) {
log.error("Error decoding message", e);
}
}

View File

@ -1 +0,0 @@
export { wakuLightPush } from "./light_push.js";

View File

@ -1,148 +0,0 @@
import { Peer, PeerId } from "@libp2p/interface";
import { ConnectionManager, LightPushCodec } from "@waku/core";
import { BaseProtocol } from "@waku/core/lib/base_protocol";
import { Logger } from "@waku/utils";
import { expect } from "chai";
import sinon from "sinon";
import { PeerManager } from "./peer_manager.js";
describe("PeerManager", () => {
let peerManager: PeerManager;
let mockConnectionManager: sinon.SinonStubbedInstance<ConnectionManager>;
let mockCore: sinon.SinonStubbedInstance<BaseProtocol>;
let mockLogger: any;
beforeEach(() => {
mockConnectionManager = sinon.createStubInstance(ConnectionManager);
mockCore = sinon.createStubInstance(BaseProtocol);
mockLogger = {
info: sinon.stub(),
warn: sinon.stub(),
error: sinon.stub(),
debug: sinon.stub(),
extend: sinon.stub().returns({
info: sinon.stub(),
warn: sinon.stub(),
error: sinon.stub(),
debug: sinon.stub()
})
};
mockCore.multicodec = LightPushCodec;
peerManager = new PeerManager(
mockConnectionManager as any,
mockCore as any,
mockLogger as Logger
);
});
afterEach(() => {
sinon.restore();
});
const createMockPeer = (id: string): Peer =>
({
id: {
toString: () => id
} as PeerId
}) as Peer;
describe("addPeer", () => {
it("should add a peer", async () => {
const peer = createMockPeer("peer1");
await peerManager.addPeer(peer);
expect(mockConnectionManager.attemptDial.calledWith(peer.id)).to.be.true;
expect(
mockLogger.info.calledWith(sinon.match(/Added and dialed peer: peer1/))
).to.be.true;
expect(await peerManager.getPeerCount()).to.equal(1);
});
});
describe("removePeer", () => {
it("should remove a peer", async () => {
const peer = createMockPeer("peer1");
await peerManager.addPeer(peer);
await peerManager.removePeer(peer.id);
expect(mockLogger.info.calledWith(sinon.match(/Removed peer: peer1/))).to
.be.true;
expect(await peerManager.getPeerCount()).to.equal(0);
});
});
describe("getPeerCount", () => {
it("should return the correct number of peers", async () => {
await peerManager.addPeer(createMockPeer("peer1"));
await peerManager.addPeer(createMockPeer("peer2"));
const count = await peerManager.getPeerCount();
expect(count).to.equal(2);
});
});
describe("hasPeers", () => {
it("should return true when peers exist", async () => {
await peerManager.addPeer(createMockPeer("peer1"));
const result = await peerManager.hasPeers();
expect(result).to.be.true;
});
it("should return false when no peers exist", async () => {
const result = await peerManager.hasPeers();
expect(result).to.be.false;
});
});
describe("removeExcessPeers", () => {
it("should remove the specified number of excess peers", async () => {
await peerManager.addPeer(createMockPeer("peer1"));
await peerManager.addPeer(createMockPeer("peer2"));
await peerManager.addPeer(createMockPeer("peer3"));
await peerManager.removeExcessPeers(2);
const count = await peerManager.getPeerCount();
expect(count).to.equal(1);
expect(mockLogger.info.calledWith(`Removing 2 excess peer(s)`)).to.be
.true;
});
});
describe("findAndAddPeers", () => {
it("should find and add new peers", async () => {
const newPeers = [createMockPeer("peer1"), createMockPeer("peer2")];
mockCore.getPeers.resolves(newPeers);
const addedPeers = await peerManager.findAndAddPeers(2);
expect(addedPeers).to.have.lengthOf(2);
expect(mockConnectionManager.attemptDial.callCount).to.equal(2);
});
it("should not add existing peers", async () => {
const existingPeer = createMockPeer("existing");
await peerManager.addPeer(existingPeer);
const newPeers = [existingPeer, createMockPeer("new")];
mockCore.getPeers.resolves(newPeers);
const addedPeers = await peerManager.findAndAddPeers(2);
expect(addedPeers).to.have.lengthOf(1);
expect(mockConnectionManager.attemptDial.callCount).to.equal(2); // Once for existing, once for new
});
it("should log when no additional peers are found", async () => {
mockCore.getPeers.resolves([]);
await peerManager.findAndAddPeers(2);
expect(mockLogger.warn.calledWith("No additional peers found")).to.be
.true;
});
});
});

View File

@ -1,113 +0,0 @@
import { Peer, PeerId } from "@libp2p/interface";
import { ConnectionManager, getHealthManager } from "@waku/core";
import { BaseProtocol } from "@waku/core/lib/base_protocol";
import { IHealthManager } from "@waku/interfaces";
import { Logger } from "@waku/utils";
import { Mutex } from "async-mutex";
export class PeerManager {
private peers: Map<string, Peer> = new Map();
private healthManager: IHealthManager;
private readMutex = new Mutex();
private writeMutex = new Mutex();
private writeLockHolder: string | null = null;
public constructor(
private readonly connectionManager: ConnectionManager,
private readonly core: BaseProtocol,
private readonly log: Logger
) {
this.healthManager = getHealthManager();
this.healthManager.updateProtocolHealth(this.core.multicodec, 0);
}
public getWriteLockHolder(): string | null {
return this.writeLockHolder;
}
public getPeers(): Peer[] {
return Array.from(this.peers.values());
}
public async addPeer(peer: Peer): Promise<void> {
return this.writeMutex.runExclusive(async () => {
this.writeLockHolder = `addPeer: ${peer.id.toString()}`;
await this.connectionManager.attemptDial(peer.id);
this.peers.set(peer.id.toString(), peer);
this.log.info(`Added and dialed peer: ${peer.id.toString()}`);
this.healthManager.updateProtocolHealth(
this.core.multicodec,
this.peers.size
);
this.writeLockHolder = null;
});
}
public async removePeer(peerId: PeerId): Promise<void> {
return this.writeMutex.runExclusive(() => {
this.writeLockHolder = `removePeer: ${peerId.toString()}`;
this.peers.delete(peerId.toString());
this.log.info(`Removed peer: ${peerId.toString()}`);
this.healthManager.updateProtocolHealth(
this.core.multicodec,
this.peers.size
);
this.writeLockHolder = null;
});
}
public async getPeerCount(): Promise<number> {
return this.readMutex.runExclusive(() => this.peers.size);
}
public async hasPeers(): Promise<boolean> {
return this.readMutex.runExclusive(() => this.peers.size > 0);
}
public async removeExcessPeers(excessPeers: number): Promise<void> {
this.log.info(`Removing ${excessPeers} excess peer(s)`);
const peersToRemove = Array.from(this.peers.values()).slice(0, excessPeers);
for (const peer of peersToRemove) {
await this.removePeer(peer.id);
}
}
/**
* Finds and adds new peers to the peers list.
* @param numPeers The number of peers to find and add.
*/
public async findAndAddPeers(numPeers: number): Promise<Peer[]> {
const additionalPeers = await this.findPeers(numPeers);
if (additionalPeers.length === 0) {
this.log.warn("No additional peers found");
return [];
}
return this.addMultiplePeers(additionalPeers);
}
/**
* Finds additional peers.
* @param numPeers The number of peers to find.
*/
public async findPeers(numPeers: number): Promise<Peer[]> {
const connectedPeers = await this.core.getPeers();
return this.readMutex.runExclusive(async () => {
const newPeers = connectedPeers
.filter((peer) => !this.peers.has(peer.id.toString()))
.slice(0, numPeers);
return newPeers;
});
}
public async addMultiplePeers(peers: Peer[]): Promise<Peer[]> {
const addedPeers: Peer[] = [];
for (const peer of peers) {
await this.addPeer(peer);
addedPeers.push(peer);
}
return addedPeers;
}
}

View File

@ -1,59 +0,0 @@
import type { Peer, PeerId } from "@libp2p/interface";
import {
ContentTopic,
CoreProtocolResult,
Libp2p,
PubsubTopic
} from "@waku/interfaces";
import { ReceiverReliabilityMonitor } from "./receiver.js";
export class ReliabilityMonitorManager {
private static receiverMonitors: Map<
PubsubTopic,
ReceiverReliabilityMonitor
> = new Map();
public static createReceiverMonitor(
pubsubTopic: PubsubTopic,
getPeers: () => Peer[],
renewPeer: (peerId: PeerId) => Promise<Peer | undefined>,
getContentTopics: () => ContentTopic[],
protocolSubscribe: (
pubsubTopic: PubsubTopic,
peer: Peer,
contentTopics: ContentTopic[]
) => Promise<CoreProtocolResult>,
addLibp2pEventListener: Libp2p["addEventListener"],
sendLightPushMessage: (peer: Peer) => Promise<void>
): ReceiverReliabilityMonitor {
if (ReliabilityMonitorManager.receiverMonitors.has(pubsubTopic)) {
return ReliabilityMonitorManager.receiverMonitors.get(pubsubTopic)!;
}
const monitor = new ReceiverReliabilityMonitor(
pubsubTopic,
getPeers,
renewPeer,
getContentTopics,
protocolSubscribe,
addLibp2pEventListener,
sendLightPushMessage
);
ReliabilityMonitorManager.receiverMonitors.set(pubsubTopic, monitor);
return monitor;
}
private constructor() {}
public static stop(pubsubTopic: PubsubTopic): void {
this.receiverMonitors.delete(pubsubTopic);
}
public static stopAll(): void {
for (const [pubsubTopic, monitor] of this.receiverMonitors) {
monitor.setMaxPingFailures(undefined);
this.receiverMonitors.delete(pubsubTopic);
}
}
}

View File

@ -1,193 +0,0 @@
import type { Peer, PeerId } from "@libp2p/interface";
import {
ContentTopic,
CoreProtocolResult,
IProtoMessage,
Libp2p,
PeerIdStr,
PubsubTopic
} from "@waku/interfaces";
import { messageHashStr } from "@waku/message-hash";
import { Logger } from "@waku/utils";
import { bytesToUtf8 } from "@waku/utils/bytes";
const log = new Logger("sdk:receiver:reliability_monitor");
const DEFAULT_MAX_PINGS = 3;
const MESSAGE_VERIFICATION_DELAY = 5_000;
export class ReceiverReliabilityMonitor {
private receivedMessagesFormPeer = new Set<string>();
private receivedMessages = new Set<string>();
private scheduledVerification = new Map<string, number>();
private verifiedPeers = new Set<string>();
private peerFailures: Map<string, number> = new Map();
private maxPingFailures: number = DEFAULT_MAX_PINGS;
private peerRenewalLocks: Set<PeerIdStr> = new Set();
public constructor(
private readonly pubsubTopic: PubsubTopic,
private getPeers: () => Peer[],
private renewPeer: (peerId: PeerId) => Promise<Peer | undefined>,
private getContentTopics: () => ContentTopic[],
private protocolSubscribe: (
pubsubTopic: PubsubTopic,
peer: Peer,
contentTopics: ContentTopic[]
) => Promise<CoreProtocolResult>,
private addLibp2pEventListener: Libp2p["addEventListener"],
private sendLightPushMessage: (peer: Peer) => Promise<void>
) {
this.addLibp2pEventListener("peer:disconnect", (evt) => {
const peerId = evt.detail;
if (this.getPeers().some((p) => p.id.equals(peerId))) {
void this.renewAndSubscribePeer(peerId);
}
});
}
public setMaxPingFailures(value: number | undefined): void {
if (value === undefined) {
return;
}
this.maxPingFailures = value;
}
public async handlePingResult(
peerId: PeerId,
result?: CoreProtocolResult
): Promise<void> {
if (result?.success) {
this.peerFailures.delete(peerId.toString());
return;
}
const failures = (this.peerFailures.get(peerId.toString()) || 0) + 1;
this.peerFailures.set(peerId.toString(), failures);
if (failures >= this.maxPingFailures) {
try {
log.info(
`Attempting to renew ${peerId.toString()} due to ping failures.`
);
await this.renewAndSubscribePeer(peerId);
this.peerFailures.delete(peerId.toString());
} catch (error) {
log.error(`Failed to renew peer ${peerId.toString()}: ${error}.`);
}
}
}
public notifyMessageReceived(
peerIdStr: string,
message: IProtoMessage
): boolean {
const hash = this.buildMessageHash(message);
this.verifiedPeers.add(peerIdStr);
this.receivedMessagesFormPeer.add(`${peerIdStr}-${hash}`);
log.info(
`notifyMessage received debug: ephemeral:${message.ephemeral}\t${bytesToUtf8(message.payload)}`
);
log.info(`notifyMessage received: peer:${peerIdStr}\tmessage:${hash}`);
if (this.receivedMessages.has(hash)) {
return true;
}
this.receivedMessages.add(hash);
return false;
}
public notifyMessageSent(peerId: PeerId, message: IProtoMessage): void {
const peerIdStr = peerId.toString();
const hash = this.buildMessageHash(message);
log.info(`notifyMessage sent debug: ${bytesToUtf8(message.payload)}`);
if (this.scheduledVerification.has(peerIdStr)) {
log.warn(
`notifyMessage sent: attempting to schedule verification for pending peer:${peerIdStr}\tmessage:${hash}`
);
return;
}
const timeout = setTimeout(
(async () => {
const receivedAnyMessage = this.verifiedPeers.has(peerIdStr);
const receivedTestMessage = this.receivedMessagesFormPeer.has(
`${peerIdStr}-${hash}`
);
if (receivedAnyMessage || receivedTestMessage) {
log.info(
`notifyMessage sent setTimeout: verified that peer pushes filter messages, peer:${peerIdStr}\tmessage:${hash}`
);
return;
}
log.warn(
`notifyMessage sent setTimeout: peer didn't return probe message, attempting renewAndSubscribe, peer:${peerIdStr}\tmessage:${hash}`
);
this.scheduledVerification.delete(peerIdStr);
await this.renewAndSubscribePeer(peerId);
}) as () => void,
MESSAGE_VERIFICATION_DELAY
) as unknown as number;
this.scheduledVerification.set(peerIdStr, timeout);
}
public shouldVerifyPeer(peerId: PeerId): boolean {
const peerIdStr = peerId.toString();
const isPeerVerified = this.verifiedPeers.has(peerIdStr);
const isVerificationPending = this.scheduledVerification.has(peerIdStr);
return !(isPeerVerified || isVerificationPending);
}
private buildMessageHash(message: IProtoMessage): string {
return messageHashStr(this.pubsubTopic, message);
}
private async renewAndSubscribePeer(
peerId: PeerId
): Promise<Peer | undefined> {
const peerIdStr = peerId.toString();
try {
if (this.peerRenewalLocks.has(peerIdStr)) {
log.info(`Peer ${peerIdStr} is already being renewed.`);
return;
}
this.peerRenewalLocks.add(peerIdStr);
const newPeer = await this.renewPeer(peerId);
if (!newPeer) {
log.warn(`Failed to renew peer ${peerIdStr}: No new peer found.`);
return;
}
await this.protocolSubscribe(
this.pubsubTopic,
newPeer,
this.getContentTopics()
);
await this.sendLightPushMessage(newPeer);
this.peerFailures.delete(peerIdStr);
return newPeer;
} catch (error) {
log.error(`Failed to renew peer ${peerIdStr}: ${error}.`);
return;
} finally {
this.peerRenewalLocks.delete(peerIdStr);
}
}
}

View File

@ -0,0 +1 @@
export { Store, wakuStore } from "./store.js";

View File

@ -12,9 +12,7 @@ import {
import { messageHash } from "@waku/message-hash";
import { ensurePubsubTopicIsConfigured, isDefined, Logger } from "@waku/utils";
import { BaseProtocolSDK } from "../base_protocol.js";
const DEFAULT_NUM_PEERS = 1;
import { PeerManager } from "../peer_manager/index.js";
const log = new Logger("waku:store:sdk");
@ -22,22 +20,16 @@ const log = new Logger("waku:store:sdk");
* StoreSDK is an implementation of the IStoreSDK interface.
* It provides methods to interact with the Waku Store protocol.
*/
export class Store extends BaseProtocolSDK implements IStore {
export class Store implements IStore {
public readonly protocol: StoreCore;
public constructor(
connectionManager: ConnectionManager,
private connectionManager: ConnectionManager,
libp2p: Libp2p,
private peerManager: PeerManager,
private options?: Partial<StoreProtocolOptions>
) {
super(
new StoreCore(connectionManager.configuredPubsubTopics, libp2p),
connectionManager,
{
numPeersToUse: DEFAULT_NUM_PEERS
}
);
this.protocol = this.core as StoreCore;
this.protocol = new StoreCore(connectionManager.pubsubTopics, libp2p);
}
/**
@ -231,24 +223,30 @@ export class Store extends BaseProtocolSDK implements IStore {
};
}
private async getPeerToUse(): Promise<Peer | null> {
const peer = this.connectedPeers.find(
(p) => p.id.toString() === this.options?.peer
);
if (peer) {
return peer;
private async getPeerToUse(): Promise<Peer | undefined> {
let peer: Peer | undefined;
if (this.options?.peer) {
const connectedPeers = await this.connectionManager.getConnectedPeers();
peer = connectedPeers.find((p) => p.id.toString() === this.options?.peer);
if (!peer) {
log.warn(
`Passed node to use for Store not found: ${this.options.peer}. Attempting to use random peers.`
);
}
}
log.warn(
`Passed node to use for Store not found: ${this.options?.peer}. Attempting to use random peers.`
);
const peers = await this.peerManager.getPeers();
return (
await this.protocol.getPeers({
numPeers: this.numPeersToUse,
maxBootstrapPeers: 1
})
)[0];
if (peers.length > 0) {
// TODO(weboko): implement smart way of getting a peer https://github.com/waku-org/js-waku/issues/2243
return peers[Math.floor(Math.random() * peers.length)];
}
log.error("No peers available to use.");
return;
}
}
@ -260,9 +258,10 @@ export class Store extends BaseProtocolSDK implements IStore {
*/
export function wakuStore(
connectionManager: ConnectionManager,
peerManager: PeerManager,
options?: Partial<StoreProtocolOptions>
): (libp2p: Libp2p) => IStore {
return (libp2p: Libp2p) => {
return new Store(connectionManager, libp2p, options);
return new Store(connectionManager, libp2p, peerManager, options);
};
}

View File

@ -1,7 +1,9 @@
import { isPeerId, PeerId, type Stream } from "@libp2p/interface";
import { isPeerId } from "@libp2p/interface";
import type { Peer, PeerId, Stream } from "@libp2p/interface";
import { multiaddr, Multiaddr, MultiaddrInput } from "@multiformats/multiaddr";
import { ConnectionManager, getHealthManager, StoreCodec } from "@waku/core";
import type {
CreateNodeOptions,
IFilter,
IHealthManager,
ILightPush,
@ -9,51 +11,20 @@ import type {
IStore,
IWaku,
Libp2p,
ProtocolCreateOptions,
PubsubTopic
} from "@waku/interfaces";
import { Protocols } from "@waku/interfaces";
import { Logger } from "@waku/utils";
import { wakuFilter } from "../protocols/filter/index.js";
import { wakuLightPush } from "../protocols/light_push/index.js";
import { wakuStore } from "../protocols/store/index.js";
import { ReliabilityMonitorManager } from "../reliability_monitor/index.js";
import { wakuFilter } from "../filter/index.js";
import { wakuLightPush } from "../light_push/index.js";
import { PeerManager } from "../peer_manager/index.js";
import { wakuStore } from "../store/index.js";
import { waitForRemotePeer } from "./wait_for_remote_peer.js";
export const DefaultPingKeepAliveValueSecs = 5 * 60;
export const DefaultRelayKeepAliveValueSecs = 5 * 60;
export const DefaultUserAgent = "js-waku";
export const DefaultPingMaxInboundStreams = 10;
const log = new Logger("waku");
export interface WakuOptions {
/**
* Set keep alive frequency in seconds: Waku will send a `/ipfs/ping/1.0.0`
* request to each peer after the set number of seconds. Set to 0 to disable.
*
* @default {@link @waku/core.DefaultPingKeepAliveValueSecs}
*/
pingKeepAlive?: number;
/**
* Set keep alive frequency in seconds: Waku will send a ping message over
* relay to each peer after the set number of seconds. Set to 0 to disable.
*
* @default {@link @waku/core.DefaultRelayKeepAliveValueSecs}
*/
relayKeepAlive?: number;
/**
* Set the user agent string to be used in identification of the node.
* @default {@link @waku/core.DefaultUserAgent}
*/
userAgent?: string;
}
export type CreateWakuNodeOptions = ProtocolCreateOptions &
Partial<WakuOptions>;
type ProtocolsEnabled = {
filter?: boolean;
lightpush?: boolean;
@ -69,9 +40,11 @@ export class WakuNode implements IWaku {
public connectionManager: ConnectionManager;
public readonly health: IHealthManager;
private readonly peerManager: PeerManager;
public constructor(
public readonly pubsubTopics: PubsubTopic[],
options: CreateWakuNodeOptions,
options: CreateNodeOptions,
libp2p: Libp2p,
protocolsEnabled: ProtocolsEnabled,
relay?: IRelay
@ -86,21 +59,21 @@ export class WakuNode implements IWaku {
...protocolsEnabled
};
const pingKeepAlive =
options.pingKeepAlive || DefaultPingKeepAliveValueSecs;
const relayKeepAlive = this.relay
? options.relayKeepAlive || DefaultRelayKeepAliveValueSecs
: 0;
const peerId = this.libp2p.peerId.toString();
this.connectionManager = ConnectionManager.create(
peerId,
this.connectionManager = new ConnectionManager({
libp2p,
{ pingKeepAlive, relayKeepAlive },
this.pubsubTopics,
this.relay
);
relay: this.relay,
pubsubTopics: this.pubsubTopics,
config: options?.connectionManager
});
this.peerManager = new PeerManager({
libp2p,
config: {
numPeersToUse: options.numPeersToUse
}
});
this.health = getHealthManager();
@ -113,22 +86,23 @@ export class WakuNode implements IWaku {
});
}
const store = wakuStore(this.connectionManager, {
const store = wakuStore(this.connectionManager, this.peerManager, {
peer: options.store?.peer
});
this.store = store(libp2p);
}
if (protocolsEnabled.lightpush) {
const lightPush = wakuLightPush(this.connectionManager, options);
const lightPush = wakuLightPush(this.connectionManager, this.peerManager);
this.lightPush = lightPush(libp2p);
}
if (protocolsEnabled.filter) {
const filter = wakuFilter(
this.connectionManager,
this.peerManager,
this.lightPush,
options
options.filter
);
this.filter = filter(libp2p);
}
@ -212,11 +186,15 @@ export class WakuNode implements IWaku {
}
public async stop(): Promise<void> {
ReliabilityMonitorManager.stopAll();
this.peerManager.stop();
this.connectionManager.stop();
await this.libp2p.stop();
}
public async getConnectedPeers(): Promise<Peer[]> {
return this.connectionManager.getConnectedPeers();
}
public async waitForPeers(
protocols?: Protocols[],
timeoutMs?: number

View File

@ -1,8 +1,4 @@
import {
NetworkConfig,
ProtocolCreateOptions,
Protocols
} from "@waku/interfaces";
import { CreateNodeOptions, NetworkConfig, Protocols } from "@waku/interfaces";
import { createRelayNode } from "@waku/relay";
import { createLightNode, WakuNode } from "@waku/sdk";
import {
@ -46,7 +42,7 @@ export async function runNodes<T>(
},
{ retries: 3 }
);
const waku_options: ProtocolCreateOptions = {
const waku_options: CreateNodeOptions = {
staticNoiseKey: NOISE_KEY_1,
libp2p: { addresses: { listen: ["/ip4/0.0.0.0/tcp/0/ws"] } },
networkConfig: shardInfo

View File

@ -1,9 +1,9 @@
import {
CreateNodeOptions,
DefaultNetworkConfig,
IWaku,
LightNode,
NetworkConfig,
ProtocolCreateOptions,
Protocols
} from "@waku/interfaces";
import { createLightNode } from "@waku/sdk";
@ -35,7 +35,7 @@ export async function runMultipleNodes(
withoutFilter
);
const wakuOptions: ProtocolCreateOptions = {
const wakuOptions: CreateNodeOptions = {
staticNoiseKey: NOISE_KEY_1,
libp2p: {
addresses: { listen: ["/ip4/0.0.0.0/tcp/0/ws"] }

View File

@ -1,323 +0,0 @@
import { ISubscription, LightNode, SDKProtocolResult } from "@waku/interfaces";
import {
createDecoder,
createEncoder,
DecodedMessage,
utf8ToBytes
} from "@waku/sdk";
import { delay } from "@waku/utils";
import { expect } from "chai";
import { describe } from "mocha";
import {
afterEachCustom,
beforeEachCustom,
DefaultTestPubsubTopic,
DefaultTestShardInfo,
runMultipleNodes,
ServiceNode,
ServiceNodesFleet,
teardownNodesWithRedundancy
} from "../../src/index.js";
describe("Waku Filter: Peer Management: E2E", function () {
this.timeout(15000);
let waku: LightNode;
let serviceNodes: ServiceNodesFleet;
const contentTopic = "/test";
const encoder = createEncoder({
pubsubTopic: DefaultTestPubsubTopic,
contentTopic
});
const decoder = createDecoder(contentTopic, DefaultTestPubsubTopic);
beforeEachCustom(this, async () => {
[serviceNodes, waku] = await runMultipleNodes(
this.ctx,
DefaultTestShardInfo,
{ lightpush: true, filter: true },
undefined,
5
);
});
afterEachCustom(this, async () => {
await teardownNodesWithRedundancy(serviceNodes, waku);
});
it("Number of peers are maintained correctly", async function () {
const messages: DecodedMessage[] = [];
const { error, results } = await waku.filter.subscribe([decoder], (msg) => {
messages.push(msg);
});
if (error) {
throw error;
}
const { successes, failures } = results;
await waku.lightPush.send(encoder, {
payload: utf8ToBytes("Hello_World")
});
expect(successes.length).to.be.greaterThan(0);
expect(successes.length).to.be.equal(waku.filter.numPeersToUse);
if (failures) {
expect(failures.length).to.equal(0);
}
});
it("Ping succeeds for all connected peers", async function () {
const { error, subscription } = await waku.filter.subscribe(
[decoder],
() => {}
);
if (error) {
throw error;
}
const pingResult = await subscription.ping();
expect(pingResult.successes.length).to.equal(waku.filter.numPeersToUse);
expect(pingResult.failures.length).to.equal(0);
});
it("Ping fails for unsubscribed peers", async function () {
const { error, subscription } = await waku.filter.subscribe(
[decoder],
() => {}
);
if (error) {
throw error;
}
await subscription.unsubscribe([contentTopic]);
const pingResult = await subscription.ping();
expect(pingResult.successes.length).to.equal(0);
expect(pingResult.failures.length).to.be.greaterThan(0);
});
it("Keep-alive pings maintain the connection", async function () {
const { error, subscription } = await waku.filter.subscribe(
[decoder],
() => {},
undefined,
{ keepAlive: 100 }
);
if (error) {
throw error;
}
await delay(1000);
const pingResult = await subscription.ping();
expect(pingResult.successes.length).to.equal(waku.filter.numPeersToUse);
expect(pingResult.failures.length).to.equal(0);
});
it("Renews peer on consistent ping failures", async function () {
const maxPingFailures = 3;
const { error, subscription } = await waku.filter.subscribe(
[decoder],
() => {},
undefined,
{
pingsBeforePeerRenewed: maxPingFailures
}
);
if (error) {
throw error;
}
const disconnectedNodePeerId = waku.filter.connectedPeers[0].id;
await waku.connectionManager.dropConnection(disconnectedNodePeerId);
// Ping multiple times to exceed max failures
for (let i = 0; i <= maxPingFailures; i++) {
await subscription.ping();
await delay(100);
}
const pingResult = await subscription.ping();
expect(pingResult.successes.length).to.equal(waku.filter.numPeersToUse);
expect(pingResult.failures.length).to.equal(0);
expect(waku.filter.connectedPeers.length).to.equal(
waku.filter.numPeersToUse
);
expect(
waku.filter.connectedPeers.some((peer) =>
peer.id.equals(disconnectedNodePeerId)
)
).to.eq(false);
});
it("Tracks peer failures correctly", async function () {
const maxPingFailures = 3;
const { error, subscription } = await waku.filter.subscribe(
[decoder],
() => {},
undefined,
{
pingsBeforePeerRenewed: maxPingFailures
}
);
if (error) {
throw error;
}
const targetPeer = waku.filter.connectedPeers[0];
await waku.connectionManager.dropConnection(targetPeer.id);
for (let i = 0; i < maxPingFailures; i++) {
await subscription.ping(targetPeer.id);
}
// At this point, the peer should not be renewed yet
expect(
waku.filter.connectedPeers.some((peer) => peer.id.equals(targetPeer.id))
).to.be.true;
// One more failure should trigger renewal
await subscription.ping(targetPeer.id);
// adds delay as renewal happens as an async operation in the bg
await delay(300);
expect(
waku.filter.connectedPeers.some((peer) => peer.id.equals(targetPeer.id))
).to.eq(false);
expect(waku.filter.connectedPeers.length).to.equal(
waku.filter.numPeersToUse
);
});
it("Maintains correct number of peers after multiple subscribe/unsubscribe cycles", async function () {
let subscription: ISubscription;
for (let i = 0; i < 3; i++) {
const { error, subscription: _subscription } =
await waku.filter.subscribe([decoder], () => {});
if (error) {
throw error;
}
subscription = _subscription;
let pingResult = await subscription.ping();
expect(pingResult.successes.length).to.equal(waku.filter.numPeersToUse);
await subscription.unsubscribe([contentTopic]);
pingResult = await subscription.ping();
expect(pingResult.failures.length).to.be.greaterThan(0);
await subscription.subscribe([decoder], () => {});
}
const finalPingResult = await subscription!.ping();
expect(finalPingResult.successes.length).to.equal(
waku.filter.numPeersToUse
);
});
it("Renews peer on consistent missed messages", async function () {
const [serviceNodes, waku] = await runMultipleNodes(
this.ctx,
DefaultTestShardInfo,
{ lightpush: true, filter: true },
undefined,
2
);
const serviceNodesPeerIdStr = await Promise.all(
serviceNodes.nodes.map(async (node) =>
(await node.getPeerId()).toString()
)
);
const nodeWithoutDiscovery = new ServiceNode("WithoutDiscovery");
await nodeWithoutDiscovery.start({ lightpush: true, filter: true });
const nodeWithouDiscoveryPeerIdStr = (
await nodeWithoutDiscovery.getPeerId()
).toString();
await waku.dial(await nodeWithoutDiscovery.getMultiaddrWithId());
const messages: DecodedMessage[] = [];
const { error, results } = await waku.filter.subscribe([decoder], (msg) => {
messages.push(msg);
});
if (error) {
throw error;
}
const { successes } = results;
expect(successes.length).to.be.greaterThan(0);
expect(successes.length).to.be.equal(waku.filter.numPeersToUse);
const sendMessage: () => Promise<SDKProtocolResult> = async () =>
waku.lightPush.send(encoder, {
payload: utf8ToBytes("Hello_World")
});
await sendMessage();
successes
.map((peerId) =>
[nodeWithouDiscoveryPeerIdStr, ...serviceNodesPeerIdStr].includes(
peerId.toString()
)
)
.forEach((isConnected) => expect(isConnected).to.eq(true));
// send 2 more messages
await sendMessage();
await sendMessage();
expect(waku.filter.connectedPeers.length).to.equal(2);
});
it("Renews peer for Filter on peer:disconnect event", async function () {
this.timeout(30000);
const messages: DecodedMessage[] = [];
const { error, subscription } = await waku.filter.subscribe(
[decoder],
(msg) => {
messages.push(msg);
}
);
if (error) {
throw error;
}
const initialPeers = waku.filter.connectedPeers;
expect(initialPeers.length).to.equal(waku.filter.numPeersToUse);
const peerToDisconnect = initialPeers[0];
await waku.connectionManager.dropConnection(peerToDisconnect.id);
await delay(5000);
expect(waku.filter.connectedPeers.length).to.equal(
waku.filter.numPeersToUse
);
const stillConnected = waku.filter.connectedPeers.some((peer) =>
peer.id.equals(peerToDisconnect.id)
);
expect(stillConnected).to.be.false;
await waku.lightPush.send(encoder, {
payload: utf8ToBytes("Hello after disconnect")
});
await delay(2000);
expect(messages.length).to.equal(1);
expect(new TextDecoder().decode(messages[0].payload)).to.equal(
"Hello after disconnect"
);
const pingResult = await subscription.ping();
expect(pingResult.successes.length).to.equal(waku.filter.numPeersToUse);
expect(pingResult.failures.length).to.equal(0);
});
});

View File

@ -1,11 +1,11 @@
import { createDecoder, createEncoder } from "@waku/core";
import {
CreateNodeOptions,
DefaultNetworkConfig,
ISubscription,
IWaku,
LightNode,
NetworkConfig,
ProtocolCreateOptions,
Protocols
} from "@waku/interfaces";
import { createLightNode } from "@waku/sdk";
@ -85,7 +85,7 @@ export async function runMultipleNodes(
withoutFilter
);
const wakuOptions: ProtocolCreateOptions = {
const wakuOptions: CreateNodeOptions = {
staticNoiseKey: NOISE_KEY_1,
libp2p: {
addresses: { listen: ["/ip4/0.0.0.0/tcp/0/ws"] }

View File

@ -1,340 +0,0 @@
import { generateKeyPair } from "@libp2p/crypto/keys";
import type { Connection, Peer, PeerStore } from "@libp2p/interface";
import { peerIdFromPrivateKey } from "@libp2p/peer-id";
import {
createLightNode,
Libp2pComponents,
type LightNode,
Tags,
utf8ToBytes
} from "@waku/sdk";
import { encodeRelayShard } from "@waku/utils";
import { expect } from "chai";
import fc from "fast-check";
import Sinon from "sinon";
import {
afterEachCustom,
beforeEachCustom,
DefaultTestShardInfo
} from "../src/index.js";
describe("getPeers", function () {
let peerStore: PeerStore;
let connectionManager: Libp2pComponents["connectionManager"];
let waku: LightNode;
const lowPingBytes = utf8ToBytes("50");
const midPingBytes = utf8ToBytes("100");
const highPingBytes = utf8ToBytes("200");
let lowPingBootstrapPeer: Peer,
lowPingNonBootstrapPeer: Peer,
midPingBootstrapPeer: Peer,
midPingNonBootstrapPeer: Peer,
highPingBootstrapPeer: Peer,
highPingNonBootstrapPeer: Peer,
differentCodecPeer: Peer,
anotherDifferentCodecPeer: Peer;
let bootstrapPeers: Peer[];
let nonBootstrapPeers: Peer[];
let allPeers: Peer[];
beforeEachCustom(this, async () => {
waku = await createLightNode({ networkConfig: DefaultTestShardInfo });
peerStore = waku.libp2p.peerStore;
connectionManager = waku.libp2p.components.connectionManager;
const [
lowPingBootstrapPeerId,
lowPingNonBootstrapPeerId,
midPingBootstrapPeerId,
midPingNonBootstrapPeerId,
highPingBootstrapPeerId,
highPingNonBootstrapPeerId,
differentCodecPeerId,
anotherDifferentCodecPeerId
] = await Promise.all([
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey),
generateKeyPair("secp256k1").then(peerIdFromPrivateKey)
]);
lowPingBootstrapPeer = {
id: lowPingBootstrapPeerId,
protocols: [waku.lightPush.protocol.multicodec],
metadata: new Map().set("ping", lowPingBytes),
tags: new Map().set(Tags.BOOTSTRAP, {})
} as Peer;
lowPingNonBootstrapPeer = {
id: lowPingNonBootstrapPeerId,
protocols: [waku.lightPush.protocol.multicodec],
metadata: new Map().set("ping", lowPingBytes),
tags: new Map().set(Tags.PEER_EXCHANGE, {})
} as Peer;
midPingBootstrapPeer = {
id: midPingBootstrapPeerId,
protocols: [waku.lightPush.protocol.multicodec],
metadata: new Map().set("ping", midPingBytes),
tags: new Map().set(Tags.BOOTSTRAP, {})
} as Peer;
midPingNonBootstrapPeer = {
id: midPingNonBootstrapPeerId,
protocols: [waku.lightPush.protocol.multicodec],
metadata: new Map().set("ping", midPingBytes),
tags: new Map().set(Tags.PEER_EXCHANGE, {})
} as Peer;
highPingBootstrapPeer = {
id: highPingBootstrapPeerId,
protocols: [waku.lightPush.protocol.multicodec],
metadata: new Map().set("ping", highPingBytes),
tags: new Map().set(Tags.BOOTSTRAP, {})
} as Peer;
highPingNonBootstrapPeer = {
id: highPingNonBootstrapPeerId,
protocols: [waku.lightPush.protocol.multicodec],
metadata: new Map().set("ping", highPingBytes),
tags: new Map().set(Tags.PEER_EXCHANGE, {})
} as Peer;
differentCodecPeer = {
id: differentCodecPeerId,
protocols: ["different/1"],
metadata: new Map().set("ping", lowPingBytes),
tags: new Map().set(Tags.BOOTSTRAP, {})
} as Peer;
anotherDifferentCodecPeer = {
id: anotherDifferentCodecPeerId,
protocols: ["different/2"],
metadata: new Map().set("ping", lowPingBytes),
tags: new Map().set(Tags.BOOTSTRAP, {})
} as Peer;
bootstrapPeers = [
lowPingBootstrapPeer,
midPingBootstrapPeer,
highPingBootstrapPeer
];
nonBootstrapPeers = [
lowPingNonBootstrapPeer,
midPingNonBootstrapPeer,
highPingNonBootstrapPeer
];
allPeers = [
...bootstrapPeers,
...nonBootstrapPeers,
differentCodecPeer,
anotherDifferentCodecPeer
];
allPeers.forEach((peer) => {
peer.metadata.set("shardInfo", encodeRelayShard(DefaultTestShardInfo));
});
Sinon.stub(peerStore, "get").callsFake(async (peerId) => {
return allPeers.find((peer) => peer.id.equals(peerId))!;
});
Sinon.stub(peerStore, "forEach").callsFake(async (callback) => {
for (const peer of allPeers) {
callback(peer);
}
});
// assume all peers have an opened connection
Sinon.stub(connectionManager, "getConnections").callsFake(() => {
const connections: Connection[] = [];
for (const peer of allPeers) {
connections.push({
status: "open",
remotePeer: peer.id,
streams: [{ protocol: waku.lightPush.protocol.multicodec }]
} as unknown as Connection);
}
return connections;
});
});
afterEachCustom(this, async () => {
Sinon.restore();
});
describe("getPeers with varying maxBootstrapPeers", function () {
const maxBootstrapPeersValues = [1, 2, 3, 4, 5, 6, 7];
maxBootstrapPeersValues.forEach((maxBootstrapPeers) => {
describe(`maxBootstrapPeers=${maxBootstrapPeers}`, function () {
it(`numPeers=1 -- returns one bootstrap peer `, async function () {
const result = (await (waku.lightPush.protocol as any).getPeers({
numPeers: 1,
maxBootstrapPeers
})) as Peer[];
// Should only have 1 peer
expect(result).to.have.lengthOf(1);
// The peer should be a bootstrap peer
expect(result[0].tags.has(Tags.BOOTSTRAP)).to.be.true;
// Peer should be of the same protocol
expect(
result[0].protocols.includes(waku.lightPush.protocol.multicodec)
).to.be.true;
// Peer should have the lowest ping
expect(result[0].metadata.get("ping")).to.equal(lowPingBytes);
});
it(`numPeers=2 -- returns total 2 peers, with max ${maxBootstrapPeers} bootstrap peers`, async function () {
const result = (await (waku.lightPush.protocol as any).getPeers({
numPeers: 2,
maxBootstrapPeers
})) as Peer[];
// Should only have 2 peers
expect(result).to.have.lengthOf(2);
// Should only have ${maxBootstrapPeers} bootstrap peers
expect(
result.filter((peer: Peer) => peer.tags.has(Tags.BOOTSTRAP)).length
).to.be.lessThanOrEqual(maxBootstrapPeers);
// Should return peers with the same protocol
expect(
result.every((peer: Peer) =>
peer.protocols.includes(waku.lightPush.protocol.multicodec)
)
).to.be.true;
// All peers should be sorted by latency
// 0th index should be the lowest ping of all peers returned
expect(result[0].metadata.get("ping")).to.equal(lowPingBytes);
});
it(`numPeers=3 -- returns total 3 peers, with max ${maxBootstrapPeers} bootstrap peers`, async function () {
const result = (await (waku.lightPush.protocol as any).getPeers({
numPeers: 3,
maxBootstrapPeers
})) as Peer[];
// Should only have 3 peers
expect(result).to.have.lengthOf(3);
// Should only have ${maxBootstrapPeers} bootstrap peers
expect(
result.filter((peer: Peer) => peer.tags.has(Tags.BOOTSTRAP)).length
).to.be.lessThanOrEqual(maxBootstrapPeers);
// Should return peers with the same protocol
expect(
result.every((peer: Peer) =>
peer.protocols.includes(waku.lightPush.protocol.multicodec)
)
).to.be.true;
// All peers should be sorted by latency
// 0th index should be the lowest ping of all peers returned
expect(result[0].metadata.get("ping")).to.equal(lowPingBytes);
});
it(`numPeers=4 -- returns total 4 peers, with max ${maxBootstrapPeers} bootstrap peers`, async function () {
const result = (await (waku.lightPush.protocol as any).getPeers({
numPeers: 4,
maxBootstrapPeers
})) as Peer[];
// Should only have 4 peers
expect(result).to.have.lengthOf(4);
// Should only have ${maxBootstrapPeers} bootstrap peers
expect(
result.filter((peer: Peer) => peer.tags.has(Tags.BOOTSTRAP)).length
).to.be.lessThanOrEqual(maxBootstrapPeers);
// Should return peers with the same protocol
expect(
result.every((peer: Peer) =>
peer.protocols.includes(waku.lightPush.protocol.multicodec)
)
).to.be.true;
// All peers should be sorted by latency
// 0th index should be the lowest ping of all peers returned
expect(result[0].metadata.get("ping")).to.equal(lowPingBytes);
});
it(`numPeers=0 -- returns all peers including all non-bootstrap with maxBootstrapPeers: ${maxBootstrapPeers}`, async function () {
const result = (await (waku.lightPush.protocol as any).getPeers({
numPeers: 0,
maxBootstrapPeers
})) as Peer[];
// Should have all non-bootstrap peers + ${maxBootstrapPeers} bootstrap peers
// Unless bootstrapPeers.length < maxBootstrapPeers
// Then it should be all non-bootstrap peers + bootstrapPeers.length
if (maxBootstrapPeers > bootstrapPeers.length) {
expect(result).to.have.lengthOf(
nonBootstrapPeers.length + bootstrapPeers.length
);
} else {
expect(result).to.have.lengthOf(
nonBootstrapPeers.length + maxBootstrapPeers
);
}
// All peers should be bootstrap peers
expect(
result.filter((peer: Peer) => peer.tags.has(Tags.BOOTSTRAP)).length
).to.be.lessThanOrEqual(maxBootstrapPeers);
// Peers should be of the same protocol
expect(
result.every((peer: Peer) =>
peer.protocols.includes(waku.lightPush.protocol.multicodec)
)
).to.be.true;
// All peers returned should be sorted by latency
// 0th index should be the lowest ping of all peers returned
expect(result[0].metadata.get("ping")).to.equal(lowPingBytes);
});
});
});
});
describe("getPeers property-based tests", function () {
it("should return the correct number of peers based on numPeers and maxBootstrapPeers", async function () {
await fc.assert(
fc.asyncProperty(
//max bootstrap peers
fc.integer({ min: 1, max: 100 }),
//numPeers
fc.integer({ min: 0, max: 100 }),
async (maxBootstrapPeers, numPeers) => {
const result = (await (waku.lightPush.protocol as any).getPeers({
numPeers,
maxBootstrapPeers
})) as Peer[];
if (numPeers === 0) {
// Expect all peers when numPeers is 0
expect(result.length).to.be.greaterThanOrEqual(1);
} else {
// Expect up to numPeers peers
expect(result.length).to.be.lessThanOrEqual(numPeers);
}
}
),
{
verbose: true
}
);
});
});
});

View File

@ -253,7 +253,9 @@ describe("Metadata Protocol", function () {
waku = await createLightNode({
networkConfig: shardInfo,
pingKeepAlive: 1
connectionManager: {
pingKeepAlive: 1
}
});
await waku.start();
await waku.libp2p.dialProtocol(nwaku1Ma, MetadataCodec);

View File

@ -115,7 +115,7 @@ describe("Wait for remote peer", function () {
await delay(1000);
await waku2.waitForPeers([Protocols.Store]);
const peers = (await waku2.store.protocol.connectedPeers()).map((peer) =>
const peers = (await waku2.getConnectedPeers()).map((peer) =>
peer.id.toString()
);
const nimPeerId = multiAddrWithId.getPeerId();
@ -145,7 +145,7 @@ describe("Wait for remote peer", function () {
await waku2.dial(multiAddrWithId);
await waitPromise;
const peers = (await waku2.store.protocol.connectedPeers()).map((peer) =>
const peers = (await waku2.getConnectedPeers()).map((peer) =>
peer.id.toString()
);
@ -174,8 +174,8 @@ describe("Wait for remote peer", function () {
await waku2.dial(multiAddrWithId);
await waku2.waitForPeers([Protocols.LightPush]);
const peers = (await waku2.lightPush.protocol.connectedPeers()).map(
(peer) => peer.id.toString()
const peers = (await waku2.getConnectedPeers()).map((peer) =>
peer.id.toString()
);
const nimPeerId = multiAddrWithId.getPeerId();
@ -203,7 +203,7 @@ describe("Wait for remote peer", function () {
await waku2.dial(multiAddrWithId);
await waku2.waitForPeers([Protocols.Filter]);
const peers = (await waku2.filter.protocol.connectedPeers()).map((peer) =>
const peers = (await waku2.getConnectedPeers()).map((peer) =>
peer.id.toString()
);
@ -213,14 +213,15 @@ describe("Wait for remote peer", function () {
expect(peers.includes(nimPeerId as string)).to.be.true;
});
// TODO: re-enable store once https://github.com/waku-org/js-waku/issues/2162 is fixed
it("Light Node - default protocols", async function () {
this.timeout(20_000);
nwaku = new ServiceNode(makeLogFileName(this));
await nwaku.start({
filter: true,
lightpush: true,
relay: false,
store: true
relay: false
// store: true
});
const multiAddrWithId = await nwaku.getMultiaddrWithId();
@ -232,26 +233,18 @@ describe("Wait for remote peer", function () {
await waku2.dial(multiAddrWithId);
await waku2.waitForPeers([
Protocols.Filter,
Protocols.Store,
// Protocols.Store,
Protocols.LightPush
]);
const filterPeers = (await waku2.filter.protocol.connectedPeers()).map(
(peer) => peer.id.toString()
const peers = (await waku2.getConnectedPeers()).map((peer) =>
peer.id.toString()
);
const storePeers = (await waku2.store.protocol.connectedPeers()).map(
(peer) => peer.id.toString()
);
const lightPushPeers = (
await waku2.lightPush.protocol.connectedPeers()
).map((peer) => peer.id.toString());
const nimPeerId = multiAddrWithId.getPeerId();
expect(nimPeerId).to.not.be.undefined;
expect(filterPeers.includes(nimPeerId as string)).to.be.true;
expect(storePeers.includes(nimPeerId as string)).to.be.true;
expect(lightPushPeers.includes(nimPeerId as string)).to.be.true;
expect(peers.includes(nimPeerId as string)).to.be.true;
});
it("Privacy Node - default protocol", async function () {

View File

@ -11,8 +11,7 @@ import {
import { createRelayNode } from "@waku/relay";
import {
createLightNode,
createEncoder as createPlainEncoder,
DefaultUserAgent
createEncoder as createPlainEncoder
} from "@waku/sdk";
import { bytesToUtf8, utf8ToBytes } from "@waku/utils/bytes";
import { expect } from "chai";
@ -278,7 +277,7 @@ describe("User Agent", function () {
waku1UserAgent
);
expect(bytesToUtf8(waku2PeerInfo.metadata.get("AgentVersion")!)).to.eq(
DefaultUserAgent
"js-waku"
);
});
});

View File

@ -9,10 +9,6 @@
"types": "./dist/index.d.ts",
"import": "./dist/index.js"
},
"./libp2p": {
"types": "./dist/libp2p/index.d.ts",
"import": "./dist/libp2p/index.js"
},
"./bytes": {
"types": "./dist/bytes/index.d.ts",
"import": "./dist/bytes/index.js"

View File

@ -1,11 +1,11 @@
import type {
ContentTopicInfo,
ProtocolCreateOptions,
CreateNodeOptions,
StaticSharding
} from "@waku/interfaces";
export function isStaticSharding(
config: NonNullable<ProtocolCreateOptions["networkConfig"]>
config: NonNullable<CreateNodeOptions["networkConfig"]>
): config is StaticSharding {
return (
"clusterId" in config && "shards" in config && !("contentTopics" in config)
@ -13,7 +13,7 @@ export function isStaticSharding(
}
export function isAutoSharding(
config: NonNullable<ProtocolCreateOptions["networkConfig"]>
config: NonNullable<CreateNodeOptions["networkConfig"]>
): config is ContentTopicInfo {
return "contentTopics" in config;
}

View File

@ -1,69 +0,0 @@
import type { Peer, PeerStore } from "@libp2p/interface";
import { bytesToUtf8 } from "../bytes/index.js";
/**
* Returns a pseudo-random peer that supports the given protocol.
* Useful for protocols such as store and light push
*/
export function selectRandomPeer(peers: Peer[]): Peer | undefined {
if (peers.length === 0) return;
const index = Math.round(Math.random() * (peers.length - 1));
return peers[index];
}
/**
* Function to sort peers by latency from lowest to highest
* @param peerStore - The Libp2p PeerStore
* @param peers - The list of peers to choose from
* @returns Sorted array of peers by latency
*/
export async function sortPeersByLatency(
peerStore: PeerStore,
peers: Peer[]
): Promise<Peer[]> {
if (peers.length === 0) return [];
const results = await Promise.all(
peers.map(async (peer) => {
try {
const pingBytes = (await peerStore.get(peer.id)).metadata.get("ping");
if (!pingBytes) return { peer, ping: Infinity };
const ping = Number(bytesToUtf8(pingBytes));
return { peer, ping };
} catch (error) {
return { peer, ping: Infinity };
}
})
);
// filter out null values
const validResults = results.filter(
(result): result is { peer: Peer; ping: number } => result !== null
);
return validResults
.sort((a, b) => a.ping - b.ping)
.map((result) => result.peer);
}
/**
* Returns the list of peers that supports the given protocol.
*/
export async function getPeersForProtocol(
peerStore: PeerStore,
protocols: string[]
): Promise<Peer[]> {
const peers: Peer[] = [];
await peerStore.forEach((peer) => {
for (let i = 0; i < protocols.length; i++) {
if (peer.protocols.includes(protocols[i])) {
peers.push(peer);
break;
}
}
});
return peers;
}