mirror of
https://github.com/logos-messaging/js-waku.git
synced 2026-01-07 16:23:09 +00:00
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:
parent
d706fe146c
commit
62f93dc842
@ -116,6 +116,7 @@
|
||||
"upgrader",
|
||||
"vacp",
|
||||
"varint",
|
||||
"weboko",
|
||||
"waku",
|
||||
"wakuconnect",
|
||||
"wakunode",
|
||||
|
||||
2
.github/workflows/ci.yml
vendored
2
.github/workflows/ci.yml
vendored
@ -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:
|
||||
|
||||
2
.github/workflows/playwright.yml
vendored
2
.github/workflows/playwright.yml
vendored
@ -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
7605
package-lock.json
generated
File diff suppressed because it is too large
Load Diff
@ -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"
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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";
|
||||
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@ -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;
|
||||
}
|
||||
1
packages/core/src/lib/connection_manager/index.ts
Normal file
1
packages/core/src/lib/connection_manager/index.ts
Normal file
@ -0,0 +1 @@
|
||||
export { ConnectionManager } from "./connection_manager.js";
|
||||
@ -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;
|
||||
25
packages/core/src/lib/connection_manager/utils.ts
Normal file
25
packages/core/src/lib/connection_manager/utils.ts
Normal 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;
|
||||
}
|
||||
};
|
||||
315
packages/core/src/lib/filter/filter.ts
Normal file
315
packages/core/src/lib/filter/filter.ts
Normal 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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";
|
||||
|
||||
@ -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);
|
||||
});
|
||||
});
|
||||
@ -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;
|
||||
}
|
||||
@ -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";
|
||||
|
||||
189
packages/core/src/lib/light_push/light_push.ts
Normal file
189
packages/core/src/lib/light_push/light_push.ts
Normal 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 };
|
||||
}
|
||||
}
|
||||
@ -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";
|
||||
|
||||
182
packages/core/src/lib/metadata/metadata.ts
Normal file
182
packages/core/src/lib/metadata/metadata.ts
Normal 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);
|
||||
}
|
||||
@ -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";
|
||||
|
||||
136
packages/core/src/lib/store/store.ts
Normal file
136
packages/core/src/lib/store/store.ts
Normal 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -32,7 +32,7 @@ export class WakuPeerExchange extends BaseProtocol implements IPeerExchange {
|
||||
components: Libp2pComponents,
|
||||
pubsubTopics: PubsubTopic[]
|
||||
) {
|
||||
super(PeerExchangeCodec, components, log, pubsubTopics);
|
||||
super(PeerExchangeCodec, components, pubsubTopics);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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;
|
||||
|
||||
|
||||
@ -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";
|
||||
|
||||
@ -1,4 +0,0 @@
|
||||
export interface KeepAliveOptions {
|
||||
pingKeepAlive: number;
|
||||
relayKeepAlive: number;
|
||||
}
|
||||
@ -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.
|
||||
*/
|
||||
|
||||
@ -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;
|
||||
};
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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;
|
||||
|
||||
/**
|
||||
|
||||
@ -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",
|
||||
|
||||
@ -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);
|
||||
|
||||
|
||||
@ -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(
|
||||
|
||||
3
packages/sdk/src/filter/constants.ts
Normal file
3
packages/sdk/src/filter/constants.ts
Normal 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;
|
||||
@ -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);
|
||||
}
|
||||
1
packages/sdk/src/filter/index.ts
Normal file
1
packages/sdk/src/filter/index.ts
Normal file
@ -0,0 +1 @@
|
||||
export { wakuFilter } from "./filter.js";
|
||||
260
packages/sdk/src/filter/subscription.ts
Normal file
260
packages/sdk/src/filter/subscription.ts
Normal 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);
|
||||
}
|
||||
}
|
||||
293
packages/sdk/src/filter/subscription_monitor.ts
Normal file
293
packages/sdk/src/filter/subscription_monitor.ts
Normal 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
15
packages/sdk/src/filter/utils.ts
Normal file
15
packages/sdk/src/filter/utils.ts
Normal 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
|
||||
};
|
||||
};
|
||||
@ -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";
|
||||
|
||||
1
packages/sdk/src/light_push/index.ts
Normal file
1
packages/sdk/src/light_push/index.ts
Normal file
@ -0,0 +1 @@
|
||||
export { LightPush, wakuLightPush } from "./light_push.js";
|
||||
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
@ -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);
|
||||
}
|
||||
1
packages/sdk/src/peer_manager/index.ts
Normal file
1
packages/sdk/src/peer_manager/index.ts
Normal file
@ -0,0 +1 @@
|
||||
export { PeerManager } from "./peer_manager.js";
|
||||
109
packages/sdk/src/peer_manager/peer_manager.spec.ts
Normal file
109
packages/sdk/src/peer_manager/peer_manager.spec.ts
Normal 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;
|
||||
}
|
||||
141
packages/sdk/src/peer_manager/peer_manager.ts
Normal file
141
packages/sdk/src/peer_manager/peer_manager.ts
Normal 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);
|
||||
}
|
||||
}
|
||||
@ -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 });
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
};
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
@ -1 +0,0 @@
|
||||
export { wakuLightPush } from "./light_push.js";
|
||||
@ -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;
|
||||
});
|
||||
});
|
||||
});
|
||||
@ -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;
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
1
packages/sdk/src/store/index.ts
Normal file
1
packages/sdk/src/store/index.ts
Normal file
@ -0,0 +1 @@
|
||||
export { Store, wakuStore } from "./store.js";
|
||||
@ -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);
|
||||
};
|
||||
}
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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"] }
|
||||
|
||||
@ -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);
|
||||
});
|
||||
});
|
||||
@ -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"] }
|
||||
|
||||
@ -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
|
||||
}
|
||||
);
|
||||
});
|
||||
});
|
||||
});
|
||||
@ -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);
|
||||
|
||||
@ -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 () {
|
||||
|
||||
@ -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"
|
||||
);
|
||||
});
|
||||
});
|
||||
|
||||
@ -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"
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -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;
|
||||
}
|
||||
Loading…
x
Reference in New Issue
Block a user