feat: use napi-rs blst bindings (#6894)

This commit is contained in:
Cayman
2024-07-25 01:13:59 -06:00
committed by GitHub
parent acbedaf140
commit 2b627548d1
88 changed files with 485 additions and 386 deletions

View File

@@ -5101,7 +5101,7 @@
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
"expr": "rate(lodestar_bls_thread_pool_signature_deserialization_main_thread_time_seconds_bucket[$rate_interval])",
"expr": "rate(lodestar_gossip_block_gossip_validate_time_bucket[$rate_interval])",
"format": "heatmap",
"instant": false,
"legendFormat": "time",
@@ -5273,7 +5273,7 @@
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
"expr": "rate(lodestar_bls_thread_pool_signature_deserialization_main_thread_time_seconds_sum[$rate_interval]) / rate(lodestar_bls_thread_pool_signature_deserialization_main_thread_time_seconds_count[$rate_interval])",
"expr": "rate(lodestar_gossip_block_gossip_validate_time_sum[$rate_interval]) / rate(lodestar_gossip_block_gossip_validate_time_count[$rate_interval])",
"format": "heatmap",
"instant": false,
"legendFormat": "time",

View File

@@ -1174,9 +1174,9 @@
"uid": "${DS_PROMETHEUS}"
},
"editorMode": "code",
"expr": "rate(lodestar_bls_thread_pool_signature_deserialization_main_thread_time_seconds_sum[$rate_interval]) * 384",
"expr": "rate(lodestar_bls_thread_pool_aggregate_with_randomness_main_thread_time_seconds_sum[$rate_interval]) * 384",
"instant": false,
"legendFormat": "signature_deserialization",
"legendFormat": "aggregate_with_randomness",
"range": true,
"refId": "A"
},
@@ -1270,7 +1270,7 @@
"disableTextWrap": false,
"editorMode": "code",
"exemplar": false,
"expr": "rate(lodestar_bls_thread_pool_signature_deserialization_main_thread_time_seconds_bucket[$rate_interval])",
"expr": "rate(lodestar_bls_thread_pool_aggregate_with_randomness_main_thread_time_seconds_bucket[$rate_interval])",
"format": "heatmap",
"fullMetaSearch": false,
"includeNullMetadata": true,
@@ -1281,7 +1281,7 @@
"useBackend": false
}
],
"title": "Main Thread Signature Aggregation Time",
"title": "Main Thread AggregateWithRandomness Time",
"type": "heatmap"
},
{

View File

@@ -95,8 +95,7 @@
},
"dependencies": {
"@chainsafe/as-sha256": "^0.4.1",
"@chainsafe/bls": "7.1.3",
"@chainsafe/blst": "^0.2.11",
"@chainsafe/blst": "^2.0.1",
"@chainsafe/discv5": "^9.0.0",
"@chainsafe/enr": "^3.0.0",
"@chainsafe/libp2p-gossipsub": "^13.0.0",

View File

@@ -1,4 +1,4 @@
import {PublicKey} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {ISignatureSet} from "@lodestar/state-transition";
export type VerifySignatureOpts = {

View File

@@ -1,5 +1,4 @@
import {CoordType, PublicKey} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {PublicKey, Signature, verify, verifyMultipleAggregateSignatures} from "@chainsafe/blst";
const MIN_SET_COUNT_TO_BATCH = 2;
@@ -16,12 +15,12 @@ export type SignatureSetDeserialized = {
export function verifySignatureSetsMaybeBatch(sets: SignatureSetDeserialized[]): boolean {
try {
if (sets.length >= MIN_SET_COUNT_TO_BATCH) {
return bls.Signature.verifyMultipleSignatures(
return verifyMultipleAggregateSignatures(
sets.map((s) => ({
publicKey: s.publicKey,
message: s.message,
pk: s.publicKey,
msg: s.message,
// true = validate signature
signature: bls.Signature.fromBytes(s.signature, CoordType.affine, true),
sig: Signature.fromBytes(s.signature, true),
}))
);
}
@@ -34,8 +33,8 @@ export function verifySignatureSetsMaybeBatch(sets: SignatureSetDeserialized[]):
// If too few signature sets verify them without batching
return sets.every((set) => {
// true = validate signature
const sig = bls.Signature.fromBytes(set.signature, CoordType.affine, true);
return sig.verify(set.publicKey, set.message);
const sig = Signature.fromBytes(set.signature, true);
return verify(set.message, set.publicKey, sig);
});
} catch (_) {
// A signature could be malformed, in that case fromBytes throws error

View File

@@ -7,8 +7,7 @@ import {spawn, Worker} from "@chainsafe/threads";
// @ts-ignore
// eslint-disable-next-line
self = undefined;
import bls from "@chainsafe/bls";
import {Implementation, PointFormat, PublicKey} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {Logger} from "@lodestar/utils";
import {ISignatureSet} from "@lodestar/state-transition";
import {QueueError, QueueErrorCode} from "../../../util/queue/index.js";
@@ -116,7 +115,6 @@ export class BlsMultiThreadWorkerPool implements IBlsVerifier {
private readonly logger: Logger;
private readonly metrics: Metrics | null;
private readonly format: PointFormat;
private readonly workers: WorkerDescriptor[];
private readonly jobs = new LinkedList<JobQueueItem>();
private bufferedJobs: {
@@ -136,14 +134,10 @@ export class BlsMultiThreadWorkerPool implements IBlsVerifier {
this.metrics = metrics;
this.blsVerifyAllMultiThread = options.blsVerifyAllMultiThread ?? false;
// TODO: Allow to customize implementation
const implementation = bls.implementation;
// Use compressed for herumi for now.
// THe worker is not able to deserialize from uncompressed
// `Error: err _wrapDeserialize`
this.format = implementation === "blst-native" ? PointFormat.uncompressed : PointFormat.compressed;
this.workers = this.createWorkers(implementation, blsPoolSize);
this.workers = this.createWorkers(blsPoolSize);
if (metrics) {
metrics.blsThreadPool.queueLength.addCollect(() => {
@@ -265,11 +259,11 @@ export class BlsMultiThreadWorkerPool implements IBlsVerifier {
);
}
private createWorkers(implementation: Implementation, poolSize: number): WorkerDescriptor[] {
private createWorkers(poolSize: number): WorkerDescriptor[] {
const workers: WorkerDescriptor[] = [];
for (let i = 0; i < poolSize; i++) {
const workerData: WorkerData = {implementation, workerId: i};
const workerData: WorkerData = {workerId: i};
const worker = new Worker(path.join(workerDir, "worker.js"), {
workerData,
} as ConstructorParameters<typeof Worker>[1]);
@@ -400,7 +394,7 @@ export class BlsMultiThreadWorkerPool implements IBlsVerifier {
try {
// Note: This can throw, must be handled per-job.
// Pubkey and signature aggregation is defered here
workReq = jobItemWorkReq(job, this.format, this.metrics);
workReq = jobItemWorkReq(job, this.metrics);
} catch (e) {
this.metrics?.blsThreadPool.errorAggregateSignatureSetsCount.inc({type: job.type});

View File

@@ -1,5 +1,4 @@
import bls from "@chainsafe/bls";
import {CoordType, PointFormat, PublicKey} from "@chainsafe/bls/types";
import {PublicKey, aggregateWithRandomness} from "@chainsafe/blst";
import {ISignatureSet, SignatureSetType} from "@lodestar/state-transition";
import {VerifySignatureOpts} from "../interface.js";
import {getAggregatedPubkey} from "../utils.js";
@@ -49,36 +48,37 @@ export function jobItemSigSets(job: JobQueueItem): number {
* Prepare BlsWorkReq from JobQueueItem
* WARNING: May throw with untrusted user input
*/
export function jobItemWorkReq(job: JobQueueItem, format: PointFormat, metrics: Metrics | null): BlsWorkReq {
export function jobItemWorkReq(job: JobQueueItem, metrics: Metrics | null): BlsWorkReq {
switch (job.type) {
case JobQueueItemType.default:
return {
opts: job.opts,
sets: job.sets.map((set) => ({
// this can throw, handled in the consumer code
publicKey: getAggregatedPubkey(set, metrics).toBytes(format),
publicKey: getAggregatedPubkey(set, metrics).toBytes(),
signature: set.signature,
message: set.signingRoot,
})),
};
case JobQueueItemType.sameMessage: {
// validate signature = true, this is slow code on main thread so should only run with network thread mode (useWorker=true)
// For a node subscribing to all subnets, with 1 signature per validator per epoch it takes around 80s
// to deserialize 750_000 signatures per epoch
// This is slow code on main thread (mainly signature deserialization + group check).
// Ideally it can be taken off-thread, but in the mean time, keep track of total time spent here.
// As of July 2024, for a node subscribing to all subnets, with 1 signature per validator per epoch,
// it takes around 2.02 min to perform this operation for a single epoch.
// cpu profile on main thread has 250s idle so this only works until we reach 3M validators
// However, for normal node with only 2 to 7 subnet subscriptions per epoch this works until 27M validators
// and not a problem in the near future
// this is monitored on v1.11.0 https://github.com/ChainSafe/lodestar/pull/5912#issuecomment-1700320307
const timer = metrics?.blsThreadPool.signatureDeserializationMainThreadDuration.startTimer();
const signatures = job.sets.map((set) => bls.Signature.fromBytes(set.signature, CoordType.affine, true));
// this is monitored on v1.21.0 https://github.com/ChainSafe/lodestar/pull/6894/files#r1687359225
const timer = metrics?.blsThreadPool.aggregateWithRandomnessMainThreadDuration.startTimer();
const {pk, sig} = aggregateWithRandomness(job.sets.map((set) => ({pk: set.publicKey, sig: set.signature})));
timer?.();
return {
opts: job.opts,
sets: [
{
publicKey: bls.PublicKey.aggregate(job.sets.map((set) => set.publicKey)).toBytes(format),
signature: bls.Signature.aggregate(signatures).toBytes(format),
publicKey: pk.toBytes(),
signature: sig.toBytes(),
message: job.message,
},
],

View File

@@ -1,7 +1,6 @@
import {VerifySignatureOpts} from "../interface.js";
export type WorkerData = {
implementation: "herumi" | "blst-native";
workerId: number;
};

View File

@@ -1,8 +1,7 @@
/* eslint-disable @typescript-eslint/strict-boolean-expressions */
import worker from "node:worker_threads";
import {expose} from "@chainsafe/threads/worker";
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {verifySignatureSetsMaybeBatch, SignatureSetDeserialized} from "../maybeBatch.js";
import {WorkerData, BlsWorkReq, WorkResult, WorkResultCode, SerializedSet, BlsWorkResult} from "./types.js";
import {chunkifyMaximizeChunkSize} from "./utils.js";
@@ -109,7 +108,7 @@ function verifyManySignatureSets(workReqArr: BlsWorkReq[]): BlsWorkResult {
function deserializeSet(set: SerializedSet): SignatureSetDeserialized {
return {
publicKey: bls.PublicKey.fromBytes(set.publicKey, CoordType.affine),
publicKey: PublicKey.fromBytes(set.publicKey),
message: set.message,
signature: set.signature,
};

View File

@@ -1,6 +1,4 @@
import {PublicKey, Signature} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/blst";
import {PublicKey, Signature, aggregatePublicKeys, aggregateSignatures, verify} from "@chainsafe/blst";
import {ISignatureSet} from "@lodestar/state-transition";
import {Metrics} from "../../metrics/index.js";
import {IBlsVerifier} from "./interface.js";
@@ -40,12 +38,12 @@ export class BlsSingleThreadVerifier implements IBlsVerifier {
message: Uint8Array
): Promise<boolean[]> {
const timer = this.metrics?.blsThreadPool.mainThreadDurationInThreadPool.startTimer();
const pubkey = bls.PublicKey.aggregate(sets.map((set) => set.publicKey));
const pubkey = aggregatePublicKeys(sets.map((set) => set.publicKey));
let isAllValid = true;
// validate signature = true
const signatures = sets.map((set) => {
try {
return bls.Signature.fromBytes(set.signature, CoordType.affine, true);
return Signature.fromBytes(set.signature, true);
} catch (_) {
// at least one set has malformed signature
isAllValid = false;
@@ -54,8 +52,8 @@ export class BlsSingleThreadVerifier implements IBlsVerifier {
});
if (isAllValid) {
const signature = bls.Signature.aggregate(signatures as Signature[]);
isAllValid = signature.verify(pubkey, message);
const signature = aggregateSignatures(signatures as Signature[]);
isAllValid = verify(message, pubkey, signature);
}
let result: boolean[];
@@ -67,7 +65,7 @@ export class BlsSingleThreadVerifier implements IBlsVerifier {
if (sig === null) {
return false;
}
return sig.verify(set.publicKey, message);
return verify(message, set.publicKey, sig);
});
}

View File

@@ -1,5 +1,4 @@
import type {PublicKey} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {PublicKey, aggregatePublicKeys} from "@chainsafe/blst";
import {ISignatureSet, SignatureSetType} from "@lodestar/state-transition";
import {Metrics} from "../../metrics/metrics.js";
@@ -10,7 +9,7 @@ export function getAggregatedPubkey(signatureSet: ISignatureSet, metrics: Metric
case SignatureSetType.aggregate: {
const timer = metrics?.blsThreadPool.pubkeysAggregationMainThreadDuration.startTimer();
const pubkeys = bls.PublicKey.aggregate(signatureSet.pubkeys);
const pubkeys = aggregatePublicKeys(signatureSet.pubkeys);
timer?.();
return pubkeys;
}

View File

@@ -1,5 +1,5 @@
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {aggregateSignatures} from "@chainsafe/blst";
import {ForkName, ForkSeq, MAX_ATTESTATIONS, MIN_ATTESTATION_INCLUSION_DELAY, SLOTS_PER_EPOCH} from "@lodestar/params";
import {phase0, Epoch, Slot, ssz, ValidatorIndex, RootHex} from "@lodestar/types";
import {
@@ -383,7 +383,7 @@ export function aggregateInto(attestation1: AttestationWithIndex, attestation2:
const signature1 = signatureFromBytesNoCheck(attestation1.attestation.signature);
const signature2 = signatureFromBytesNoCheck(attestation2.attestation.signature);
attestation1.attestation.signature = bls.Signature.aggregate([signature1, signature2]).toBytes();
attestation1.attestation.signature = aggregateSignatures([signature1, signature2]).toBytes();
}
/**

View File

@@ -1,6 +1,5 @@
import {PointFormat, Signature} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {BitArray} from "@chainsafe/ssz";
import {Signature, aggregateSignatures} from "@chainsafe/blst";
import {phase0, Slot, RootHex} from "@lodestar/types";
import {MapDef} from "@lodestar/utils";
import {IClock} from "../../util/clock.js";
@@ -191,10 +190,7 @@ function aggregateAttestationInto(aggregate: AggregateFast, attestation: phase0.
}
aggregate.aggregationBits.set(bitIndex, true);
aggregate.signature = bls.Signature.aggregate([
aggregate.signature,
signatureFromBytesNoCheck(attestation.signature),
]);
aggregate.signature = aggregateSignatures([aggregate.signature, signatureFromBytesNoCheck(attestation.signature)]);
return InsertOutcome.Aggregated;
}
@@ -217,6 +213,6 @@ function fastToAttestation(aggFast: AggregateFast): phase0.Attestation {
return {
data: aggFast.data,
aggregationBits: aggFast.aggregationBits,
signature: aggFast.signature.toBytes(PointFormat.compressed),
signature: aggFast.signature.toBytes(),
};
}

View File

@@ -1,6 +1,5 @@
import {PointFormat, Signature} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {BitArray, toHexString} from "@chainsafe/ssz";
import {Signature, aggregateSignatures} from "@chainsafe/blst";
import {SYNC_COMMITTEE_SIZE, SYNC_COMMITTEE_SUBNET_COUNT} from "@lodestar/params";
import {altair, Root, Slot, SubcommitteeIndex} from "@lodestar/types";
import {MapDef} from "@lodestar/utils";
@@ -108,7 +107,7 @@ export class SyncCommitteeMessagePool {
return {
...contribution,
aggregationBits: contribution.aggregationBits,
signature: contribution.signature.toBytes(PointFormat.compressed),
signature: contribution.signature.toBytes(),
};
}
@@ -136,7 +135,7 @@ function aggregateSignatureInto(
}
contribution.aggregationBits.set(indexInSubcommittee, true);
contribution.signature = bls.Signature.aggregate([
contribution.signature = aggregateSignatures([
contribution.signature,
signatureFromBytesNoCheck(signature.signature),
]);

View File

@@ -1,6 +1,5 @@
import type {Signature} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {BitArray, toHexString} from "@chainsafe/ssz";
import {Signature, aggregateSignatures} from "@chainsafe/blst";
import {SYNC_COMMITTEE_SIZE, SYNC_COMMITTEE_SUBNET_SIZE} from "@lodestar/params";
import {altair, Slot, Root, ssz} from "@lodestar/types";
import {G2_POINT_AT_INFINITY} from "@lodestar/state-transition";
@@ -182,6 +181,6 @@ export function aggregate(bestContributionBySubnet: Map<number, SyncContribution
}
return {
syncCommitteeBits,
syncCommitteeSignature: bls.Signature.aggregate(signatures).toBytes(),
syncCommitteeSignature: aggregateSignatures(signatures).toBytes(),
};
}

View File

@@ -1,5 +1,4 @@
import bls from "@chainsafe/bls";
import {CoordType, Signature} from "@chainsafe/bls/types";
import {Signature} from "@chainsafe/blst";
import {BLS_WITHDRAWAL_PREFIX} from "@lodestar/params";
import {CachedBeaconStateAllForks} from "@lodestar/state-transition";
import {Slot, capella} from "@lodestar/types";
@@ -30,7 +29,7 @@ export function pruneBySlot(map: Map<Slot, unknown>, slot: Slot, slotsRetained:
* No need to verify Signature is valid, already run sig-verify = false
*/
export function signatureFromBytesNoCheck(signature: Uint8Array): Signature {
return bls.Signature.fromBytes(signature, CoordType.affine, false);
return Signature.fromBytes(signature);
}
/**

View File

@@ -1,4 +1,4 @@
import type {PublicKey} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {DOMAIN_AGGREGATE_AND_PROOF} from "@lodestar/params";
import {ssz} from "@lodestar/types";
import {Epoch, phase0} from "@lodestar/types";

View File

@@ -1,4 +1,4 @@
import type {PublicKey} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {DOMAIN_SELECTION_PROOF} from "@lodestar/params";
import {phase0, Slot, ssz} from "@lodestar/types";
import {computeSigningRoot, createSingleSignatureSetFromComponents, ISignatureSet} from "@lodestar/state-transition";

View File

@@ -1,4 +1,4 @@
import type {PublicKey} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {altair, ssz} from "@lodestar/types";
import {DOMAIN_SYNC_COMMITTEE} from "@lodestar/params";
import {CachedBeaconStateAltair, computeSigningRoot, ISignatureSet, SignatureSetType} from "@lodestar/state-transition";

View File

@@ -482,9 +482,9 @@ export function createLodestarMetrics(
name: "lodestar_bls_thread_pool_batchable_sig_sets_total",
help: "Count of total batchable signature sets",
}),
signatureDeserializationMainThreadDuration: register.histogram({
name: "lodestar_bls_thread_pool_signature_deserialization_main_thread_time_seconds",
help: "Total time spent deserializing signatures on main thread",
aggregateWithRandomnessMainThreadDuration: register.histogram({
name: "lodestar_bls_thread_pool_aggregate_with_randomness_main_thread_time_seconds",
help: "Total time performing aggregateWithRandomness on main thread",
buckets: [0.001, 0.005, 0.01, 0.1],
}),
pubkeysAggregationMainThreadDuration: register.histogram({

View File

@@ -1,5 +1,5 @@
import {describe, it, beforeEach, afterEach, expect} from "vitest";
import bls from "@chainsafe/bls";
import {aggregateSerializedPublicKeys} from "@chainsafe/blst";
import {createBeaconConfig, ChainConfig} from "@lodestar/config";
import {chainConfig as chainConfigDef} from "@lodestar/config/default";
import {getClient, HttpHeader, routes} from "@lodestar/api";
@@ -129,7 +129,7 @@ describe("lightclient api", function () {
const committeePubkeys = Array.from({length: SYNC_COMMITTEE_SIZE}, (_, i) =>
i % 2 === 0 ? pubkeys[0] : pubkeys[1]
);
const aggregatePubkey = bls.aggregatePublicKeys(committeePubkeys);
const aggregatePubkey = aggregateSerializedPublicKeys(committeePubkeys).toBytes();
// single committee hash since we requested for the first period
expect(committeeRes.value()).toEqual([
ssz.altair.SyncCommittee.hashTreeRoot({

View File

@@ -1,6 +1,5 @@
import {describe, it, beforeAll, expect, beforeEach, afterEach} from "vitest";
import bls from "@chainsafe/bls";
import {PublicKey} from "@chainsafe/bls/types";
import {PublicKey, SecretKey} from "@chainsafe/blst";
import {ISignatureSet, SignatureSetType} from "@lodestar/state-transition";
import {BlsMultiThreadWorkerPool} from "../../../../src/chain/bls/multithread/index.js";
import {testLogger} from "../../../utils/logger.js";
@@ -29,7 +28,7 @@ describe("chain / bls / multithread queue", function () {
beforeAll(() => {
for (let i = 0; i < 3; i++) {
const sk = bls.SecretKey.fromBytes(Buffer.alloc(32, i + 1));
const sk = SecretKey.fromBytes(Buffer.alloc(32, i + 1));
const msg = Buffer.alloc(32, i + 1);
const pk = sk.toPublicKey();
const sig = sk.sign(msg);

View File

@@ -1,4 +1,4 @@
import {PublicKey} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {IBlsVerifier} from "../../src/chain/bls/index.js";
export class BlsVerifierMock implements IBlsVerifier {

View File

@@ -1,5 +1,4 @@
import {itBench} from "@dapplion/benchmark";
import {PointFormat} from "@chainsafe/bls/types";
// eslint-disable-next-line import/no-relative-packages
import {generatePerfTestCachedStatePhase0, numValidators} from "../../../../../../state-transition/test/perf/util.js";
import {getPubkeysForIndices} from "../../../../../src/api/impl/validator/utils.js";
@@ -36,7 +35,7 @@ describe("api / impl / validator", () => {
fn: () => {
for (let i = 0; i < reqCount; i++) {
const pubkey = state.epochCtx.index2pubkey[i];
pubkey.toBytes(PointFormat.compressed);
pubkey.toBytes();
}
},
});

View File

@@ -1,7 +1,14 @@
import crypto from "node:crypto";
import {itBench} from "@dapplion/benchmark";
import bls from "@chainsafe/bls";
import {CoordType, type PublicKey, type SecretKey} from "@chainsafe/bls/types";
import {
PublicKey,
SecretKey,
Signature,
aggregatePublicKeys,
aggregateSignatures,
verify,
verifyMultipleAggregateSignatures,
} from "@chainsafe/blst";
import {linspace} from "../../../src/util/numpy.js";
describe("BLS ops", function () {
@@ -20,7 +27,7 @@ describe("BLS ops", function () {
const bytes = new Uint8Array(32);
const dataView = new DataView(bytes.buffer, bytes.byteOffset, bytes.byteLength);
dataView.setUint32(0, i + 1, true);
const secretKey = bls.SecretKey.fromBytes(bytes);
const secretKey = SecretKey.fromKeygen(bytes);
const publicKey = secretKey.toPublicKey();
keypair = {secretKey, publicKey};
keypairs.set(i, keypair);
@@ -53,8 +60,8 @@ describe("BLS ops", function () {
}
// Note: getSet() caches the value, does not re-compute every time
itBench({id: `BLS verify - ${bls.implementation}`, beforeEach: () => getSet(0)}, (set) => {
const isValid = bls.Signature.fromBytes(set.signature).verify(set.publicKey, set.message);
itBench({id: "BLS verify - blst", beforeEach: () => getSet(0)}, (set) => {
const isValid = verify(set.message, set.publicKey, Signature.fromBytes(set.signature));
if (!isValid) throw Error("Invalid");
});
@@ -62,14 +69,14 @@ describe("BLS ops", function () {
// We may want to bundle up to 32 sets in a single batch.
for (const count of [3, 8, 32, 64, 128]) {
itBench({
id: `BLS verifyMultipleSignatures ${count} - ${bls.implementation}`,
id: `BLS verifyMultipleSignatures ${count} - blst`,
beforeEach: () => linspace(0, count - 1).map((i) => getSet(i)),
fn: (sets) => {
const isValid = bls.Signature.verifyMultipleSignatures(
const isValid = verifyMultipleAggregateSignatures(
sets.map((set) => ({
publicKey: set.publicKey,
message: set.message,
signature: bls.Signature.fromBytes(set.signature),
pk: set.publicKey,
msg: set.message,
sig: Signature.fromBytes(set.signature),
}))
);
if (!isValid) throw Error("Invalid");
@@ -86,7 +93,7 @@ describe("BLS ops", function () {
fn: () => {
for (const signature of signatures) {
// true = validate signature
bls.Signature.fromBytes(signature, CoordType.affine, true);
Signature.fromBytes(signature, true);
}
},
});
@@ -97,15 +104,13 @@ describe("BLS ops", function () {
// TODO: figure out why it does not work with 256 or more
for (const count of [3, 8, 32, 64, 128]) {
itBench({
id: `BLS verifyMultipleSignatures - same message - ${count} - ${bls.implementation}`,
id: `BLS verifyMultipleSignatures - same message - ${count} - blst`,
beforeEach: () => linspace(0, count - 1).map((i) => getSetSameMessage(i)),
fn: (sets) => {
// aggregate and verify aggregated signatures
const aggregatedPubkey = bls.PublicKey.aggregate(sets.map((set) => set.publicKey));
const aggregatedSignature = bls.Signature.aggregate(
sets.map((set) => bls.Signature.fromBytes(set.signature, CoordType.affine, false))
);
const isValid = aggregatedSignature.verify(aggregatedPubkey, sets[0].message);
const aggregatedPubkey = aggregatePublicKeys(sets.map((set) => set.publicKey));
const aggregatedSignature = aggregateSignatures(sets.map((set) => Signature.fromBytes(set.signature)));
const isValid = verify(sets[0].message, aggregatedPubkey, aggregatedSignature);
if (!isValid) throw Error("Invalid");
},
});
@@ -114,10 +119,10 @@ describe("BLS ops", function () {
// Attestations in Mainnet contain 128 max on average
for (const count of [32, 128]) {
itBench({
id: `BLS aggregatePubkeys ${count} - ${bls.implementation}`,
id: `BLS aggregatePubkeys ${count} - blst`,
beforeEach: () => linspace(0, count - 1).map((i) => getKeypair(i).publicKey),
fn: (pubkeys) => {
bls.PublicKey.aggregate(pubkeys);
aggregatePublicKeys(pubkeys);
},
});
}

View File

@@ -1,7 +1,14 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/bls/types";
import {
PublicKey,
SecretKey,
Signature,
aggregateSignatures,
aggregateVerify,
fastAggregateVerify,
verifyMultipleAggregateSignatures,
verify as _verify,
} from "@chainsafe/blst";
import {fromHexString} from "@chainsafe/ssz";
import {toHexString} from "@lodestar/utils";
/* eslint-disable @typescript-eslint/naming-convention */
@@ -31,7 +38,15 @@ export const testFnByType: Record<string, "skip" | ((data: any) => any)> = {
*/
function aggregate_verify(input: {pubkeys: string[]; messages: string[]; signature: string}): boolean {
const {pubkeys, messages, signature} = input;
return bls.verifyMultiple(pubkeys.map(fromHexString), messages.map(fromHexString), fromHexString(signature));
try {
return aggregateVerify(
messages.map(fromHexString),
pubkeys.map((pk) => PublicKey.fromHex(pk)),
Signature.fromHex(signature)
);
} catch (e) {
return false;
}
}
/**
@@ -41,8 +56,8 @@ function aggregate_verify(input: {pubkeys: string[]; messages: string[]; signatu
* ```
*/
function aggregate(input: string[]): string {
const pks = input.map((pkHex) => bls.Signature.fromHex(pkHex));
const agg = bls.Signature.aggregate(pks);
const pks = input.map((pkHex) => Signature.fromHex(pkHex));
const agg = aggregateSignatures(pks);
return agg.toHex();
}
@@ -58,9 +73,10 @@ function aggregate(input: string[]): string {
function fast_aggregate_verify(input: {pubkeys: string[]; message: string; signature: string}): boolean | null {
const {pubkeys, message, signature} = input;
try {
return bls.Signature.fromBytes(fromHexString(signature), undefined, true).verifyAggregate(
pubkeys.map((hex) => bls.PublicKey.fromBytes(fromHexString(hex), CoordType.jacobian, true)),
fromHexString(message)
return fastAggregateVerify(
fromHexString(message),
pubkeys.map((hex) => PublicKey.fromHex(hex, true)),
Signature.fromHex(signature, true)
);
} catch (e) {
return false;
@@ -80,11 +96,11 @@ function fast_aggregate_verify(input: {pubkeys: string[]; message: string; signa
function batch_verify(input: {pubkeys: string[]; messages: string[]; signatures: string[]}): boolean | null {
const {pubkeys, messages, signatures} = input;
try {
return bls.Signature.verifyMultipleSignatures(
return verifyMultipleAggregateSignatures(
pubkeys.map((pubkey, i) => ({
publicKey: bls.PublicKey.fromBytes(fromHexString(pubkey), CoordType.jacobian, true),
message: fromHexString(messages[i]),
signature: bls.Signature.fromBytes(fromHexString(signatures[i]), undefined, true),
pk: PublicKey.fromHex(pubkey, true),
msg: fromHexString(messages[i]),
sig: Signature.fromHex(signatures[i], true),
}))
);
} catch (e) {
@@ -103,8 +119,8 @@ function batch_verify(input: {pubkeys: string[]; messages: string[]; signatures:
*/
function sign(input: {privkey: string; message: string}): string | null {
const {privkey, message} = input;
const signature = bls.sign(fromHexString(privkey), fromHexString(message));
return toHexString(signature);
const signature = SecretKey.fromHex(privkey).sign(fromHexString(message));
return signature.toHex();
}
/**
@@ -119,7 +135,11 @@ function sign(input: {privkey: string; message: string}): string | null {
*/
function verify(input: {pubkey: string; message: string; signature: string}): boolean {
const {pubkey, message, signature} = input;
return bls.verify(fromHexString(pubkey), fromHexString(message), fromHexString(signature));
try {
return _verify(fromHexString(message), PublicKey.fromHex(pubkey), Signature.fromHex(signature));
} catch (e) {
return false;
}
}
/**
@@ -131,7 +151,7 @@ function verify(input: {pubkey: string; message: string; signature: string}): bo
*/
function deserialization_G1(input: {pubkey: string}): boolean {
try {
bls.PublicKey.fromBytes(fromHexString(input.pubkey), CoordType.jacobian, true);
PublicKey.fromHex(input.pubkey, true);
return true;
} catch (e) {
return false;
@@ -147,7 +167,7 @@ function deserialization_G1(input: {pubkey: string}): boolean {
*/
function deserialization_G2(input: {signature: string}): boolean {
try {
bls.Signature.fromBytes(fromHexString(input.signature), undefined, true);
Signature.fromHex(input.signature, true);
return true;
} catch (e) {
return false;

View File

@@ -1,8 +1,15 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/bls/types";
import {fromHexString} from "@chainsafe/ssz";
import {
PublicKey,
SecretKey,
Signature,
aggregateSerializedPublicKeys,
aggregateSignatures,
aggregateVerify,
fastAggregateVerify,
verify as _verify,
} from "@chainsafe/blst";
import {InputType} from "@lodestar/spec-test-util";
import {toHexString} from "@lodestar/utils";
import {TestRunnerFn} from "../utils/types.js";
/* eslint-disable @typescript-eslint/naming-convention */
@@ -63,10 +70,14 @@ type BlsTestCase = {
* output: BLS Signature -- expected output, single BLS signature or empty.
* ```
*/
function aggregate(input: string[]): string {
const pks = input.map((pkHex) => bls.Signature.fromHex(pkHex));
const agg = bls.Signature.aggregate(pks);
return agg.toHex();
function aggregate(input: string[]): string | null {
try {
const pks = input.map((pkHex) => Signature.fromHex(pkHex));
const agg = aggregateSignatures(pks);
return agg.toHex();
} catch (e) {
return null;
}
}
/**
@@ -80,7 +91,15 @@ function aggregate(input: string[]): string {
*/
function aggregate_verify(input: {pubkeys: string[]; messages: string[]; signature: string}): boolean {
const {pubkeys, messages, signature} = input;
return bls.verifyMultiple(pubkeys.map(fromHexString), messages.map(fromHexString), fromHexString(signature));
try {
return aggregateVerify(
messages.map(fromHexString),
pubkeys.map((pk) => PublicKey.fromHex(pk)),
Signature.fromHex(signature)
);
} catch (e) {
return false;
}
}
/**
@@ -95,8 +114,11 @@ function eth_aggregate_pubkeys(input: string[]): string | null {
if (pk === G1_POINT_AT_INFINITY) return null;
}
const agg = bls.aggregatePublicKeys(input.map((hex) => fromHexString(hex)));
return toHexString(agg);
try {
return aggregateSerializedPublicKeys(input.map((hex) => fromHexString(hex))).toHex();
} catch (e) {
return null;
}
}
/**
@@ -120,11 +142,15 @@ function eth_fast_aggregate_verify(input: {pubkeys: string[]; message: string; s
if (pk === G1_POINT_AT_INFINITY) return false;
}
return bls.verifyAggregate(
pubkeys.map((hex) => fromHexString(hex)),
fromHexString(message),
fromHexString(signature)
);
try {
return fastAggregateVerify(
fromHexString(message),
pubkeys.map((hex) => PublicKey.fromHex(hex)),
Signature.fromHex(signature)
);
} catch (e) {
return false;
}
}
/**
@@ -139,9 +165,10 @@ function eth_fast_aggregate_verify(input: {pubkeys: string[]; message: string; s
function fast_aggregate_verify(input: {pubkeys: string[]; message: string; signature: string}): boolean | null {
const {pubkeys, message, signature} = input;
try {
return bls.Signature.fromBytes(fromHexString(signature), undefined, true).verifyAggregate(
pubkeys.map((hex) => bls.PublicKey.fromBytes(fromHexString(hex), CoordType.jacobian, true)),
fromHexString(message)
return fastAggregateVerify(
fromHexString(message),
pubkeys.map((hex) => PublicKey.fromHex(hex, true)),
Signature.fromHex(signature, true)
);
} catch (e) {
return false;
@@ -156,8 +183,11 @@ function fast_aggregate_verify(input: {pubkeys: string[]; message: string; signa
*/
function sign(input: {privkey: string; message: string}): string | null {
const {privkey, message} = input;
const signature = bls.sign(fromHexString(privkey), fromHexString(message));
return toHexString(signature);
try {
return SecretKey.fromHex(privkey).sign(fromHexString(message)).toHex();
} catch (e) {
return null;
}
}
/**
@@ -169,5 +199,9 @@ function sign(input: {privkey: string; message: string}): string | null {
*/
function verify(input: {pubkey: string; message: string; signature: string}): boolean {
const {pubkey, message, signature} = input;
return bls.verify(fromHexString(pubkey), fromHexString(message), fromHexString(signature));
try {
return _verify(fromHexString(message), PublicKey.fromHex(pubkey), Signature.fromHex(signature));
} catch (e) {
return false;
}
}

View File

@@ -1,5 +1,4 @@
import {expect} from "vitest";
import {init} from "@chainsafe/bls/switchable";
import {isForkLightClient} from "@lodestar/params";
import {altair, phase0, RootHex, Slot, ssz, sszTypesFor} from "@lodestar/types";
import {InputType} from "@lodestar/spec-test-util";
@@ -78,8 +77,6 @@ const UPDATE_FILE_NAME = "^(update)_([0-9a-zA-Z_]+)$";
export const sync: TestRunnerFn<SyncTestCase, void> = (fork) => {
return {
testFunction: async (testcase) => {
await init("blst-native");
// Grab only the ALTAIR_FORK_EPOCH, since the domains are the same as minimal
const config = createBeaconConfig(
pickConfigForkEpochs(testcase.config),

View File

@@ -1,7 +1,5 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/blst";
import {PublicKey} from "@chainsafe/bls/types";
import {describe, it, expect, beforeEach} from "vitest";
import {PublicKey, SecretKey, Signature} from "@chainsafe/blst";
import {ISignatureSet, SignatureSetType} from "@lodestar/state-transition";
import {BlsSingleThreadVerifier} from "../../../../src/chain/bls/singleThread.js";
import {BlsMultiThreadWorkerPool} from "../../../../src/chain/bls/multithread/index.js";
@@ -10,7 +8,7 @@ import {testLogger} from "../../../utils/logger.js";
describe("BlsVerifier ", function () {
// take time for creating thread pool
const numKeys = 3;
const secretKeys = Array.from({length: numKeys}, (_, i) => bls.SecretKey.fromKeygen(Buffer.alloc(32, i)));
const secretKeys = Array.from({length: numKeys}, (_, i) => SecretKey.fromKeygen(Buffer.alloc(32, i)));
const verifiers = [
new BlsSingleThreadVerifier({metrics: null}),
new BlsMultiThreadWorkerPool({}, {metrics: null, logger: testLogger()}),
@@ -46,7 +44,7 @@ describe("BlsVerifier ", function () {
it("should return false if at least one signature is malformed", async () => {
// signature is malformed
const malformedSignature = Buffer.alloc(96, 10);
expect(() => bls.Signature.fromBytes(malformedSignature, CoordType.affine, true)).toThrow();
expect(() => Signature.fromBytes(malformedSignature, true, true)).toThrow();
sets[1].signature = malformedSignature;
expect(await verifier.verifySignatureSets(sets)).toBe(false);
});
@@ -79,7 +77,7 @@ describe("BlsVerifier ", function () {
it("should return false for malformed signature", async () => {
// signature is malformed
const malformedSignature = Buffer.alloc(96, 10);
expect(() => bls.Signature.fromBytes(malformedSignature, CoordType.affine, true)).toThrow();
expect(() => Signature.fromBytes(malformedSignature, true, true)).toThrow();
sets[1].signature = malformedSignature;
expect(await verifier.verifySignatureSetsSameMessage(sets, signingRoot)).toEqual([true, false, true]);
});

View File

@@ -1,7 +1,7 @@
/* eslint-disable @typescript-eslint/naming-convention */
import type {SecretKey, PublicKey} from "@chainsafe/bls/types";
import {toHexString} from "@chainsafe/ssz";
import {describe, it, expect} from "vitest";
import {PublicKey, SecretKey} from "@chainsafe/blst";
import {DOMAIN_DEPOSIT, MAX_EFFECTIVE_BALANCE} from "@lodestar/params";
import {config} from "@lodestar/config/default";
import {computeDomain, computeSigningRoot, interopSecretKey, ZERO_HASH} from "@lodestar/state-transition";

View File

@@ -1,7 +1,6 @@
import type {SecretKey} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {BitArray, fromHexString, toHexString} from "@chainsafe/ssz";
import {describe, it, expect, beforeEach, beforeAll, afterEach, vi} from "vitest";
import {SecretKey, Signature, fastAggregateVerify} from "@chainsafe/blst";
import {CachedBeaconStateAllForks, newFilledArray} from "@lodestar/state-transition";
import {FAR_FUTURE_EPOCH, ForkName, MAX_EFFECTIVE_BALANCE, SLOTS_PER_EPOCH} from "@lodestar/params";
import {ssz, phase0} from "@lodestar/types";
@@ -303,8 +302,8 @@ describe("MatchingDataAttestationGroup aggregateInto", function () {
let sk2: SecretKey;
beforeAll(async () => {
sk1 = bls.SecretKey.fromBytes(Buffer.alloc(32, 1));
sk2 = bls.SecretKey.fromBytes(Buffer.alloc(32, 2));
sk1 = SecretKey.fromBytes(Buffer.alloc(32, 1));
sk2 = SecretKey.fromBytes(Buffer.alloc(32, 2));
attestation1.signature = sk1.sign(attestationDataRoot).toBytes();
attestation2.signature = sk2.sign(attestationDataRoot).toBytes();
});
@@ -315,7 +314,9 @@ describe("MatchingDataAttestationGroup aggregateInto", function () {
aggregateInto(attWithIndex1, attWithIndex2);
expect(renderBitArray(attWithIndex1.attestation.aggregationBits)).toEqual(renderBitArray(mergedBitArray));
const aggregatedSignature = bls.Signature.fromBytes(attWithIndex1.attestation.signature, undefined, true);
expect(aggregatedSignature.verifyAggregate([sk1.toPublicKey(), sk2.toPublicKey()], attestationDataRoot)).toBe(true);
const aggregatedSignature = Signature.fromBytes(attWithIndex1.attestation.signature, true, true);
expect(fastAggregateVerify(attestationDataRoot, [sk1.toPublicKey(), sk2.toPublicKey()], aggregatedSignature)).toBe(
true
);
});
});

View File

@@ -1,6 +1,6 @@
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {describe, it, expect, beforeEach, beforeAll, afterEach, vi, MockedObject} from "vitest";
import {SecretKey} from "@chainsafe/blst";
import {altair} from "@lodestar/types";
import {SyncCommitteeMessagePool} from "../../../../src/chain/opPools/index.js";
import {Clock} from "../../../../src/util/clock.js";
@@ -18,7 +18,7 @@ describe("chain / opPools / SyncCommitteeMessagePool", function () {
const cutOffTime = 1;
beforeAll(async () => {
const sk = bls.SecretKey.fromBytes(Buffer.alloc(32, 1));
const sk = SecretKey.fromBytes(Buffer.alloc(32, 1));
syncCommittee = {
slot,
beaconBlockRoot,
@@ -42,7 +42,7 @@ describe("chain / opPools / SyncCommitteeMessagePool", function () {
clockStub.secFromSlot.mockReturnValue(0);
let contribution = cache.getContribution(subcommitteeIndex, syncCommittee.slot, syncCommittee.beaconBlockRoot);
expect(contribution).not.toBeNull();
const newSecretKey = bls.SecretKey.fromBytes(Buffer.alloc(32, 2));
const newSecretKey = SecretKey.fromBytes(Buffer.alloc(32, 2));
const newSyncCommittee: altair.SyncCommitteeMessage = {
slot: syncCommittee.slot,
beaconBlockRoot,

View File

@@ -1,7 +1,6 @@
import type {SecretKey} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {BitArray} from "@chainsafe/ssz";
import {describe, it, expect, beforeEach, beforeAll} from "vitest";
import {SecretKey, Signature, fastAggregateVerify} from "@chainsafe/blst";
import {newFilledArray} from "@lodestar/state-transition";
import {ssz} from "@lodestar/types";
import {SYNC_COMMITTEE_SIZE, SYNC_COMMITTEE_SUBNET_COUNT} from "@lodestar/params";
@@ -83,7 +82,7 @@ describe("aggregate", function () {
let bestContributionBySubnet: Map<number, SyncContributionFast>;
beforeAll(async () => {
for (let i = 0; i < SYNC_COMMITTEE_SUBNET_COUNT; i++) {
sks.push(bls.SecretKey.fromBytes(Buffer.alloc(32, i + 1)));
sks.push(SecretKey.fromBytes(Buffer.alloc(32, i + 1)));
}
bestContributionBySubnet = new Map<number, SyncContributionFast>();
});
@@ -112,10 +111,10 @@ describe("aggregate", function () {
renderBitArray(BitArray.fromBoolArray(expectSyncCommittees))
);
expect(
bls.verifyAggregate(
testSks.map((sk) => sk.toPublicKey().toBytes()),
fastAggregateVerify(
blockRoot,
syncAggregate.syncCommitteeSignature
testSks.map((sk) => sk.toPublicKey()),
Signature.fromBytes(syncAggregate.syncCommitteeSignature)
)
).toBe(true);
});

View File

@@ -1,6 +1,5 @@
import bls from "@chainsafe/bls";
import type {PublicKey, SecretKey} from "@chainsafe/bls/types";
import {afterEach, beforeEach, describe, expect, it, vi} from "vitest";
import {PublicKey, SecretKey} from "@chainsafe/blst";
import {ForkName} from "@lodestar/params";
import {SignatureSetType} from "@lodestar/state-transition";
import {ssz} from "@lodestar/types";
@@ -49,7 +48,7 @@ describe("validateGossipAttestationsSameAttData", () => {
const bytes = new Uint8Array(32);
const dataView = new DataView(bytes.buffer, bytes.byteOffset, bytes.byteLength);
dataView.setUint32(0, i + 1, true);
const secretKey = bls.SecretKey.fromBytes(bytes);
const secretKey = SecretKey.fromKeygen(bytes);
const publicKey = secretKey.toPublicKey();
keypair = {secretKey, publicKey};
keypairs.set(i, keypair);

View File

@@ -1,7 +1,6 @@
import {digest} from "@chainsafe/as-sha256";
import bls from "@chainsafe/bls";
import {PointFormat} from "@chainsafe/bls/types";
import {describe, it, beforeEach, afterEach, vi} from "vitest";
import {SecretKey} from "@chainsafe/blst";
import {config as defaultConfig} from "@lodestar/config/default";
import {computeSigningRoot} from "@lodestar/state-transition";
import {capella, ssz} from "@lodestar/types";
@@ -29,12 +28,12 @@ describe("validate bls to execution change", () => {
// Validator has to be active for long enough
stateEmpty.slot = defaultConfig.SHARD_COMMITTEE_PERIOD * SLOTS_PER_EPOCH;
// A withdrawal key which we will keep same on the two vals we generate
const wsk = bls.SecretKey.fromKeygen();
const wsk = SecretKey.fromKeygen(Buffer.alloc(32));
// Generate and add first val
const sk1 = bls.SecretKey.fromKeygen();
const pubkey1 = sk1.toPublicKey().toBytes(PointFormat.compressed);
const fromBlsPubkey = wsk.toPublicKey().toBytes(PointFormat.compressed);
const sk1 = SecretKey.fromKeygen(Buffer.alloc(32, 1));
const pubkey1 = sk1.toPublicKey().toBytes();
const fromBlsPubkey = wsk.toPublicKey().toBytes();
const withdrawalCredentials = digest(fromBlsPubkey);
withdrawalCredentials[0] = BLS_WITHDRAWAL_PREFIX;
const validator = ssz.phase0.Validator.toViewDU({
@@ -50,8 +49,8 @@ describe("validate bls to execution change", () => {
stateEmpty.validators[0] = validator;
// Gen and add second val
const sk2 = bls.SecretKey.fromKeygen();
const pubkey2 = sk2.toPublicKey().toBytes(PointFormat.compressed);
const sk2 = SecretKey.fromKeygen(Buffer.alloc(32, 2));
const pubkey2 = sk2.toPublicKey().toBytes();
// Set the next validator to already eth1 credential
const withdrawalCredentialsTwo = digest(fromBlsPubkey);
withdrawalCredentialsTwo[0] = ETH1_ADDRESS_WITHDRAWAL_PREFIX;

View File

@@ -1,6 +1,5 @@
import bls from "@chainsafe/bls";
import {PointFormat} from "@chainsafe/bls/types";
import {describe, it, beforeEach, beforeAll, vi, afterEach} from "vitest";
import {SecretKey} from "@chainsafe/blst";
import {config} from "@lodestar/config/default";
import {
CachedBeaconStateAllForks,
@@ -25,7 +24,7 @@ describe("validate voluntary exit", () => {
let opPool: MockedBeaconChain["opPool"];
beforeAll(() => {
const sk = bls.SecretKey.fromKeygen();
const sk = SecretKey.fromKeygen(Buffer.alloc(32));
const stateEmpty = ssz.phase0.BeaconState.defaultValue();
@@ -34,7 +33,7 @@ describe("validate voluntary exit", () => {
// Add a validator that's active since genesis and ready to exit
const validator = ssz.phase0.Validator.toViewDU({
pubkey: sk.toPublicKey().toBytes(PointFormat.compressed),
pubkey: sk.toPublicKey().toBytes(),
withdrawalCredentials: Buffer.alloc(32, 0),
effectiveBalance: 32e9,
slashed: false,

View File

@@ -1,4 +1,4 @@
import type {SecretKey} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {toHexString} from "@chainsafe/ssz";
export function memoOnce<R>(fn: () => R): () => R {

View File

@@ -1,6 +1,6 @@
import tmp from "tmp";
import {vi} from "vitest";
import type {SecretKey} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {LevelDbController} from "@lodestar/db";
import {interopSecretKey} from "@lodestar/state-transition";
import {SlashingProtection, Validator, Signer, SignerType, ValidatorProposerConfig} from "@lodestar/validator";

View File

@@ -1,4 +1,4 @@
import bls from "@chainsafe/bls";
import {SecretKey} from "@chainsafe/blst";
import {config as minimalConfig} from "@lodestar/config/default";
import {
BeaconStateAllForks,
@@ -55,7 +55,7 @@ export function generateState(
opts.validators ??
(withPubkey
? Array.from({length: numValidators}, (_, i) => {
const sk = bls.SecretKey.fromBytes(Buffer.alloc(32, i + 1));
const sk = SecretKey.fromBytes(Buffer.alloc(32, i + 1));
return generateValidator({
...validatorOpts,
pubkey: sk.toPublicKey().toBytes(),

View File

@@ -51,10 +51,9 @@
"blockchain"
],
"dependencies": {
"@chainsafe/bls": "7.1.3",
"@chainsafe/bls-keygen": "^0.4.0",
"@chainsafe/bls-keystore": "^3.1.0",
"@chainsafe/blst": "^0.2.11",
"@chainsafe/blst": "^2.0.1",
"@chainsafe/discv5": "^9.0.0",
"@chainsafe/enr": "^3.0.0",
"@chainsafe/persistent-merkle-tree": "^0.7.1",

View File

@@ -1,6 +1,5 @@
import {fromHexString} from "@chainsafe/ssz";
import bls from "@chainsafe/bls";
import {PointFormat} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {computeSigningRoot} from "@lodestar/state-transition";
import {DOMAIN_BLS_TO_EXECUTION_CHANGE, ForkName} from "@lodestar/params";
import {createBeaconConfig} from "@lodestar/config";
@@ -68,8 +67,8 @@ like to choose for BLS To Execution Change.",
throw new Error(`Validator pubkey ${publicKey} not found in state`);
}
const blsPrivkey = bls.SecretKey.fromBytes(fromHexString(args.fromBlsPrivkey));
const fromBlsPubkey = blsPrivkey.toPublicKey().toBytes(PointFormat.compressed);
const blsPrivkey = SecretKey.fromBytes(fromHexString(args.fromBlsPrivkey));
const fromBlsPubkey = blsPrivkey.toPublicKey().toBytes();
const blsToExecutionChange: capella.BLSToExecutionChange = {
validatorIndex: validator.index,

View File

@@ -1,6 +1,6 @@
import fs from "node:fs";
import path from "node:path";
import bls from "@chainsafe/bls";
import {SecretKey} from "@chainsafe/blst";
import {Keystore} from "@chainsafe/bls-keystore";
import {SignerLocal, SignerType} from "@lodestar/validator";
import {LogLevel, Logger} from "@lodestar/utils";
@@ -73,7 +73,7 @@ export async function decryptKeystoreDefinitions(
(secretKeyBytes: Uint8Array) => {
const signer: SignerLocal = {
type: SignerType.Local,
secretKey: bls.SecretKey.fromBytes(secretKeyBytes),
secretKey: SecretKey.fromBytes(secretKeyBytes),
};
signers[index] = signer;
@@ -107,7 +107,7 @@ export async function decryptKeystoreDefinitions(
const signer: SignerLocal = {
type: SignerType.Local,
secretKey: bls.SecretKey.fromBytes(secretKeyBytes),
secretKey: SecretKey.fromBytes(secretKeyBytes),
};
signers[index] = signer;

View File

@@ -1,6 +1,6 @@
import bls from "@chainsafe/bls";
import {Keystore} from "@chainsafe/bls-keystore";
import {fromHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {
DeleteRemoteKeyStatus,
DeletionStatus,
@@ -149,7 +149,7 @@ export class KeymanagerApi implements Api {
decryptKeystores.queue(
{keystoreStr, password},
async (secretKeyBytes: Uint8Array) => {
const secretKey = bls.SecretKey.fromBytes(secretKeyBytes);
const secretKey = SecretKey.fromBytes(secretKeyBytes);
// Persist the key to disk for restarts, before adding to in-memory store
// If the keystore exist and has a lock it will throw

View File

@@ -1,8 +1,7 @@
import fs from "node:fs";
import path from "node:path";
import bls from "@chainsafe/bls";
import {Keystore} from "@chainsafe/bls-keystore";
import {PointFormat} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {SignerLocal, SignerType} from "@lodestar/validator";
import {fromHex, toHex} from "@lodestar/utils";
import {writeFile600Perm} from "../../../util/file.js";
@@ -40,8 +39,8 @@ export async function loadKeystoreCache(
const result: SignerLocal[] = [];
for (const [index, k] of keystores.entries()) {
const secretKeyBytes = Uint8Array.prototype.slice.call(secretKeyConcatenatedBytes, index * 32, (index + 1) * 32);
const secretKey = bls.SecretKey.fromBytes(secretKeyBytes);
const publicKey = secretKey.toPublicKey().toBytes(PointFormat.compressed);
const secretKey = SecretKey.fromBytes(secretKeyBytes);
const publicKey = secretKey.toPublicKey().toBytes();
if (toHex(publicKey) !== toHex(fromHex(k.pubkey))) {
throw new Error(

View File

@@ -1,7 +1,6 @@
import path from "node:path";
import bls from "@chainsafe/bls";
import {deriveEth2ValidatorKeys, deriveKeyFromMnemonic} from "@chainsafe/bls-keygen";
import {toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {interopSecretKey} from "@lodestar/state-transition";
import {externalSignerGetKeys, Signer, SignerType} from "@lodestar/validator";
import {LogLevel, Logger, isValidHttpUrl} from "@lodestar/utils";
@@ -72,7 +71,7 @@ export async function getSignersFromArgs(
const indexes = parseRange(args.mnemonicIndexes);
return indexes.map((index) => ({
type: SignerType.Local,
secretKey: bls.SecretKey.fromBytes(deriveEth2ValidatorKeys(masterSK, index).signing),
secretKey: SecretKey.fromBytes(deriveEth2ValidatorKeys(masterSK, index).signing),
}));
}
@@ -150,7 +149,7 @@ export async function getSignersFromArgs(
export function getSignerPubkeyHex(signer: Signer): string {
switch (signer.type) {
case SignerType.Local:
return toHexString(signer.secretKey.toPublicKey().toBytes());
return signer.secretKey.toPublicKey().toHex();
case SignerType.Remote:
return signer.pubkey;

View File

@@ -1,5 +1,5 @@
import inquirer from "inquirer";
import bls from "@chainsafe/bls";
import {Signature} from "@chainsafe/blst";
import {
computeEpochAtSlot,
computeSigningRoot,
@@ -8,7 +8,7 @@ import {
} from "@lodestar/state-transition";
import {createBeaconConfig, BeaconConfig} from "@lodestar/config";
import {phase0, ssz, ValidatorIndex, Epoch} from "@lodestar/types";
import {CliCommand, toHex} from "@lodestar/utils";
import {CliCommand, fromHex, toHex} from "@lodestar/utils";
import {externalSignerPostSignature, SignableMessageType, Signer, SignerType} from "@lodestar/validator";
import {ApiClient, getClient} from "@lodestar/api";
import {ensure0xPrefix, YargsError, wrapError} from "../../util/index.js";
@@ -161,7 +161,7 @@ async function processVoluntaryExit(
data: voluntaryExit,
type: SignableMessageType.VOLUNTARY_EXIT,
});
signature = bls.Signature.fromHex(signatureHex);
signature = Signature.fromBytes(fromHex(signatureHex));
break;
}
default:

View File

@@ -1,5 +1,4 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/blst";
import {PublicKey} from "@chainsafe/blst";
import {fromHexString} from "@chainsafe/ssz";
/**
@@ -52,7 +51,7 @@ export function parseRange(range: string): number[] {
export function assertValidPubkeysHex(pubkeysHex: string[]): void {
for (const pubkeyHex of pubkeysHex) {
const pubkeyBytes = fromHexString(pubkeyHex);
bls.PublicKey.fromBytes(pubkeyBytes, CoordType.jacobian, true);
PublicKey.fromBytes(pubkeyBytes, true);
}
}

View File

@@ -3,7 +3,7 @@ import {randomBytes} from "node:crypto";
import {describe, it, expect, beforeEach, vi} from "vitest";
import tmp from "tmp";
import {Keystore} from "@chainsafe/bls-keystore";
import bls from "@chainsafe/bls";
import {SecretKey} from "@chainsafe/blst";
import {interopSecretKey} from "@lodestar/state-transition";
import {SignerLocal, SignerType} from "@lodestar/validator";
import {loadKeystoreCache, writeKeystoreCache} from "../../../../../src/cmds/validator/keymanager/keystoreCache.js";
@@ -27,7 +27,7 @@ describe("keystoreCache", () => {
keystoreCacheFile = tmp.tmpNameSync({postfix: ".cache"});
for (let i = 0; i < numberOfSigners; i++) {
const secretKey = bls.SecretKey.fromBytes(interopSecretKey(i).toBytes());
const secretKey = SecretKey.fromBytes(interopSecretKey(i).toBytes());
const keystorePath = tmp.tmpNameSync({postfix: ".json"});
const password = secretKey.toHex();
const keystore = await Keystore.create(

View File

@@ -1,5 +1,6 @@
import type {SecretKey} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {routes} from "@lodestar/api/beacon";
import {toHex} from "@lodestar/utils";
import {AssertionResult, ValidatorClientKeys, Assertion, ValidatorClient} from "../interfaces.js";
import {arrayEquals} from "../utils/index.js";
import {neverMatcher} from "./matchers.js";
@@ -38,7 +39,7 @@ export const nodeAssertion: Assertion<"node", {health: number; keyManagerKeys: s
}
const expectedPublicKeys = node.validator
? getAllKeys(node.validator.keys).map((k) => k.toPublicKey().toHex())
? getAllKeys(node.validator.keys).map((k) => toHex(k.toPublicKey().toBytes()))
: [];
if (!arrayEquals(keyManagerKeys.sort(), expectedPublicKeys.sort())) {

View File

@@ -1,6 +1,6 @@
import type {SecretKey} from "@chainsafe/bls/types";
import {fromHexString} from "@chainsafe/ssz";
import {fastify, FastifyInstance} from "fastify";
import {SecretKey} from "@chainsafe/blst";
import {EXTERNAL_SIGNER_BASE_PORT} from "./constants.js";
/* eslint-disable no-console */

View File

@@ -1,7 +1,7 @@
/* eslint-disable @typescript-eslint/naming-convention */
import {ChildProcess} from "node:child_process";
import type {SecretKey} from "@chainsafe/bls/types";
import {Web3} from "web3";
import {SecretKey} from "@chainsafe/blst";
import {ApiClient} from "@lodestar/api";
import {ApiClient as KeyManagerApi} from "@lodestar/api/keymanager";
import {ChainForkConfig} from "@lodestar/config";

View File

@@ -58,7 +58,7 @@
"blockchain"
],
"dependencies": {
"@chainsafe/bls": "7.1.3",
"@chainsafe/blst": "^2.0.1",
"@chainsafe/bls-keygen": "^0.4.0",
"@lodestar/api": "^1.20.2",
"@lodestar/config": "^1.20.2",

View File

@@ -1,5 +1,4 @@
import bls from "@chainsafe/bls";
import type {SecretKey} from "@chainsafe/bls/types";
import {SecretKey, aggregateSignatures} from "@chainsafe/blst";
import {getClient} from "@lodestar/api";
import {phase0, ssz} from "@lodestar/types";
import {config as chainConfig} from "@lodestar/config/default";
@@ -149,5 +148,5 @@ function signAttestationDataBigint(
const signingRoot = computeSigningRoot(ssz.phase0.AttestationDataBigint, data, proposerDomain);
const sigs = sks.map((sk) => sk.sign(signingRoot));
return bls.Signature.aggregate(sigs).toBytes();
return aggregateSignatures(sigs).toBytes();
}

View File

@@ -1,4 +1,4 @@
import type {SecretKey} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {getClient} from "@lodestar/api";
import {phase0, ssz} from "@lodestar/types";
import {config as chainConfig} from "@lodestar/config/default";

View File

@@ -1,6 +1,5 @@
import bls from "@chainsafe/bls";
import type {SecretKey} from "@chainsafe/bls/types";
import {deriveEth2ValidatorKeys, deriveKeyFromMnemonic} from "@chainsafe/bls-keygen";
import {SecretKey} from "@chainsafe/blst";
import {interopSecretKey} from "@lodestar/state-transition";
import {CliCommandOptions} from "@lodestar/utils";
import {YargsError} from "./errors.js";
@@ -42,7 +41,7 @@ export function deriveSecretKeys(args: SecretKeysArgs): SecretKey[] {
return indexes.map((index) => {
const {signing} = deriveEth2ValidatorKeys(masterSK, index);
return bls.SecretKey.fromBytes(signing);
return SecretKey.fromBytes(signing);
});
}

View File

@@ -74,6 +74,7 @@
},
"dependencies": {
"@chainsafe/bls": "7.1.3",
"@chainsafe/blst": "^0.2.0",
"@chainsafe/persistent-merkle-tree": "^0.7.1",
"@chainsafe/ssz": "^0.15.1",
"@lodestar/api": "^1.20.2",

View File

@@ -59,8 +59,7 @@
"types": "lib/index.d.ts",
"dependencies": {
"@chainsafe/as-sha256": "^0.4.1",
"@chainsafe/bls": "7.1.3",
"@chainsafe/blst": "^0.2.11",
"@chainsafe/blst": "^2.0.1",
"@chainsafe/persistent-merkle-tree": "^0.7.1",
"@chainsafe/persistent-ts": "^0.19.1",
"@chainsafe/ssz": "^0.15.1",

View File

@@ -1,5 +1,4 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/bls/types";
import {PublicKey, Signature, verify} from "@chainsafe/blst";
import {phase0, ssz} from "@lodestar/types";
import {verifyMerkleBranch} from "@lodestar/utils";
@@ -56,9 +55,9 @@ export function processDeposit(fork: ForkSeq, state: CachedBeaconStateAllForks,
const signingRoot = computeSigningRoot(ssz.phase0.DepositMessage, depositMessage, domain);
try {
// Pubkeys must be checked for group + inf. This must be done only once when the validator deposit is processed
const publicKey = bls.PublicKey.fromBytes(pubkey, CoordType.affine, true);
const signature = bls.Signature.fromBytes(deposit.data.signature, CoordType.affine, true);
if (!signature.verify(publicKey, signingRoot)) {
const publicKey = PublicKey.fromBytes(pubkey, true);
const signature = Signature.fromBytes(deposit.data.signature, true);
if (!verify(signingRoot, publicKey, signature)) {
return;
}
} catch (e) {

View File

@@ -1,5 +1,4 @@
import {CoordType} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {PublicKey} from "@chainsafe/blst";
import {BLSSignature, CommitteeIndex, Epoch, Slot, ValidatorIndex, phase0, SyncPeriod} from "@lodestar/types";
import {createBeaconConfig, BeaconConfig, ChainConfig} from "@lodestar/config";
import {
@@ -762,7 +761,7 @@ export class EpochCache {
addPubkey(index: ValidatorIndex, pubkey: Uint8Array): void {
this.pubkey2index.set(pubkey, index);
this.index2pubkey[index] = bls.PublicKey.fromBytes(pubkey, CoordType.jacobian); // Optimize for aggregation
this.index2pubkey[index] = PublicKey.fromBytes(pubkey); // Optimize for aggregation
}
getShufflingAtSlot(slot: Slot): EpochShuffling {

View File

@@ -1,5 +1,4 @@
import {CoordType, PublicKey} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {PublicKey} from "@chainsafe/blst";
import {ValidatorIndex} from "@lodestar/types";
import {BeaconStateAllForks} from "./types.js";
@@ -72,6 +71,6 @@ export function syncPubkeys(
// Pubkeys must be checked for group + inf. This must be done only once when the validator deposit is processed.
// Afterwards any public key is the state consider validated.
// > Do not do any validation here
index2pubkey.push(bls.PublicKey.fromBytes(pubkey, CoordType.jacobian)); // Optimize for aggregation
index2pubkey.push(PublicKey.fromBytes(pubkey)); // Optimize for aggregation
}
}

View File

@@ -1,5 +1,4 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {BeaconConfig} from "@lodestar/config";
import {loadState} from "../util/loadState/loadState.js";
import {EpochCache, EpochCacheImmutableData, EpochCacheOpts} from "./epochCache.js";
@@ -180,7 +179,7 @@ export function loadCachedBeaconState<T extends BeaconStateAllForks & BeaconStat
const validator = validators.getReadonly(validatorIndex);
const pubkey = validator.pubkey;
pubkey2index.set(pubkey, validatorIndex);
index2pubkey[validatorIndex] = bls.PublicKey.fromBytes(pubkey, CoordType.jacobian);
index2pubkey[validatorIndex] = PublicKey.fromBytes(pubkey);
}
return createCachedBeaconState(

View File

@@ -1,4 +1,4 @@
import bls from "@chainsafe/bls";
import {aggregateSerializedPublicKeys} from "@chainsafe/blst";
import {EPOCHS_PER_SYNC_COMMITTEE_PERIOD} from "@lodestar/params";
import {ssz} from "@lodestar/types";
import {getNextSyncCommitteeIndices} from "../util/seed.js";
@@ -32,7 +32,7 @@ export function processSyncCommitteeUpdates(state: CachedBeaconStateAltair): voi
state.currentSyncCommittee = state.nextSyncCommittee;
state.nextSyncCommittee = ssz.altair.SyncCommittee.toViewDU({
pubkeys: nextSyncCommitteePubkeys,
aggregatePubkey: bls.aggregatePublicKeys(nextSyncCommitteePubkeys),
aggregatePubkey: aggregateSerializedPublicKeys(nextSyncCommitteePubkeys).toBytes(),
});
// Rotate syncCommittee cache

View File

@@ -1,5 +1,4 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/bls/types";
import {PublicKey} from "@chainsafe/blst";
import {DOMAIN_BLS_TO_EXECUTION_CHANGE, ForkName} from "@lodestar/params";
import {capella, ssz} from "@lodestar/types";
import {BeaconConfig} from "@lodestar/config";
@@ -29,7 +28,7 @@ export function getBlsToExecutionChangeSignatureSet(
type: SignatureSetType.single,
// The withdrawal pubkey is the same as signedBLSToExecutionChange's fromBlsPubkey as it should
// be validated against the withdrawal credentials digest
pubkey: bls.PublicKey.fromBytes(signedBLSToExecutionChange.message.fromBlsPubkey, CoordType.affine, true),
pubkey: PublicKey.fromBytes(signedBLSToExecutionChange.message.fromBlsPubkey, true),
signingRoot: computeSigningRoot(ssz.capella.BLSToExecutionChange, signedBLSToExecutionChange.message, domain),
signature: signedBLSToExecutionChange.signature,
};

View File

@@ -1,7 +1,6 @@
import {toBufferBE} from "bigint-buffer";
import {digest} from "@chainsafe/as-sha256";
import type {SecretKey} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {SecretKey} from "@chainsafe/blst";
import {bytesToBigInt, intToBytes} from "@lodestar/utils";
let curveOrder: bigint;
@@ -19,5 +18,5 @@ export function interopSecretKeys(validatorCount: number): SecretKey[] {
export function interopSecretKey(index: number): SecretKey {
const CURVE_ORDER = getCurveOrder();
const secretKeyBytes = toBufferBE(bytesToBigInt(digest(intToBytes(index, 32))) % CURVE_ORDER, 32);
return bls.SecretKey.fromBytes(secretKeyBytes);
return SecretKey.fromBytes(secretKeyBytes);
}

View File

@@ -1,5 +1,4 @@
import type {PublicKey} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {PublicKey, Signature, fastAggregateVerify, verify} from "@chainsafe/blst";
import {Root} from "@lodestar/types";
export enum SignatureSetType {
@@ -25,14 +24,14 @@ export type ISignatureSet = SingleSignatureSet | AggregatedSignatureSet;
export function verifySignatureSet(signatureSet: ISignatureSet): boolean {
// All signatures are not trusted and must be group checked (p2.subgroup_check)
const signature = bls.Signature.fromBytes(signatureSet.signature, undefined, true);
const signature = Signature.fromBytes(signatureSet.signature, true);
switch (signatureSet.type) {
case SignatureSetType.single:
return signature.verify(signatureSet.pubkey, signatureSet.signingRoot);
return verify(signatureSet.signingRoot, signatureSet.pubkey, signature);
case SignatureSetType.aggregate:
return signature.verifyAggregate(signatureSet.pubkeys, signatureSet.signingRoot);
return fastAggregateVerify(signatureSet.signingRoot, signatureSet.pubkeys, signature);
default:
throw Error("Unknown signature set type");

View File

@@ -1,4 +1,4 @@
import bls from "@chainsafe/bls";
import {aggregateSerializedPublicKeys} from "@chainsafe/blst";
import {
BASE_REWARD_FACTOR,
EFFECTIVE_BALANCE_INCREMENT,
@@ -32,7 +32,7 @@ export function getNextSyncCommittee(
indices,
syncCommittee: {
pubkeys,
aggregatePubkey: bls.aggregatePublicKeys(pubkeys),
aggregatePubkey: aggregateSerializedPublicKeys(pubkeys).toBytes(),
},
};
}

View File

@@ -1,6 +1,6 @@
import bls from "@chainsafe/bls";
import {toGindex, Tree} from "@chainsafe/persistent-merkle-tree";
import {BitArray} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {altair, phase0, ssz} from "@lodestar/types";
import {DOMAIN_DEPOSIT, SYNC_COMMITTEE_SIZE} from "@lodestar/params";
import {config} from "@lodestar/config/default";
@@ -210,7 +210,7 @@ function getDeposits(preState: CachedBeaconStateAllForks, count: number): phase0
depositRootViewDU["dirtyLength"] = true;
for (let i = 0; i < count; i++) {
const sk = bls.SecretKey.fromBytes(Buffer.alloc(32, i + 1));
const sk = SecretKey.fromBytes(Buffer.alloc(32, i + 1));
const pubkey = sk.toPublicKey().toBytes();
const depositMessage: phase0.DepositMessage = {pubkey, withdrawalCredentials, amount: 32e9};
// Sign with disposable keys

View File

@@ -1,6 +1,5 @@
import {CoordType, PublicKey, SecretKey} from "@chainsafe/bls/types";
import bls from "@chainsafe/bls";
import {BitArray, fromHexString} from "@chainsafe/ssz";
import {PublicKey, SecretKey} from "@chainsafe/blst";
import {phase0, ssz, Slot, BeaconState} from "@lodestar/types";
import {config} from "@lodestar/config/default";
import {createBeaconConfig, createChainForkConfig} from "@lodestar/config";
@@ -64,7 +63,7 @@ export const perfStateEpoch = epoch;
// eslint-disable-next-line @typescript-eslint/explicit-function-return-type
export function getPubkeys(vc = numValidators) {
const pubkeysMod = interopPubkeysCached(keypairsMod);
const pubkeysModObj = pubkeysMod.map((pk) => bls.PublicKey.fromBytes(pk, CoordType.jacobian));
const pubkeysModObj = pubkeysMod.map((pk) => PublicKey.fromBytes(pk));
const pubkeys = Array.from({length: vc}, (_, i) => pubkeysMod[i % keypairsMod]);
return {pubkeysMod, pubkeysModObj, pubkeys};
}

View File

@@ -1,6 +1,5 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/bls/types";
import {itBench, setBenchOpts} from "@dapplion/benchmark";
import {PublicKey} from "@chainsafe/blst";
import {loadState} from "../../../../src/util/loadState/loadState.js";
import {createCachedBeaconState} from "../../../../src/cache/stateCache.js";
import {Index2PubkeyCache, PubkeyIndexMap} from "../../../../src/cache/pubkeyCache.js";
@@ -78,7 +77,7 @@ describe("loadState", function () {
const validator = validators.getReadonly(validatorIndex);
const pubkey = validator.pubkey;
pubkey2index.set(pubkey, validatorIndex);
index2pubkey[validatorIndex] = bls.PublicKey.fromBytes(pubkey, CoordType.jacobian);
index2pubkey[validatorIndex] = PublicKey.fromBytes(pubkey);
}
// skip computimg shuffling in performance test because in reality we have a ShufflingCache
// eslint-disable-next-line @typescript-eslint/explicit-function-return-type

View File

@@ -5,6 +5,6 @@ import {G2_POINT_AT_INFINITY} from "../../src/index.js";
describe("constants", () => {
it("G2_POINT_AT_INFINITY", () => {
const p2 = blst.Signature.fromBytes(G2_POINT_AT_INFINITY);
expect(p2.value.is_inf()).toBe(true);
expect(() => p2.sigValidate(true)).toThrow();
});
});

View File

@@ -1,7 +1,7 @@
import crypto from "node:crypto";
import {describe, it, expect} from "vitest";
import bls from "@chainsafe/bls";
import {BitArray} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {config} from "@lodestar/config/default";
import {phase0, capella, ValidatorIndex, BLSSignature, ssz} from "@lodestar/types";
import {FAR_FUTURE_EPOCH, MAX_EFFECTIVE_BALANCE} from "@lodestar/params";
@@ -61,7 +61,7 @@ describe("signatureSets", () => {
exit: FAR_FUTURE_EPOCH,
});
for (const validator of validators) {
validator.pubkey = bls.SecretKey.fromKeygen().toPublicKey().toBytes();
validator.pubkey = SecretKey.fromKeygen(Buffer.alloc(32)).toPublicKey().toBytes();
}
const state = generateCachedState(config, {validators});

View File

@@ -57,7 +57,7 @@
"blockchain"
],
"dependencies": {
"@chainsafe/bls": "7.1.3",
"@chainsafe/blst": "^2.0.1",
"@chainsafe/bls-keystore": "^3.1.0",
"@lodestar/params": "^1.20.2",
"@lodestar/utils": "^1.20.2",

View File

@@ -1,5 +1,5 @@
import bls from "@chainsafe/bls";
import {Keystore} from "@chainsafe/bls-keystore";
import {SecretKey} from "@chainsafe/blst";
import {fromHex} from "@lodestar/utils";
/**
@@ -10,7 +10,7 @@ export async function getKeystoresStr(password: string, secretKeys: string[]): P
for (const secretKey of secretKeys) {
const sk = fromHex(secretKey);
const pk = bls.SecretKey.fromBytes(sk).toPublicKey().toBytes();
const pk = SecretKey.fromBytes(sk).toPublicKey().toBytes();
const keystore = await Keystore.create(password, sk, pk, "");
keystoresStr.push(keystore.stringify());
}

View File

@@ -45,7 +45,7 @@
"blockchain"
],
"dependencies": {
"@chainsafe/bls": "7.1.3",
"@chainsafe/blst": "^2.0.1",
"@chainsafe/ssz": "^0.15.1",
"@lodestar/api": "^1.20.2",
"@lodestar/config": "^1.20.2",

View File

@@ -1,6 +1,5 @@
import bls from "@chainsafe/bls";
import {CoordType} from "@chainsafe/bls/types";
import {fromHexString} from "@chainsafe/ssz";
import {PublicKey} from "@chainsafe/blst";
import {ChainForkConfig} from "@lodestar/config";
import {SLOTS_PER_EPOCH} from "@lodestar/params";
import {toSafePrintableUrl} from "@lodestar/utils";
@@ -79,6 +78,6 @@ export function pollExternalSignerPubkeys(
function assertValidPubkeysHex(pubkeysHex: string[]): void {
for (const pubkeyHex of pubkeysHex) {
const pubkeyBytes = fromHexString(pubkeyHex);
bls.PublicKey.fromBytes(pubkeyBytes, CoordType.jacobian, true);
PublicKey.fromBytes(pubkeyBytes, true);
}
}

View File

@@ -1,5 +1,5 @@
import type {SecretKey} from "@chainsafe/bls/types";
import {BitArray, fromHexString, toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {
computeEpochAtSlot,
computeSigningRoot,

View File

@@ -1,4 +1,4 @@
import type {SecretKey} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {BLSPubkey} from "@lodestar/types";
import {DatabaseController} from "@lodestar/db";

View File

@@ -1,6 +1,6 @@
import {describe, it, expect, beforeAll, beforeEach, afterEach, vi} from "vitest";
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {ssz} from "@lodestar/types";
import {routes} from "@lodestar/api";
import {AttestationService, AttestationServiceOpts} from "../../../src/services/attestation.js";
@@ -28,7 +28,7 @@ describe("AttestationService", function () {
let pubkeys: Uint8Array[]; // Initialize pubkeys in before() so bls is already initialized
beforeAll(() => {
const secretKeys = Array.from({length: 1}, (_, i) => bls.SecretKey.fromBytes(Buffer.alloc(32, i + 1)));
const secretKeys = Array.from({length: 1}, (_, i) => SecretKey.fromBytes(Buffer.alloc(32, i + 1)));
pubkeys = secretKeys.map((sk) => sk.toPublicKey().toBytes());
validatorStore.votingPubkeys.mockReturnValue(pubkeys.map(toHexString));
validatorStore.hasVotingPubkey.mockReturnValue(true);

View File

@@ -1,7 +1,7 @@
import {describe, it, expect, beforeAll, vi, Mocked, beforeEach, afterEach} from "vitest";
import {toBufferBE} from "bigint-buffer";
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {chainConfig} from "@lodestar/config/default";
import {routes} from "@lodestar/api";
import {ssz} from "@lodestar/types";
@@ -37,7 +37,7 @@ describe("AttestationDutiesService", function () {
};
beforeAll(async () => {
const secretKeys = [bls.SecretKey.fromBytes(toBufferBE(BigInt(98), 32))];
const secretKeys = [SecretKey.fromBytes(toBufferBE(BigInt(98), 32))];
pubkeys = secretKeys.map((sk) => sk.toPublicKey().toBytes());
validatorStore = await initValidatorStore(secretKeys, api, chainConfig);
});

View File

@@ -1,6 +1,6 @@
import {describe, it, expect, beforeAll, beforeEach, afterEach, vi} from "vitest";
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {createChainForkConfig} from "@lodestar/config";
import {config as mainnetConfig} from "@lodestar/config/default";
import {sleep} from "@lodestar/utils";
@@ -25,7 +25,7 @@ describe("BlockDutiesService", function () {
const config = createChainForkConfig(mainnetConfig);
beforeAll(() => {
const secretKeys = Array.from({length: 2}, (_, i) => bls.SecretKey.fromBytes(Buffer.alloc(32, i + 1)));
const secretKeys = Array.from({length: 2}, (_, i) => SecretKey.fromBytes(Buffer.alloc(32, i + 1)));
pubkeys = secretKeys.map((sk) => sk.toPublicKey().toBytes());
validatorStore.votingPubkeys.mockReturnValue(pubkeys.map(toHexString));
});

View File

@@ -1,7 +1,7 @@
import {describe, it, expect, beforeAll, beforeEach, afterEach, vi} from "vitest";
import {toBufferBE} from "bigint-buffer";
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {routes} from "@lodestar/api";
import {chainConfig} from "@lodestar/config/default";
import {toHex} from "@lodestar/utils";
@@ -19,7 +19,7 @@ describe("BlockDutiesService", function () {
let pubkeys: Uint8Array[]; // Initialize pubkeys in before() so bls is already initialized
beforeAll(async () => {
const secretKeys = Array.from({length: 3}, (_, i) => bls.SecretKey.fromBytes(toBufferBE(BigInt(i + 1), 32)));
const secretKeys = Array.from({length: 3}, (_, i) => SecretKey.fromBytes(toBufferBE(BigInt(i + 1), 32)));
pubkeys = secretKeys.map((sk) => sk.toPublicKey().toBytes());
validatorStore = await initValidatorStore(secretKeys, api);
});

View File

@@ -1,7 +1,6 @@
import {MockedFunction, afterAll, afterEach, beforeAll, beforeEach, describe, expect, it, vi} from "vitest";
import {toBufferBE} from "bigint-buffer";
import bls from "@chainsafe/bls";
import {SecretKey} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {createChainForkConfig} from "@lodestar/config";
import {chainConfig} from "@lodestar/config/default";
import {ExternalSignerOptions, pollExternalSignerPubkeys} from "../../../src/services/externalSignerSync.js";
@@ -32,7 +31,7 @@ describe("External signer sync", () => {
beforeAll(() => {
vi.useFakeTimers();
secretKeys = Array.from({length: 3}, (_, i) => bls.SecretKey.fromBytes(toBufferBE(BigInt(i + 1), 32)));
secretKeys = Array.from({length: 3}, (_, i) => SecretKey.fromBytes(toBufferBE(BigInt(i + 1), 32)));
pubkeys = secretKeys.map((sk) => sk.toPublicKey().toHex());
externalSignerGetKeysStub = vi.mocked(externalSignerGetKeys);
});

View File

@@ -1,7 +1,7 @@
import {describe, it, expect, beforeAll} from "vitest";
import {toBufferBE} from "bigint-buffer";
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {getApiClientStub} from "../../utils/apiStub.js";
import {testLogger} from "../../utils/logger.js";
import {IndicesService} from "../../../src/services/indices.js";
@@ -14,8 +14,8 @@ describe("IndicesService", function () {
beforeAll(() => {
const secretKeys = [
bls.SecretKey.fromBytes(toBufferBE(BigInt(98), 32)),
bls.SecretKey.fromBytes(toBufferBE(BigInt(99), 32)),
SecretKey.fromBytes(toBufferBE(BigInt(98), 32)),
SecretKey.fromBytes(toBufferBE(BigInt(99), 32)),
];
pubkeys = secretKeys.map((sk) => sk.toPublicKey().toBytes());
});

View File

@@ -1,8 +1,8 @@
import {describe, it, expect, beforeAll, beforeEach, afterEach} from "vitest";
import {when} from "vitest-when";
import {toBufferBE} from "bigint-buffer";
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {createChainForkConfig} from "@lodestar/config";
import {config as mainnetConfig} from "@lodestar/config/default";
import {routes} from "@lodestar/api";
@@ -43,8 +43,8 @@ describe("SyncCommitteeDutiesService", function () {
beforeAll(async () => {
const secretKeys = [
bls.SecretKey.fromBytes(toBufferBE(BigInt(98), 32)),
bls.SecretKey.fromBytes(toBufferBE(BigInt(99), 32)),
SecretKey.fromBytes(toBufferBE(BigInt(98), 32)),
SecretKey.fromBytes(toBufferBE(BigInt(99), 32)),
];
pubkeys = secretKeys.map((sk) => sk.toPublicKey().toBytes());
validatorStore = await initValidatorStore(secretKeys, api, altair0Config);

View File

@@ -1,6 +1,6 @@
import {describe, it, expect, beforeAll, beforeEach, afterEach, vi} from "vitest";
import bls from "@chainsafe/bls";
import {toHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {createChainForkConfig} from "@lodestar/config";
import {config as mainnetConfig} from "@lodestar/config/default";
import {ssz} from "@lodestar/types";
@@ -37,7 +37,7 @@ describe("SyncCommitteeService", function () {
});
beforeAll(() => {
const secretKeys = Array.from({length: 1}, (_, i) => bls.SecretKey.fromBytes(Buffer.alloc(32, i + 1)));
const secretKeys = Array.from({length: 1}, (_, i) => SecretKey.fromBytes(Buffer.alloc(32, i + 1)));
pubkeys = secretKeys.map((sk) => sk.toPublicKey().toBytes());
validatorStore.votingPubkeys.mockReturnValue(pubkeys.map(toHexString));
validatorStore.hasVotingPubkey.mockReturnValue(true);

View File

@@ -1,7 +1,7 @@
import {describe, it, expect, beforeEach, afterEach, vi} from "vitest";
import {toBufferBE} from "bigint-buffer";
import bls from "@chainsafe/bls";
import {toHexString, fromHexString} from "@chainsafe/ssz";
import {SecretKey} from "@chainsafe/blst";
import {chainConfig} from "@lodestar/config/default";
import {bellatrix} from "@lodestar/types";
import {routes} from "@lodestar/api";
@@ -92,7 +92,7 @@ describe("ValidatorStore", function () {
});
});
const secretKeys = Array.from({length: 3}, (_, i) => bls.SecretKey.fromBytes(toBufferBE(BigInt(i + 1), 32)));
const secretKeys = Array.from({length: 3}, (_, i) => SecretKey.fromBytes(toBufferBE(BigInt(i + 1), 32)));
const pubkeys = secretKeys.map((sk) => sk.toPublicKey().toBytes());
const valRegF00G100 = {

View File

@@ -1,4 +1,4 @@
import {SecretKey} from "@chainsafe/bls/types";
import {SecretKey} from "@chainsafe/blst";
import {ApiClient} from "@lodestar/api";
import {chainConfig} from "@lodestar/config/default";
import {createBeaconConfig, ChainConfig} from "@lodestar/config";

255
yarn.lock
View File

@@ -306,7 +306,32 @@
"@chainsafe/bls-keygen" "^0.4.0"
bls-eth-wasm "^0.4.8"
"@chainsafe/blst@^0.2.11":
"@chainsafe/blst-darwin-arm64@2.0.1":
version "2.0.1"
resolved "https://registry.yarnpkg.com/@chainsafe/blst-darwin-arm64/-/blst-darwin-arm64-2.0.1.tgz#d96d6dd906a6c9c809d6f5b5539e031ccff63b25"
integrity sha512-ZmRLimvo+BoMcpalzuS3Pj0j6l2cSDU7qMBwjch49ljkrsr4/rls7COMW8MSyDyVUSfzg0agotAByYfs+Bg3ZQ==
"@chainsafe/blst-darwin-x64@2.0.1":
version "2.0.1"
resolved "https://registry.yarnpkg.com/@chainsafe/blst-darwin-x64/-/blst-darwin-x64-2.0.1.tgz#56018c5955337a5e2b754e941bdcda0a2c6f1a84"
integrity sha512-5DPtmKhia5/k0szjsrgxF7GCOE5pnSRcsLvtlChzkLY7KhfxnGt5XeNtCK6NoAAfcxzN8mZrwvvzDfsDmImTQg==
"@chainsafe/blst-linux-arm64-gnu@2.0.1":
version "2.0.1"
resolved "https://registry.yarnpkg.com/@chainsafe/blst-linux-arm64-gnu/-/blst-linux-arm64-gnu-2.0.1.tgz#23a66095eaf4c23a8c54ef7df27d9c80d45ebb32"
integrity sha512-REIW0uM9a97iasE+RX+M1yEmIywOF1ly9/xl7JTYYASXoDDQD2eL06k17N5kXE/382wU28fU/h6V2qmWcqiWAQ==
"@chainsafe/blst-linux-x64-gnu@2.0.1":
version "2.0.1"
resolved "https://registry.yarnpkg.com/@chainsafe/blst-linux-x64-gnu/-/blst-linux-x64-gnu-2.0.1.tgz#1769640e9b9140ee3e4142520c00d09c9ebc66e9"
integrity sha512-zH7GkMI+wWVKGp5MA+8A2EGx9fe5/jktz/KB2SrGhBu956IXh7qCV9mMCOzA4mmpxSuxjzD4auXpQc/D4uApLw==
"@chainsafe/blst-win32-x64-msvc@2.0.1":
version "2.0.1"
resolved "https://registry.yarnpkg.com/@chainsafe/blst-win32-x64-msvc/-/blst-win32-x64-msvc-2.0.1.tgz#dfebc2ea23875bc02ffac11d2d15bf6e644d59ed"
integrity sha512-RcOo1Sl1Ai5igp56l3I5kwWkesfbD14PBoUcoR61phvIHnhyZzOtVX6rhuCYwcl0IIgSDSWtNNMdSwX34GGmxA==
"@chainsafe/blst@^0.2.0":
version "0.2.11"
resolved "https://registry.yarnpkg.com/@chainsafe/blst/-/blst-0.2.11.tgz#5ec85cd663592819d1dc51127e75dfd834250e3d"
integrity sha512-URyOLq5GtxBoxibOnd2pgLydCy0UZzbiIIBcsRAvGxAsRzjZL04TsQfwRkz5aphU3a1ebeRoMmI/HHyMCiFSQg==
@@ -315,6 +340,17 @@
node-fetch "^2.6.1"
node-gyp "^8.4.0"
"@chainsafe/blst@^2.0.1":
version "2.0.1"
resolved "https://registry.yarnpkg.com/@chainsafe/blst/-/blst-2.0.1.tgz#9e8ceb6766fcb231e4fa378eee6378275018a159"
integrity sha512-+sIlLzFb6htv1WH3XIF2WDN+qlstGpo8Zl5ibxzT6VsiBSswsH05AQMpd4uQfRO1uFKhEk9JDKi8bHvIDmN8Jg==
optionalDependencies:
"@chainsafe/blst-darwin-arm64" "2.0.1"
"@chainsafe/blst-darwin-x64" "2.0.1"
"@chainsafe/blst-linux-arm64-gnu" "2.0.1"
"@chainsafe/blst-linux-x64-gnu" "2.0.1"
"@chainsafe/blst-win32-x64-msvc" "2.0.1"
"@chainsafe/discv5@^9.0.0":
version "9.0.0"
resolved "https://registry.yarnpkg.com/@chainsafe/discv5/-/discv5-9.0.0.tgz#05d4d9d671894b41f0fafa8f32c48ae3ed761bd1"
@@ -1555,6 +1591,29 @@
progress-events "^1.0.0"
uint8arraylist "^2.4.8"
"@libp2p/interface@^1.2.0", "@libp2p/interface@^1.6.1":
version "1.6.1"
resolved "https://registry.yarnpkg.com/@libp2p/interface/-/interface-1.6.1.tgz#dc8f1a49da7ee81cd068e872bde3445c3b631d45"
integrity sha512-bpkIYTvZhGGc/ajITKvgFpaP8UtPWoSj+xHVrj6zyAN8U/cAqN0IQQt4a7daJr5VZa8B86i4d1iccdG42/mz+g==
dependencies:
"@multiformats/multiaddr" "^12.2.3"
it-pushable "^3.2.3"
it-stream-types "^2.0.1"
multiformats "^13.1.0"
progress-events "^1.0.0"
uint8arraylist "^2.4.8"
"@libp2p/logger@^4.0.10":
version "4.0.17"
resolved "https://registry.yarnpkg.com/@libp2p/logger/-/logger-4.0.17.tgz#1ae663bb2dec3c7e6b29d2195756f64e5e77d939"
integrity sha512-NPGN27uOXFGuKkxnX39InMvxS0lMenq6/aFqQHN1N0f0S3LaG9RuTcz/VE3qyO1Ik1aAockR6qqCwbfFxJuO0g==
dependencies:
"@libp2p/interface" "^1.6.1"
"@multiformats/multiaddr" "^12.2.3"
debug "^4.3.4"
interface-datastore "^8.2.11"
multiformats "^13.1.0"
"@libp2p/logger@^4.0.11", "@libp2p/logger@^4.0.6":
version "4.0.11"
resolved "https://registry.yarnpkg.com/@libp2p/logger/-/logger-4.0.11.tgz#671692a0cceee73a0c0bf9b5f05ea14fde05f5e5"
@@ -1845,6 +1904,19 @@
uint8-varint "^2.0.1"
uint8arrays "^5.0.0"
"@multiformats/multiaddr@^12.2.3":
version "12.3.0"
resolved "https://registry.yarnpkg.com/@multiformats/multiaddr/-/multiaddr-12.3.0.tgz#b1422813446e5cdec4b0f6cba51f93239f390884"
integrity sha512-JQ8Gc/jgucqqvEaDTFN/AvxlYDHEE7lgEWLMYW7hKZkWggER+GvG/tVxUgUxIP8M0vFpvEHKKHE0lKzyMsgi8Q==
dependencies:
"@chainsafe/is-ip" "^2.0.1"
"@chainsafe/netmask" "^2.0.0"
"@libp2p/interface" "^1.0.0"
"@multiformats/dns" "^1.0.3"
multiformats "^13.0.0"
uint8-varint "^2.0.1"
uint8arrays "^5.0.0"
"@napi-rs/snappy-android-arm-eabi@7.2.2":
version "7.2.2"
resolved "https://registry.yarnpkg.com/@napi-rs/snappy-android-arm-eabi/-/snappy-android-arm-eabi-7.2.2.tgz#85fee3ba198dad4b444b5f12bceebcf72db0d65e"
@@ -2061,7 +2133,7 @@
"@npmcli/fs@^1.0.0":
version "1.1.1"
resolved "https://registry.npmjs.org/@npmcli/fs/-/fs-1.1.1.tgz"
resolved "https://registry.yarnpkg.com/@npmcli/fs/-/fs-1.1.1.tgz#72f719fe935e687c56a4faecf3c03d06ba593257"
integrity sha512-8KG5RD0GVP4ydEzRn/I4BNDuxDtqVbOdm8675T49OIG/NGhaK0pjPX7ZcDlvKYbA+ulvVK3ztfcF4uBdOxuJbQ==
dependencies:
"@gar/promisify" "^1.0.1"
@@ -7407,7 +7479,7 @@ http-proxy-agent@^7.0.0:
agent-base "^7.1.0"
debug "^4.3.4"
http-proxy-agent@^7.0.2:
http-proxy-agent@^7.0.1, http-proxy-agent@^7.0.2:
version "7.0.2"
resolved "https://registry.yarnpkg.com/http-proxy-agent/-/http-proxy-agent-7.0.2.tgz#9a8b1f246866c028509486585f62b8f2c18c270e"
integrity sha512-T1gkAiYYDWYx3V5Bmyu7HcfcvL7mUrTWiM6yOfa3PIphViJ/gFPbvidQ+veqSOHci/PxBcDabeUNCzpOODJZig==
@@ -7453,14 +7525,6 @@ https-proxy-agent@^5.0.0:
agent-base "6"
debug "4"
https-proxy-agent@^7.0.0, https-proxy-agent@^7.0.4:
version "7.0.4"
resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-7.0.4.tgz#8e97b841a029ad8ddc8731f26595bad868cb4168"
integrity sha512-wlwpilI7YdjSkWaQ/7omYBMTliDcmCN8OLihO6I9B86g06lMyAoqgoDpV0XqoaPOKj+0DIdAvnsWfyAAhmimcg==
dependencies:
agent-base "^7.0.2"
debug "4"
https-proxy-agent@^7.0.2:
version "7.0.2"
resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-7.0.2.tgz#e2645b846b90e96c6e6f347fb5b2e41f1590b09b"
@@ -7469,6 +7533,22 @@ https-proxy-agent@^7.0.2:
agent-base "^7.0.2"
debug "4"
https-proxy-agent@^7.0.3, https-proxy-agent@^7.0.5:
version "7.0.5"
resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-7.0.5.tgz#9e8b5013873299e11fab6fd548405da2d6c602b2"
integrity sha512-1e4Wqeblerz+tMKPIq2EMGiiWW1dIjZOksyHWSUm1rmuvw/how9hBHZ38lAGj5ID4Ik6EdkOw7NmWPy6LAwalw==
dependencies:
agent-base "^7.0.2"
debug "4"
https-proxy-agent@^7.0.4:
version "7.0.4"
resolved "https://registry.yarnpkg.com/https-proxy-agent/-/https-proxy-agent-7.0.4.tgz#8e97b841a029ad8ddc8731f26595bad868cb4168"
integrity sha512-wlwpilI7YdjSkWaQ/7omYBMTliDcmCN8OLihO6I9B86g06lMyAoqgoDpV0XqoaPOKj+0DIdAvnsWfyAAhmimcg==
dependencies:
agent-base "^7.0.2"
debug "4"
human-signals@^2.1.0:
version "2.1.0"
resolved "https://registry.yarnpkg.com/human-signals/-/human-signals-2.1.0.tgz#dc91fcba42e4d06e4abaed33b3e7a3c02f514ea0"
@@ -7675,16 +7755,19 @@ internal-slot@^1.0.5:
has "^1.0.3"
side-channel "^1.0.4"
ip-address@^9.0.5:
version "9.0.5"
resolved "https://registry.yarnpkg.com/ip-address/-/ip-address-9.0.5.tgz#117a960819b08780c3bd1f14ef3c1cc1d3f3ea5a"
integrity sha512-zHtQzGojZXTwZTHQqra+ETKd4Sn3vgi7uBmlPoXVWZqYvuKmtI0l/VZTjqGmJY9x88GGOaZ9+G9ES8hC4T4X8g==
dependencies:
jsbn "1.1.0"
sprintf-js "^1.1.3"
ip-regex@^2.1.0:
version "2.1.0"
resolved "https://registry.npmjs.org/ip-regex/-/ip-regex-2.1.0.tgz"
integrity sha1-+ni/XS5pE8kRzp+BnuUUa7bYROk=
ip@^1.1.8:
version "1.1.9"
resolved "https://registry.yarnpkg.com/ip/-/ip-1.1.9.tgz#8dfbcc99a754d07f425310b86a99546b1151e396"
integrity sha512-cyRxvOEpNHNtchU3Ln9KC/auJgup87llfQpQ+t5ghoC/UhL16SWzbueiCsdTnWmqAWl7LadfuwhlqmtOaqMHdQ==
ip@^2.0.0:
version "2.0.1"
resolved "https://registry.yarnpkg.com/ip/-/ip-2.0.1.tgz#e8f3595d33a3ea66490204234b77636965307105"
@@ -8301,6 +8384,11 @@ js-yaml@^3.10.0:
argparse "^1.0.7"
esprima "^4.0.0"
jsbn@1.1.0:
version "1.1.0"
resolved "https://registry.yarnpkg.com/jsbn/-/jsbn-1.1.0.tgz#b01307cb29b618a1ed26ec79e911f803c4da0040"
integrity sha512-4bYVV3aAMtDTTu4+xsDYa6sy9GyJ69/amsu9sYF2zqjiEoZA5xJi3BrfX3uY+/IekIu7MwdObdbDWpoZdBv3/A==
jsdom@^23.0.1:
version "23.0.1"
resolved "https://registry.yarnpkg.com/jsdom/-/jsdom-23.0.1.tgz#ede7ff76e89ca035b11178d200710d8982ebfee0"
@@ -8972,7 +9060,7 @@ make-fetch-happen@^11.0.0, make-fetch-happen@^11.0.1, make-fetch-happen@^11.1.1:
make-fetch-happen@^9.1.0:
version "9.1.0"
resolved "https://registry.npmjs.org/make-fetch-happen/-/make-fetch-happen-9.1.0.tgz"
resolved "https://registry.yarnpkg.com/make-fetch-happen/-/make-fetch-happen-9.1.0.tgz#53085a09e7971433e6765f7971bf63f4e05cb968"
integrity sha512-+zopwDy7DNknmwPQplem5lAZX/eCOzSvSNNcSKm5eVwTkOBzoktEfXsa9L23J/GIRhxRsaxzkPEhrJEpE2F4Gg==
dependencies:
agentkeepalive "^4.1.3"
@@ -9584,7 +9672,7 @@ node-gyp-build@^4.3.0:
node-gyp@^8.4.0:
version "8.4.1"
resolved "https://registry.npmjs.org/node-gyp/-/node-gyp-8.4.1.tgz"
resolved "https://registry.yarnpkg.com/node-gyp/-/node-gyp-8.4.1.tgz#3d49308fc31f768180957d6b5746845fbd429937"
integrity sha512-olTJRgUtAb/hOXG0E93wZDs5YiJlgbXxTwQAFHyNlRsXQnYzUaF2aGgujZbw+hR8aF4ZG/rST57bWMWD16jr9w==
dependencies:
env-paths "^2.2.0"
@@ -10259,27 +10347,26 @@ p-waterfall@2.1.1:
dependencies:
p-reduce "^2.0.0"
pac-proxy-agent@^7.0.0:
version "7.0.1"
resolved "https://registry.yarnpkg.com/pac-proxy-agent/-/pac-proxy-agent-7.0.1.tgz#6b9ddc002ec3ff0ba5fdf4a8a21d363bcc612d75"
integrity sha512-ASV8yU4LLKBAjqIPMbrgtaKIvxQri/yh2OpI+S6hVa9JRkUI3Y3NPFbfngDtY7oFtSMD3w31Xns89mDa3Feo5A==
pac-proxy-agent@^7.0.1:
version "7.0.2"
resolved "https://registry.yarnpkg.com/pac-proxy-agent/-/pac-proxy-agent-7.0.2.tgz#0fb02496bd9fb8ae7eb11cfd98386daaac442f58"
integrity sha512-BFi3vZnO9X5Qt6NRz7ZOaPja3ic0PhlsmCRYLOpN11+mWBCR6XJDqW5RF3j8jm4WGGQZtBA+bTfxYzeKW73eHg==
dependencies:
"@tootallnate/quickjs-emscripten" "^0.23.0"
agent-base "^7.0.2"
debug "^4.3.4"
get-uri "^6.0.1"
http-proxy-agent "^7.0.0"
https-proxy-agent "^7.0.2"
pac-resolver "^7.0.0"
socks-proxy-agent "^8.0.2"
https-proxy-agent "^7.0.5"
pac-resolver "^7.0.1"
socks-proxy-agent "^8.0.4"
pac-resolver@^7.0.0:
version "7.0.0"
resolved "https://registry.yarnpkg.com/pac-resolver/-/pac-resolver-7.0.0.tgz#79376f1ca26baf245b96b34c339d79bff25e900c"
integrity sha512-Fd9lT9vJbHYRACT8OhCbZBbxr6KRSawSovFpy8nDGshaK99S/EBhVIHp9+crhxrsZOuvLpgL1n23iyPg6Rl2hg==
pac-resolver@^7.0.1:
version "7.0.1"
resolved "https://registry.yarnpkg.com/pac-resolver/-/pac-resolver-7.0.1.tgz#54675558ea368b64d210fd9c92a640b5f3b8abb6"
integrity sha512-5NPgf87AT2STgwa2ntRMr45jTKrYBGkVU36yT0ig/n/GMAa3oPqhZfIQ2kMEimReg0+t9kZViDVZ83qfVUlckg==
dependencies:
degenerator "^5.0.0"
ip "^1.1.8"
netmask "^2.0.2"
pacote@^15.2.0:
@@ -10683,19 +10770,19 @@ proxy-addr@^2.0.7:
forwarded "0.2.0"
ipaddr.js "1.9.1"
proxy-agent@6.3.0:
version "6.3.0"
resolved "https://registry.yarnpkg.com/proxy-agent/-/proxy-agent-6.3.0.tgz#72f7bb20eb06049db79f7f86c49342c34f9ba08d"
integrity sha512-0LdR757eTj/JfuU7TL2YCuAZnxWXu3tkJbg4Oq3geW/qFNT/32T0sp2HnZ9O0lMR4q3vwAt0+xCA8SR0WAD0og==
proxy-agent@6.4.0:
version "6.4.0"
resolved "https://registry.yarnpkg.com/proxy-agent/-/proxy-agent-6.4.0.tgz#b4e2dd51dee2b377748aef8d45604c2d7608652d"
integrity sha512-u0piLU+nCOHMgGjRbimiXmA9kM/L9EHh3zL81xCdp7m+Y2pHIsnmbdDoEDoAz5geaonNR6q6+yOPQs6n4T6sBQ==
dependencies:
agent-base "^7.0.2"
debug "^4.3.4"
http-proxy-agent "^7.0.0"
https-proxy-agent "^7.0.0"
http-proxy-agent "^7.0.1"
https-proxy-agent "^7.0.3"
lru-cache "^7.14.1"
pac-proxy-agent "^7.0.0"
pac-proxy-agent "^7.0.1"
proxy-from-env "^1.1.0"
socks-proxy-agent "^8.0.1"
socks-proxy-agent "^8.0.2"
proxy-from-env@^1.1.0:
version "1.1.0"
@@ -11357,6 +11444,13 @@ semver@7.5.3:
dependencies:
lru-cache "^6.0.0"
semver@7.6.0, semver@^7.6.0:
version "7.6.0"
resolved "https://registry.yarnpkg.com/semver/-/semver-7.6.0.tgz#1a46a4db4bffcccd97b743b5005c8325f23d4e2d"
integrity sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg==
dependencies:
lru-cache "^6.0.0"
semver@^6.1.0, semver@^6.2.0, semver@^6.3.1:
version "6.3.1"
resolved "https://registry.yarnpkg.com/semver/-/semver-6.3.1.tgz#556d2ef8689146e46dcea4bfdd095f3434dffcb4"
@@ -11369,13 +11463,6 @@ semver@^7.0.0, semver@^7.1.1, semver@^7.3.2, semver@^7.3.4, semver@^7.3.5, semve
dependencies:
lru-cache "^6.0.0"
semver@^7.6.0:
version "7.6.0"
resolved "https://registry.yarnpkg.com/semver/-/semver-7.6.0.tgz#1a46a4db4bffcccd97b743b5005c8325f23d4e2d"
integrity sha512-EnwXhrlwXMk9gKu5/flx5sv/an57AkRplG3hTK68W7FRDN+k+OWBj65M7719OkA82XLBxrcX0KSHj+X5COhOVg==
dependencies:
lru-cache "^6.0.0"
serialize-error@^11.0.1:
version "11.0.3"
resolved "https://registry.yarnpkg.com/serialize-error/-/serialize-error-11.0.3.tgz#b54f439e15da5b4961340fbbd376b6b04aa52e92"
@@ -11567,13 +11654,13 @@ snappyjs@^0.7.0:
integrity sha512-u5iEEXkMe2EInQio6Wv9LWHOQYRDbD2O9hzS27GpT/lwfIQhTCnHCTqedqHIHe9ZcvQo+9au6vngQayipz1NYw==
socks-proxy-agent@^6.0.0:
version "6.1.1"
resolved "https://registry.npmjs.org/socks-proxy-agent/-/socks-proxy-agent-6.1.1.tgz"
integrity sha512-t8J0kG3csjA4g6FTbsMOWws+7R7vuRC8aQ/wy3/1OWmsgwA68zs/+cExQ0koSitUDXqhufF/YJr9wtNMZHw5Ew==
version "6.2.1"
resolved "https://registry.yarnpkg.com/socks-proxy-agent/-/socks-proxy-agent-6.2.1.tgz#2687a31f9d7185e38d530bef1944fe1f1496d6ce"
integrity sha512-a6KW9G+6B3nWZ1yB8G7pJwL3ggLy1uTzKAgCb7ttblwqdz9fMGJUuTy3uFzEP48FAs9FLILlmzDlE2JJhVQaXQ==
dependencies:
agent-base "^6.0.2"
debug "^4.3.1"
socks "^2.6.1"
debug "^4.3.3"
socks "^2.6.2"
socks-proxy-agent@^7.0.0:
version "7.0.0"
@@ -11584,15 +11671,6 @@ socks-proxy-agent@^7.0.0:
debug "^4.3.3"
socks "^2.6.2"
socks-proxy-agent@^8.0.1:
version "8.0.3"
resolved "https://registry.yarnpkg.com/socks-proxy-agent/-/socks-proxy-agent-8.0.3.tgz#6b2da3d77364fde6292e810b496cb70440b9b89d"
integrity sha512-VNegTZKhuGq5vSD6XNKlbqWhyt/40CgoEw8XxD6dhnm8Jq9IEa3nIa4HwnM8XOqU0CdB0BwWVXusqiFXfHB3+A==
dependencies:
agent-base "^7.1.1"
debug "^4.3.4"
socks "^2.7.1"
socks-proxy-agent@^8.0.2:
version "8.0.2"
resolved "https://registry.yarnpkg.com/socks-proxy-agent/-/socks-proxy-agent-8.0.2.tgz#5acbd7be7baf18c46a3f293a840109a430a640ad"
@@ -11602,7 +11680,16 @@ socks-proxy-agent@^8.0.2:
debug "^4.3.4"
socks "^2.7.1"
socks@^2.6.1, socks@^2.6.2, socks@^2.7.1:
socks-proxy-agent@^8.0.4:
version "8.0.4"
resolved "https://registry.yarnpkg.com/socks-proxy-agent/-/socks-proxy-agent-8.0.4.tgz#9071dca17af95f483300316f4b063578fa0db08c"
integrity sha512-GNAq/eg8Udq2x0eNiFkr9gRg5bA7PXEWagQdeRX4cPSG+X/8V38v637gim9bjFptMk1QWsCTr0ttrJEiXbNnRw==
dependencies:
agent-base "^7.1.1"
debug "^4.3.4"
socks "^2.8.3"
socks@^2.6.2, socks@^2.7.1:
version "2.7.1"
resolved "https://registry.yarnpkg.com/socks/-/socks-2.7.1.tgz#d8e651247178fde79c0663043e07240196857d55"
integrity sha512-7maUZy1N7uo6+WVEX6psASxtNlKaNVMlGQKkG/63nEDdLOWNbiUMoLK7X4uYoLhQstau72mLgfEWcXcwsaHbYQ==
@@ -11610,6 +11697,14 @@ socks@^2.6.1, socks@^2.6.2, socks@^2.7.1:
ip "^2.0.0"
smart-buffer "^4.2.0"
socks@^2.8.3:
version "2.8.3"
resolved "https://registry.yarnpkg.com/socks/-/socks-2.8.3.tgz#1ebd0f09c52ba95a09750afe3f3f9f724a800cb5"
integrity sha512-l5x7VUUWbjVFbafGLxPWkYsHIhEvmF85tbIeFZWc8ZPtoMyybuEhL7Jye/ooC4/d48FgOjSJXgsF/AJPYCW8Zw==
dependencies:
ip-address "^9.0.5"
smart-buffer "^4.2.0"
sonic-boom@^3.7.0:
version "3.8.0"
resolved "https://registry.yarnpkg.com/sonic-boom/-/sonic-boom-3.8.0.tgz#e442c5c23165df897d77c3c14ef3ca40dec66a66"
@@ -11707,6 +11802,11 @@ sprintf-js@^1.1.2:
resolved "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.1.2.tgz"
integrity sha512-VE0SOVEHCk7Qc8ulkWw3ntAzXuqf7S2lvwQaDLRnUeIEaKNQJzV6BwmLKhOqT61aGhfUMrXeaBk+oDGCzvhcug==
sprintf-js@^1.1.3:
version "1.1.3"
resolved "https://registry.yarnpkg.com/sprintf-js/-/sprintf-js-1.1.3.tgz#4914b903a2f8b685d17fdf78a70e917e872e444a"
integrity sha512-Oo+0REFV59/rz3gfJNKQiBlwfHaSESl1pcGyABQsnnIfWOFt6JNj5gCog2U6MLZ//IGYD+nA8nI+mTShREReaA==
sprintf-js@~1.0.2:
version "1.0.3"
resolved "https://registry.npmjs.org/sprintf-js/-/sprintf-js-1.0.3.tgz"
@@ -12089,16 +12189,7 @@ tapable@^2.2.0:
resolved "https://registry.yarnpkg.com/tapable/-/tapable-2.2.1.tgz#1967a73ef4060a82f12ab96af86d52fdb76eeca0"
integrity sha512-GNzQvQTOIP6RyTfE2Qxb8ZVlNmw0n88vp1szwWRimP02mnTsx3Wtn5qRdqY9w2XduFNUgvOwhNnQsjwCp+kqaQ==
tar-fs@3.0.4, tar-fs@^3.0.4:
version "3.0.4"
resolved "https://registry.yarnpkg.com/tar-fs/-/tar-fs-3.0.4.tgz#a21dc60a2d5d9f55e0089ccd78124f1d3771dbbf"
integrity sha512-5AFQU8b9qLfZCX9zp2duONhPmZv0hGYiBPJsyUdqMjzq/mqVpy/rEUSeHk1+YitmxugaptgBh5oDGU3VsAJq4w==
dependencies:
mkdirp-classic "^0.5.2"
pump "^3.0.0"
tar-stream "^3.1.5"
tar-fs@^3.0.5:
tar-fs@3.0.5, tar-fs@^3.0.5:
version "3.0.5"
resolved "https://registry.yarnpkg.com/tar-fs/-/tar-fs-3.0.5.tgz#f954d77767e4e6edf973384e1eb95f8f81d64ed9"
integrity sha512-JOgGAmZyMgbqpLwct7ZV8VzkEB6pxXFBVErLtb+XCOqzc6w1xiWKI9GVd6bwk68EX7eJ4DWmfXVmq8K2ziZTGg==
@@ -12109,6 +12200,15 @@ tar-fs@^3.0.5:
bare-fs "^2.1.1"
bare-path "^2.1.0"
tar-fs@^3.0.4:
version "3.0.4"
resolved "https://registry.yarnpkg.com/tar-fs/-/tar-fs-3.0.4.tgz#a21dc60a2d5d9f55e0089ccd78124f1d3771dbbf"
integrity sha512-5AFQU8b9qLfZCX9zp2duONhPmZv0hGYiBPJsyUdqMjzq/mqVpy/rEUSeHk1+YitmxugaptgBh5oDGU3VsAJq4w==
dependencies:
mkdirp-classic "^0.5.2"
pump "^3.0.0"
tar-stream "^3.1.5"
tar-fs@~2.0.1:
version "2.0.1"
resolved "https://registry.yarnpkg.com/tar-fs/-/tar-fs-2.0.1.tgz#e44086c1c60d31a4f0cf893b1c4e155dabfae9e2"
@@ -13689,20 +13789,7 @@ yargs@16.2.0, yargs@^16.2.0:
y18n "^5.0.5"
yargs-parser "^20.2.2"
yargs@17.7.1:
version "17.7.1"
resolved "https://registry.yarnpkg.com/yargs/-/yargs-17.7.1.tgz#34a77645201d1a8fc5213ace787c220eabbd0967"
integrity sha512-cwiTb08Xuv5fqF4AovYacTFNxk62th7LKJ6BL9IGUpTJrWoU7/7WdQGTP2SjKf1dUNBGzDd28p/Yfs/GI6JrLw==
dependencies:
cliui "^8.0.1"
escalade "^3.1.1"
get-caller-file "^2.0.5"
require-directory "^2.1.1"
string-width "^4.2.3"
y18n "^5.0.5"
yargs-parser "^21.1.1"
yargs@^17.1.1, yargs@^17.5.1, yargs@^17.6.2, yargs@^17.7.1:
yargs@17.7.2, yargs@^17.1.1, yargs@^17.5.1, yargs@^17.6.2, yargs@^17.7.1:
version "17.7.2"
resolved "https://registry.yarnpkg.com/yargs/-/yargs-17.7.2.tgz#991df39aca675a192b816e1e0363f9d75d2aa269"
integrity sha512-7dSzzRQ++CKnNI/krKnYRV7JKKPUXMEh61soaHKg9mrWEhzFWhFnxPxGl+69cD1Ou63C13NUPCnmIcrvqCuM6w==