Remove puppeteer-cluster + iframe filtering + health check refactor + logging improvements (0.9.0-beta.0) (#219)

* This commit removes puppeteer-cluster as a dependency in favor of
a simpler concurrency implementation, using p-queue to limit
concurrency to the number of available workers. As part of the
refactor, the custom window concurrency model in windowconcur.js
is removed and its logic implemented in the new Worker class's
initPage method.

* Remove concurrency models, always use new tab

* logging improvements: include worker-id in logs, use 'worker' context
- logging: log info string / version as first line
- logging: improve logging of error stack traces
- interruption: support interrupting crawl directly with 'interrupt' check which stops the job queue
- interruption: don't repair if interrupting, wait for queue to be idle
- log text extraction
- init order: ensure wb-manager init called first, then logs created
- logging: adjust info->debug logging
- Log no jobs available as debug

* tests: bail on first failure

* iframe filtering:
- fix filtering for about:blank iframes, support non-async shouldProcessFrame()
- filter iframes both for behaviors and for link extraction
- add 5-second timeout to link extraction, to avoid link extraction holding up crawl!
- cache filtered frames

* healthcheck/worker reuse:
- refactor healthchecker into separate class
- increment healthchecker (if provided) if new page load fails
- remove expermeintal repair functionality for now
- add healthcheck

* deps: bump puppeteer-core to 17.1.2
- bump to 0.9.0-beta.0

--------
Co-authored-by: Ilya Kreymer <ikreymer@gmail.com>
This commit is contained in:
Tessa Walsh 2023-03-08 21:31:19 -05:00 committed by GitHub
parent ac5a720362
commit 1bee46b321
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
13 changed files with 622 additions and 334 deletions

View file

@ -1,22 +1,22 @@
# Browsertrix Crawler
Browsertrix Crawler is a simplified (Chrome) browser-based high-fidelity crawling system, designed to run a complex, customizable browser-based crawl in a single Docker container. Browsertrix Crawler uses [puppeteer-cluster](https://github.com/thomasdondorf/puppeteer-cluster)
and [puppeteer](https://github.com/puppeteer/puppeteer) to control one or more browsers in parallel.
Browsertrix Crawler is a simplified (Chrome) browser-based high-fidelity crawling system, designed to run a complex, customizable browser-based crawl in a single Docker container. Browsertrix Crawler uses [puppeteer](https://github.com/puppeteer/puppeteer) to control one or more browser pages or windows in parallel.
## Features
Thus far, Browsertrix Crawler supports:
- Single-container, browser based crawling with multiple headless/headful browsers.
- Support for custom browser behaviors, using [Browsertix Behaviors](https://github.com/webrecorder/browsertrix-behaviors) including autoscroll, video autoplay and site-specific behaviors.
- Single-container, browser based crawling with a headless/headful browser running multiple pages/windows.
- Support for custom browser behaviors, using [Browsertrix Behaviors](https://github.com/webrecorder/browsertrix-behaviors) including autoscroll, video autoplay and site-specific behaviors.
- YAML-based configuration, passed via file or via stdin.
- Seed lists and per-seed scoping rules.
- URL blocking rules to block capture of specific URLs (including by iframe URL and/or by iframe contents).
- Screencasting: Ability to watch crawling in real-time (experimental).
- Screenshotting: Ability to take thumbnails, full page screenshots, and/or screenshots of the initial page view.
- Optimized (non-browser) capture of non-HTML resources.
- Extensible Puppeteer driver script for customizing behavior per crawl or page.
- Ability to create and reuse browser profiles interactively or via automated user/password login using an embedded browser.
- Multi-platform support -- prebuilt Docker images available for Intel/AMD and Apple (M1) CPUs.
- Multi-platform support -- prebuilt Docker images available for Intel/AMD and Apple Silicon (M1/M2) CPUs.
## Getting Started
@ -663,13 +663,7 @@ Details for each corresponding release tag are also available on GitHub at: http
The Docker container provided here packages up several components used in Browsertrix.
The system uses:
- `oldwebtoday/chrome` or `oldwebtoday/chromium` - to install a recent version of Chrome (currently chrome:90) or Chromium (see below).
- `puppeteer-cluster` - for running Chrome browsers in parallel
- `pywb` - in recording mode for capturing the content
The crawl produces a single pywb collection, at `/crawls/collections/<collection name>` in the Docker container.
The system uses `pywb` in recording mode for capturing the content. The crawl produces a single pywb collection, at `/crawls/collections/<collection name>` in the Docker container.
To access the contents of the crawl, the `/crawls` directory in the container should be mounted to a volume (default in the Docker Compose setup).
@ -697,31 +691,26 @@ docker-compose run crawler crawl --url https://webrecorder.net/ --generateCDX --
In this example, the crawl data is written to `./crawls/collections/wr-net` by default.
While the crawl is running, the status of the crawl (provide by puppeteer-cluster monitoring) prints the progress to the Docker log.
While the crawl is running, the status of the crawl prints the progress to the JSON log output. This can be disabled by using the `--logging` option and not including `stats`.
### Multi-Platform Build / Support for Apple M1
### Multi-Platform Build / Support for Apple Silicon (M1/M2)
Browsertrix Crawler uses a browser image which supports amd64 and arm64 (currently `oldwebtoday/chrome:91`).
Browsertrix Crawler uses a browser image which supports amd64 and arm64.
This means Browsertrix Crawler can be built natively on Apple M1 systems using the default settings. Simply running `docker-compose build` on an Apple M1 should build a native version that should work for development.
This means Browsertrix Crawler can be built natively on Apple Silicon systems using the default settings. Simply running `docker-compose build` on an Apple Silicon should build a native version that should work for development.
On M1 system, the browser used will be Chromium instead of Chrome since there is no Linux build of Chrome for ARM, and this now is handled automatically as part of the build. Note that Chromium is different than Chrome, and for example, some video codecs may not be supported in the ARM / Chromium-based version that would be in the amd64 / Chrome version. For production crawling, it is recommended to run on an amd64 Linux environment.
On an Apple Silicon system, the browser used will be Chromium instead of Chrome since there is no Linux build of Chrome for ARM, and this now is handled automatically as part of the build. Note that Chromium is different than Chrome, and for example, some video codecs may not be supported in the ARM / Chromium-based version that would be in the amd64 / Chrome version. For production crawling, it is recommended to run on an amd64 Linux environment.
### Custom Browser Image
### Modifying Browser Image
It is also possible to build Browsertrix Crawler with a different browser image. Currently, browser images from `oldwebtoday/chrome` and `oldwebtoday/chromium` are supported.
It is also possible to build Browsertrix Crawler with a different browser image. Currently, browser images using Chrome/Chromium (depending on host system chip architecture) and Brave Browser are supported via [browsertrix-browser-base](https://github.com/webrecorder/browsertrix-browser-base).
For example, Webrecorder provides the `oldwebtoday/chromium:91-arm` for running Browsertrix Crawler on ARM-based systems.
The browser base image used is specified and can be changed at the top of the Dockerfile in this repo.
To build with this specific Chromium image on an Apple M1 machine, run:
Custom browser images can be used by forking [browsertrix-browser-base](https://github.com/webrecorder/browsertrix-browser-base), locally building or publishing an image, and then modifying the Dockerfile in this repo to build from that image.
```
docker-compose build --build-arg BROWSER_IMAGE_BASE=oldwebtoday/chromium --build-arg "BROWSER_VERSION=91-arm" --build-arg BROWSER_BIN=chromium-browser
```
The build arguments specify the base image, version and browser binary. This approach can also be used to install a different browser in general from any Debian-based Docker image. Additional browser images may be added in the future.
### Viewing crawled data with pywb
@ -749,3 +738,29 @@ License
[AGPLv3](https://www.gnu.org/licenses/agpl-3.0) or later, see
[LICENSE](LICENSE) for more details.
Contains portions of code borrowed and refactored from Thomas Dondorf's [puppeteer-cluster](https://github.com/thomasdondorf/puppeteer-cluster), which is licensed with the MIT License, included here in its entirety to comply with its licensing terms:
<details>
<summary>MIT License
Copyright (c) 2019 Thomas Dondorf</summary>
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in all
copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
SOFTWARE.
</details>

View file

@ -3,12 +3,8 @@ import path from "path";
import fs from "fs";
import os from "os";
import fsp from "fs/promises";
import http from "http";
import url from "url";
import fetch from "node-fetch";
import puppeteer from "puppeteer-core";
import { Cluster } from "puppeteer-cluster";
import { RedisCrawlState, MemoryCrawlState } from "./util/state.js";
import AbortController from "abort-controller";
import Sitemapper from "sitemapper";
@ -17,13 +13,15 @@ import yaml from "js-yaml";
import * as warcio from "warcio";
import { HealthChecker } from "./util/healthcheck.js";
import { TextExtract } from "./util/textextract.js";
import { initStorage, getFileSize, getDirSize, interpolateFilename } from "./util/storage.js";
import { ScreenCaster, WSTransport, RedisPubSubTransport } from "./util/screencaster.js";
import { Screenshots } from "./util/screenshots.js";
import { parseArgs } from "./util/argParser.js";
import { initRedis } from "./util/redis.js";
import { Logger, setExternalLogStream } from "./util/logger.js";
import { Logger, errJSON, setExternalLogStream, setDebugLogging } from "./util/logger.js";
import { WorkerPool } from "./util/worker.js";
import { getBrowserExe, loadProfile, chromeArgs, getDefaultUA, evaluateWithCLI } from "./util/browser.js";
@ -44,6 +42,10 @@ const HTTP_AGENT = HTTPAgent();
const behaviors = fs.readFileSync(new URL("./node_modules/browsertrix-behaviors/dist/behaviors.js", import.meta.url), {encoding: "utf8"});
//todo: move elsewhere?
const LINK_EXTRACT_TIMEOUT_SECS = 5;
// ============================================================================
export class Crawler {
constructor() {
@ -55,12 +57,12 @@ export class Crawler {
this.collDir = path.join(this.params.cwd, "collections", this.params.collection);
this.logDir = path.join(this.collDir, "logs");
this.logFilename = path.join(this.logDir, `crawl-${new Date().toISOString().replace(/[^\d]/g, "")}.log`);
this.logFH = fs.createWriteStream(this.logFilename);
const debugLogging = this.params.logging.includes("debug");
this.logger = new Logger(debugLogging);
setDebugLogging(debugLogging);
this.logger = new Logger();
this.logger.debug("Writing log to: " + this.logFilename, {}, "init");
setExternalLogStream(this.logFH);
this.headers = {};
this.crawlState = null;
@ -83,14 +85,8 @@ export class Crawler {
this.lastSaveTime = 0;
this.saveStateInterval = this.params.saveStateInterval * 1000;
if (this.params.profile) {
this.logger.info(`With Browser Profile: ${this.params.profile}"`);
}
this.emulateDevice = this.params.emulateDevice;
this.logger.info("Seeds", this.params.scopedSeeds);
this.captureBasePrefix = `http://${process.env.PROXY_HOST}:${process.env.PROXY_PORT}/${this.params.collection}/record`;
this.capturePrefix = process.env.NO_PROXY ? "" : this.captureBasePrefix + "/id_/";
@ -108,7 +104,7 @@ export class Crawler {
this.blockRules = null;
this.adBlockRules = null;
this.errorCount = 0;
this.healthChecker = null;
this.interrupted = false;
this.finalExit = false;
@ -169,18 +165,18 @@ export class Crawler {
}
}
this.logger.info(`Storing state via Redis ${redisUrl} @ key prefix "${this.crawlId}"`, {}, "state");
this.logger.debug(`Storing state via Redis ${redisUrl} @ key prefix "${this.crawlId}"`, {}, "state");
this.crawlState = new RedisCrawlState(redis, this.params.crawlId, this.params.behaviorTimeout + this.params.timeout, os.hostname());
} else {
this.logger.info("Storing state in memory", {}, "state");
this.logger.debug("Storing state in memory", {}, "state");
this.crawlState = new MemoryCrawlState();
}
if (this.params.saveState === "always" && this.params.saveStateInterval) {
this.logger.info(`Saving crawl state every ${this.params.saveStateInterval} seconds, keeping last ${this.params.saveStateHistory} states`, {}, "state");
this.logger.debug(`Saving crawl state every ${this.params.saveStateInterval} seconds, keeping last ${this.params.saveStateHistory} states`, {}, "state");
}
return this.crawlState;
@ -205,22 +201,33 @@ export class Crawler {
}
async bootstrap() {
if (this.params.overwrite) {
this.logger.info(`Clearing ${this.collDir} before starting`);
try {
fs.rmSync(this.collDir, { recursive: true, force: true });
} catch(e) {
this.logger.error(`Unable to clear ${this.collDir}`, e);
}
}
const initRes = child_process.spawnSync("wb-manager", ["init", this.params.collection], {cwd: this.params.cwd});
if (initRes.status) {
this.logger.info("wb-manager init failed, collection likely already exists");
}
await fsp.mkdir(this.logDir, {recursive: true});
fs.mkdirSync(this.logDir, {recursive: true});
this.logFH = fs.createWriteStream(this.logFilename);
setExternalLogStream(this.logFH);
this.infoString = await this.getInfoString();
this.logger.info(this.infoString);
this.logger.info("Seeds", this.params.scopedSeeds);
if (this.params.profile) {
this.logger.info("With Browser Profile", {url: this.params.profile});
}
if (this.params.overwrite) {
this.logger.debug(`Clearing ${this.collDir} before starting`);
try {
fs.rmSync(this.collDir, { recursive: true, force: true });
} catch(e) {
this.logger.error(`Unable to clear ${this.collDir}`, e);
}
}
let opts = {};
let redisStdio;
@ -276,7 +283,7 @@ export class Crawler {
}
get puppeteerArgs() {
// Puppeter Options
// Puppeteer Options
return {
headless: this.params.headless,
executablePath: this.browserExe,
@ -359,10 +366,10 @@ export class Crawler {
}
}
isInScope({seedId, url, depth, extraHops} = {}) {
isInScope({seedId, url, depth, extraHops} = {}, logDetails = {}) {
const seed = this.params.scopedSeeds[seedId];
return seed.isIncluded(url, depth, extraHops);
return seed.isIncluded(url, depth, extraHops, logDetails);
}
async crawlPage(opts) {
@ -371,9 +378,9 @@ export class Crawler {
const {page, data} = opts;
const {url} = data;
const logDetails = {page: url};
const logDetails = {page: url, workerid: page._workerid};
if (!this.isInScope(data)) {
if (!this.isInScope(data, logDetails)) {
this.logger.info("Page no longer in scope", data);
return;
}
@ -406,7 +413,7 @@ export class Crawler {
if (this.params.screenshot) {
if (!page.isHTMLPage) {
this.logger.info("Skipping screenshots for non-HTML page", logDetails);
this.logger.debug("Skipping screenshots for non-HTML page", logDetails);
}
const archiveDir = path.join(this.collDir, "archive");
const screenshots = new Screenshots({page, url, directory: archiveDir});
@ -423,6 +430,7 @@ export class Crawler {
let text = "";
if (this.params.text && page.isHTMLPage) {
this.logger.debug("Extracting text", logDetails, "general");
const client = await page.target().createCDPSession();
const result = await client.send("DOM.getDocument", {"depth": -1, "pierce": true});
text = await new TextExtract(result).parseTextFromDom();
@ -432,18 +440,15 @@ export class Crawler {
if (this.params.behaviorOpts) {
if (!page.isHTMLPage) {
this.logger.info("Skipping behaviors for non-HTML page", logDetails, "behavior");
this.logger.debug("Skipping behaviors for non-HTML page", logDetails, "behavior");
} else {
const behaviorTimeout = this.params.behaviorTimeout / 1000;
this.logger.info("Behaviors started", {behaviorTimeout, ...logDetails}, "behavior");
const res = await Promise.race([
this.sleep(behaviorTimeout),
Promise.allSettled(
page.frames().
filter(frame => this.shouldRunBehavior(frame, logDetails)).
map(frame => evaluateWithCLI(frame, "self.__bx_behaviors.run();", logDetails, "behavior"))
)
this.runBehaviors(page, logDetails)
]);
if (res && res.length) {
this.logger.info("Behaviors finished", {finished: res.length, ...logDetails}, "behavior");
} else {
@ -459,12 +464,28 @@ export class Crawler {
await this.serializeConfig();
} catch (e) {
this.logger.error("Page Errored", {...e, ...logDetails}, "pageStatus");
this.logger.error("Page Errored", {...errJSON(e), ...logDetails}, "pageStatus");
await this.markPageFailed(page);
}
}
async shouldRunBehavior(frame, logDetails) {
async runBehaviors(page, logDetails) {
try {
const frames = page.__filteredFrames;
this.logger.info("Running behaviors", {frames: frames.length, frameUrls: frames.map(frame => frame.url()), ...logDetails}, "behavior");
return await Promise.allSettled(
frames.map(frame => evaluateWithCLI(frame, "self.__bx_behaviors.run();", logDetails, "behavior"))
);
} catch (e) {
this.logger.warn("Behavior run failed", {...errJSON(e), ...logDetails}, "behavior");
return null;
}
}
shouldIncludeFrame(frame, logDetails) {
if (!frame.parentFrame()) {
return true;
}
@ -476,25 +497,30 @@ export class Crawler {
if (frameUrl === "about:blank") {
res = false;
} else {
res = !(await this.adBlockRules.shouldBlock(null, frameUrl, logDetails));
res = !this.adBlockRules.isAdUrl(frameUrl);
}
if (!res) {
this.logger.info("Skipping behavior for frame", {frameUrl, ...logDetails}, "behavior");
this.logger.debug("Skipping processing frame", {frameUrl, ...logDetails}, "behavior");
}
return res;
}
async createWARCInfo(filename) {
const warcVersion = "WARC/1.0";
const type = "warcinfo";
async getInfoString() {
const packageFileJSON = JSON.parse(await fsp.readFile("../app/package.json"));
const warcioPackageJSON = JSON.parse(await fsp.readFile("/app/node_modules/warcio/package.json"));
const pywbVersion = child_process.execSync("pywb -V", {encoding: "utf8"}).trim().split(" ")[1];
return `Browsertrix-Crawler ${packageFileJSON.version} (with warcio.js ${warcioPackageJSON.version} pywb ${pywbVersion})`;
}
async createWARCInfo(filename) {
const warcVersion = "WARC/1.0";
const type = "warcinfo";
const info = {
"software": `Browsertrix-Crawler ${packageFileJSON.version} (with warcio.js ${warcioPackageJSON.version} pywb ${pywbVersion})`,
"software": this.infoString,
"format": "WARC File Format 1.0"
};
@ -504,24 +530,6 @@ export class Crawler {
return buffer;
}
async healthCheck(req, res) {
const threshold = this.params.workers * 2;
const pathname = url.parse(req.url).pathname;
switch (pathname) {
case "/healthz":
if (this.errorCount < threshold) {
this.logger.debug(`health check ok, num errors ${this.errorCount} < ${threshold}`);
res.writeHead(200);
res.end();
}
return;
}
this.logger.error(`health check failed: ${this.errorCount} >= ${threshold}`);
res.writeHead(503);
res.end();
}
async checkLimits() {
let interrupt = false;
@ -553,6 +561,7 @@ export class Crawler {
gracefulFinish() {
this.crawlState.setDrain(true);
this.interrupted = true;
this.workerPool.interrupt();
if (!this.params.waitOnDone) {
this.finalExit = true;
}
@ -578,9 +587,7 @@ export class Crawler {
this.profileDir = await loadProfile(this.params.profile);
if (this.params.healthCheckPort) {
this.healthServer = http.createServer((...args) => this.healthCheck(...args));
this.logger.info(`Healthcheck server started on ${this.params.healthCheckPort}`);
this.healthServer.listen(this.params.healthCheckPort);
this.healthChecker = new HealthChecker(this.params.healthCheckPort, this.params.workers);
}
try {
@ -621,20 +628,10 @@ export class Crawler {
this.storage = initStorage();
}
// Puppeteer Cluster init and options
this.cluster = await Cluster.launch({
concurrency: this.params.newContext,
maxConcurrency: this.params.workers,
skipDuplicateUrls: false,
// effectively disable
timeout: 1e8,
puppeteerOptions: this.puppeteerArgs,
puppeteer,
monitor: false
});
this.cluster.jobQueue = this.crawlState;
if (initState === "finalize") {
await this.postCrawl();
return;
}
await this.crawlState.setStatus("running");
@ -642,8 +639,6 @@ export class Crawler {
await this.crawlState.load(this.params.state, this.params.scopedSeeds, true);
}
this.cluster.task((opts) => this.crawlPage(opts));
await this.initPages();
this.adBlockRules = new AdBlockRules(this.captureBasePrefix, this.params.adBlockMessage, this.logger);
@ -654,10 +649,6 @@ export class Crawler {
this.screencaster = this.initScreenCaster();
if (this.cluster.browser.setScreencaster) {
this.cluster.browser.setScreencaster(this.screencaster);
}
for (let i = 0; i < this.params.scopedSeeds.length; i++) {
const seed = this.params.scopedSeeds[i];
if (!await this.queueUrl(i, seed.url, 0, 0)) {
@ -671,8 +662,18 @@ export class Crawler {
}
}
await this.cluster.idle();
await this.cluster.close();
this.workerPool = new WorkerPool({
maxConcurrency: this.params.workers,
puppeteerOptions: this.puppeteerArgs,
crawlState: this.crawlState,
screencaster: this.screencaster,
healthChecker: this.healthChecker,
task: (opts) => this.crawlPage(opts)
});
await this.workerPool.work();
await this.workerPool.close();
await this.serializeConfig(true);
@ -686,6 +687,10 @@ export class Crawler {
// extra wait for all resources to land into WARCs
await this.awaitPendingClear();
await this.postCrawl();
}
async postCrawl() {
if (this.params.combineWARC) {
await this.combineWARC();
}
@ -808,7 +813,7 @@ export class Crawler {
awaitProcess(proc) {
proc.stdout.on("data", (data) => {
this.logger.info(data.toString());
this.logger.debug(data.toString());
});
proc.stderr.on("data", (data) => {
@ -851,7 +856,7 @@ export class Crawler {
async loadPage(page, urlData, selectorOptsList = DEFAULT_SELECTORS) {
const {url, seedId, depth, extraHops = 0} = urlData;
const logDetails = {page: url};
const logDetails = {page: url, workerid: page._workerid};
let isHTMLPage = true;
@ -893,24 +898,36 @@ export class Crawler {
const gotoOpts = isHTMLPage ? this.gotoOpts : "domcontentloaded";
this.logger.info("Awaiting page load", logDetails);
try {
await page.goto(url, gotoOpts);
if (this.errorCount > 0) {
this.logger.info(`Page loaded, resetting error count ${this.errorCount} to 0`, logDetails);
this.errorCount = 0;
const resp = await page.goto(url, gotoOpts);
isHTMLPage = this.isHTMLContentType(resp.headers["content-type"]);
if (this.healthChecker) {
this.healthChecker.resetErrors();
}
} catch (e) {
let msg = e.message || "";
if (!msg.startsWith("net::ERR_ABORTED") || !ignoreAbort) {
const mainMessage = e.name === "TimeoutError" ? "Page Load Timeout" : "Page Load Error";
this.logger.error(mainMessage, {msg, ...logDetails});
this.errorCount++;
if (this.healthChecker) {
this.healthChecker.incError();
}
}
}
page.isHTMLPage = isHTMLPage;
if (isHTMLPage) {
page.__filteredFrames = page.frames().filter(frame => this.shouldIncludeFrame(frame, logDetails));
} else {
page.__filteredFrames = null;
}
if (!isHTMLPage) {
this.logger.debug("Skipping link extraction for non-HTML page", logDetails);
return;
}
@ -925,15 +942,19 @@ export class Crawler {
return;
}
this.logger.debug("Extracting links");
for (const opts of selectorOptsList) {
const links = await this.extractLinks(page, opts);
await this.queueInScopeUrls(seedId, links, depth, extraHops);
const links = await this.extractLinks(page, opts, logDetails);
await this.queueInScopeUrls(seedId, links, depth, extraHops, logDetails);
}
}
async markPageFailed(page) {
page.__failed = true;
this.errorCount++;
if (this.healthChecker) {
this.healthChecker.incError();
}
if (this.screencaster) {
await this.screencaster.endTarget(page.target());
}
@ -950,12 +971,12 @@ export class Crawler {
try {
await page.waitForNetworkIdle({timeout: this.params.netIdleWait * 1000});
} catch (e) {
this.logger.info("waitForNetworkIdle timed out, ignoring", details);
this.logger.debug("waitForNetworkIdle timed out, ignoring", details);
// ignore, continue
}
}
async extractLinks(page, {selector = "a[href]", extract = "href", isAttribute = false} = {}) {
async extractLinks(page, {selector = "a[href]", extract = "href", isAttribute = false} = {}, logDetails) {
const results = [];
const loadProp = (selector, extract) => {
@ -969,14 +990,24 @@ export class Crawler {
const loadFunc = isAttribute ? loadAttr : loadProp;
try {
const linkResults = await Promise.allSettled(page.frames().map(frame => frame.evaluate(loadFunc, selector, extract)));
const frames = page.__filteredFrames;
const linkResults = await Promise.allSettled(
frames.map(frame => Promise.race([frame.evaluate(loadFunc, selector, extract), this.sleep(LINK_EXTRACT_TIMEOUT_SECS)]))
);
if (linkResults) {
let i = 0;
for (const linkResult of linkResults) {
if (!linkResult) {
this.logger.warn("Link Extraction timed out in frame", {frameUrl: frames[i].url, ...logDetails});
continue;
}
if (!linkResult.value) continue;
for (const link of linkResult.value) {
results.push(link);
}
i++;
}
}
@ -986,7 +1017,7 @@ export class Crawler {
return results;
}
async queueInScopeUrls(seedId, urls, depth, extraHops = 0) {
async queueInScopeUrls(seedId, urls, depth, extraHops = 0, logDetails = {}) {
try {
depth += 1;
@ -994,7 +1025,7 @@ export class Crawler {
const newExtraHops = extraHops + 1;
for (const possibleUrl of urls) {
const res = this.isInScope({url: possibleUrl, extraHops: newExtraHops, depth, seedId});
const res = this.isInScope({url: possibleUrl, extraHops: newExtraHops, depth, seedId}, logDetails);
if (!res) {
continue;
@ -1013,8 +1044,10 @@ export class Crawler {
async checkCF(page, logDetails) {
try {
this.logger.debug("Check CF Blocking", logDetails);
while (await page.$("div.cf-browser-verification.cf-im-under-attack")) {
this.logger.info("Cloudflare Check Detected, waiting for reload...", logDetails);
this.logger.debug("Cloudflare Check Detected, waiting for reload...", logDetails);
await this.sleep(5.5);
}
} catch (e) {
@ -1023,6 +1056,7 @@ export class Crawler {
}
async queueUrl(seedId, url, depth, extraHops = 0) {
this.logger.debug(`Queuing url ${url}`);
if (this.limitHit) {
return false;
}
@ -1041,7 +1075,7 @@ export class Crawler {
if (extraHops) {
urlData.extraHops = extraHops;
}
this.cluster.queue(urlData);
await this.crawlState.push(urlData);
return true;
}
@ -1061,10 +1095,10 @@ export class Crawler {
const header = {"format": "json-pages-1.0", "id": "pages", "title": "All Pages"};
if (this.params.text) {
header["hasText"] = true;
this.logger.info("Text Extraction: Enabled");
this.logger.debug("Text Extraction: Enabled");
} else {
header["hasText"] = false;
this.logger.info("Text Extraction: Disabled");
this.logger.debug("Text Extraction: Disabled");
}
const header_formatted = JSON.stringify(header).concat("\n");
await this.pagesFH.writeFile(header_formatted);
@ -1111,26 +1145,29 @@ export class Crawler {
return true;
}
const contentType = resp.headers.get("Content-Type");
return this.isHTMLContentType(resp.headers.get("Content-Type"));
// just load if no content-type
if (!contentType) {
return true;
}
const mime = contentType.split(";")[0];
if (HTML_TYPES.includes(mime)) {
return true;
}
return false;
} catch(e) {
// can't confirm not html, so try in browser
return true;
}
}
isHTMLContentType(contentType) {
// just load if no content-type
if (!contentType) {
return true;
}
const mime = contentType.split(";")[0];
if (HTML_TYPES.includes(mime)) {
return true;
}
return false;
}
async directFetchCapture(url) {
//console.log(`Direct capture: ${this.capturePrefix}${url}`);
const abort = new AbortController();

View file

@ -1,6 +1,6 @@
{
"name": "browsertrix-crawler",
"version": "0.8.1",
"version": "0.9.0-beta.0",
"main": "browsertrix-crawler",
"type": "module",
"repository": "https://github.com/webrecorder/browsertrix-crawler",
@ -8,7 +8,7 @@
"license": "AGPL-3.0-or-later",
"scripts": {
"lint": "eslint *.js util/*.js tests/*.test.js",
"test": "yarn node --experimental-vm-modules $(yarn bin jest)"
"test": "yarn node --experimental-vm-modules $(yarn bin jest --bail 1)"
},
"dependencies": {
"@novnc/novnc": "1.4.0-beta",
@ -18,8 +18,8 @@
"ioredis": "^4.27.1",
"js-yaml": "^4.1.0",
"minio": "7.0.26",
"puppeteer-cluster": "github:ikreymer/puppeteer-cluster#async-job-queue",
"puppeteer-core": "^17.1.2",
"p-queue": "^7.3.0",
"puppeteer-core": "^19.7.2",
"request": "^2.88.2",
"sitemapper": "^3.1.2",
"uuid": "8.3.2",

View file

@ -4,11 +4,9 @@ import os from "os";
import yaml from "js-yaml";
import puppeteer from "puppeteer-core";
import { Cluster } from "puppeteer-cluster";
import yargs from "yargs";
import { hideBin } from "yargs/helpers";
import { ReuseWindowConcurrency } from "./windowconcur.js";
import { BEHAVIOR_LOG_FUNC, WAIT_UNTIL_OPTS } from "./constants.js";
import { ScopedSeed } from "./seeds.js";
import { interpolateFilename } from "./storage.js";
@ -404,14 +402,6 @@ class ArgParser {
logger.info("Note: The newContext argument is deprecated in 0.8.0. Values passed to this option will be ignored");
}
if (argv.workers > 1) {
logger.info("Window context being used to support >1 workers");
argv.newContext = ReuseWindowConcurrency;
} else {
logger.info("Page context being used with 1 worker");
argv.newContext = Cluster.CONCURRENCY_PAGE;
}
if (argv.mobileDevice) {
argv.emulateDevice = puppeteer.devices[argv.mobileDevice];
if (!argv.emulateDevice) {
@ -440,7 +430,7 @@ class ArgParser {
} else {
argv.netIdleWait = 2;
}
logger.info(`Set netIdleWait to ${argv.netIdleWait} seconds`);
//logger.debug(`Set netIdleWait to ${argv.netIdleWait} seconds`);
}
// prefer argv.include only if string or a non-empty array

View file

@ -1,6 +1,6 @@
import fs from "fs";
import { Logger } from "./logger.js";
import { Logger, errJSON } from "./logger.js";
const logger = new Logger();
@ -89,7 +89,7 @@ export class BlockRules
try {
await this.handleRequest(request, logDetails);
} catch (e) {
this.logger.warn("Error handling request", {...e, ...logDetails}, "blocking");
this.logger.warn("Error handling request", {...errJSON(e), ...logDetails}, "blocking");
}
});
}
@ -109,7 +109,7 @@ export class BlockRules
}
} catch (e) {
this.logger.debug(`Block: (${blockState}) Failed On: ${url}`, {...e, ...logDetails}, "blocking");
this.logger.debug(`Block: (${blockState}) Failed On: ${url}`, {...errJSON(e), ...logDetails}, "blocking");
}
}
@ -209,7 +209,7 @@ export class BlockRules
return !!text.match(frameTextMatch);
} catch (e) {
this.logger.debug("Error determining text match", {...e, ...logDetails}, "blocking");
this.logger.debug("Error determining text match", {...errJSON(e), ...logDetails}, "blocking");
}
}
@ -255,15 +255,19 @@ export class AdBlockRules extends BlockRules
try {
await this.handleRequest(request, logDetails);
} catch (e) {
this.logger.warn("Error handling request", {...e, ...logDetails}, "blocking");
this.logger.warn("Error handling request", {...errJSON(e), ...logDetails}, "blocking");
}
});
}
async shouldBlock(request, url, logDetails) {
isAdUrl(url) {
const fragments = url.split("/");
const domain = fragments.length > 2 ? fragments[2] : null;
if (this.adhosts.includes(domain)) {
return this.adhosts.includes(domain);
}
async shouldBlock(request, url, logDetails) {
if (this.isAdUrl(url)) {
this.logger.debug("URL blocked for being an ad", {url, ...logDetails}, "blocking");
await this.recordBlockMsg(url);
return BlockState.BLOCK_AD;

49
util/healthcheck.js Normal file
View file

@ -0,0 +1,49 @@
import http from "http";
import url from "url";
import { Logger } from "./logger.js";
const logger = new Logger();
// ===========================================================================
export class HealthChecker
{
constructor(port, errorThreshold) {
this.port = port;
this.errorCount = 0;
this.errorThreshold = errorThreshold;
this.healthServer = http.createServer((...args) => this.healthCheck(...args));
logger.info(`Healthcheck server started on ${port}`, {}, "healthcheck");
this.healthServer.listen(port);
}
async healthCheck(req, res) {
const pathname = url.parse(req.url).pathname;
switch (pathname) {
case "/healthz":
if (this.errorCount < this.errorThreshold) {
logger.debug(`health check ok, num errors ${this.errorCount} < ${this.errorThreshold}`, {}, "healthcheck");
res.writeHead(200);
res.end();
}
return;
}
logger.error(`health check failed: ${this.errorCount} >= ${this.errorThreshold}`, {}, "healthcheck");
res.writeHead(503);
res.end();
}
resetErrors() {
if (this.errorCount > 0) {
logger.info(`Page loaded, resetting error count ${this.errorCount} to 0`, {}, "healthcheck");
this.errorCount = 0;
}
}
incError() {
this.errorCount++;
}
}

20
util/job.js Normal file
View file

@ -0,0 +1,20 @@
export class Job
{
constructor(data, callbacks) {
this.data = data;
this.callbacks = callbacks;
}
getUrl() {
if (!this.data) {
return undefined;
}
if (typeof this.data === "string") {
return this.data;
}
if (typeof (this.data).url === "string") {
return (this.data).url;
}
return undefined;
}
}

View file

@ -1,24 +1,32 @@
// ===========================================================================
let logStream = null;
let debugLogging = false;
export function setExternalLogStream(logFH) {
logStream = logFH;
}
export function setDebugLogging(debugLog) {
debugLogging = debugLog;
}
// ===========================================================================
// to fix serialization of regexes for logging purposes
RegExp.prototype.toJSON = RegExp.prototype.toString;
// ===========================================================================
export function errJSON(e) {
return {"type": "exception", "message": e.message, "stack": e.stack};
}
// ===========================================================================
export class Logger
{
constructor(debugLogging=false) {
this.debugLogging = debugLogging;
}
logAsJSON(message, data, context, logLevel="info") {
if (data instanceof Error) {
data = {"type": "exception", "message": data.message, "stack": data.stack};
data = errJSON(data);
} else if (typeof data !== "object") {
data = {"message": data.toString()};
}
@ -49,7 +57,7 @@ export class Logger
}
debug(message, data={}, context="general") {
if (this.debugLogging) {
if (debugLogging) {
this.logAsJSON(message, data, context, "debug");
}
}

View file

@ -39,16 +39,16 @@ export class ScopedSeed
}
}
parseUrl(url) {
parseUrl(url, logDetails = {}) {
let parsedUrl = null;
try {
parsedUrl = new URL(url.trim());
} catch (e) {
logger.error(`Invalid Seed "${url}" - not a valid URL`);
logger.warn("Invalid Seed - not a valid URL", {url, ...logDetails});
}
if (parsedUrl.protocol !== "http:" && parsedUrl.protocol != "https:") {
logger.error(`Invalid Seed "${url}" - URL must start with http:// or https://`);
logger.warn("Invalid Seed - URL must start with http:// or https://", {url, ...logDetails});
parsedUrl = null;
}
@ -110,12 +110,12 @@ export class ScopedSeed
return depth >= this.maxDepth;
}
isIncluded(url, depth, extraHops = 0) {
isIncluded(url, depth, extraHops = 0, logDetails = {}) {
if (depth > this.maxDepth) {
return false;
}
url = this.parseUrl(url);
url = this.parseUrl(url, logDetails);
if (!url) {
return false;
}

View file

@ -1,9 +1,6 @@
import mod from "puppeteer-cluster/dist/Job.js";
import { Job } from "./job.js";
import { Logger } from "./logger.js";
const Job = mod.default;
const logger = new Logger();
@ -73,9 +70,9 @@ export class MemoryCrawlState extends BaseState
return true;
}
push(job) {
this.pending.delete(job.data.url);
this.queue.unshift(job.data);
push(urlData) {
this.pending.delete(urlData.url);
this.queue.unshift(urlData);
}
realSize() {
@ -85,6 +82,10 @@ export class MemoryCrawlState extends BaseState
shift() {
const data = this.queue.pop();
if (!data) {
return;
}
const url = data.url;
const state = this;
@ -117,7 +118,7 @@ export class MemoryCrawlState extends BaseState
}
};
return new Job(data, undefined, callbacks);
return new Job(data, callbacks);
}
has(url) {
@ -314,8 +315,8 @@ return 0;
return (res >= 3);
}
async push(job) {
await this.redis.addqueue(this.pkey, this.qkey, job.data.url, JSON.stringify(job.data));
async push(urlData) {
await this.redis.addqueue(this.pkey, this.qkey, urlData.url, JSON.stringify(urlData));
}
async shift() {
@ -352,7 +353,7 @@ return 0;
}
};
return new Job(data, undefined, callbacks);
return new Job(data, callbacks);
}
async has(url) {

View file

@ -1,118 +0,0 @@
import sbi from "puppeteer-cluster/dist/concurrency/SingleBrowserImplementation.js";
import { Logger } from "./logger.js";
const logger = new Logger();
const SingleBrowserImplementation = sbi.default;
// ===========================================================================
export class ReuseWindowConcurrency extends SingleBrowserImplementation {
async init() {
await super.init();
this.pendingTargets = new Map();
this.startPage = "about:blank?_browsertrix" + Math.random().toString(36).slice(2);
this.pages = [];
this.reuseCount = 25;
this.screencaster = null;
const mainTarget = this.browser.target();
this.cdp = await mainTarget.createCDPSession();
this.sessionId = this.cdp.id();
this.browser.on("targetcreated", (target) => {
if (target.url() === this.startPage) {
this.pendingTargets.set(target._targetId, target);
}
});
}
setScreencaster(screencaster) {
this.screencaster = screencaster;
}
async repair() {
if (this.openInstances !== 0 || this.repairing) {
// already repairing or there are still pages open? wait for start/finish
await new Promise(resolve => this.waitingForRepairResolvers.push(resolve));
return;
}
this.repairing = true;
logger.warn("Starting browser repair");
if (this.screencaster) {
this.screencaster.endAllTargets();
}
try {
// will probably fail, but just in case the repair was not necessary
await this.browser.close();
} catch (e) {
logger.warn("Unable to close browser");
}
try {
await this.init();
} catch (err) {
logger.warn("Unable to restart chrome");
}
this.repairRequested = false;
this.repairing = false;
this.waitingForRepairResolvers.forEach(resolve => resolve());
this.waitingForRepairResolvers = [];
}
async getNewPage() {
while (true) {
let targetId;
try {
const res = await this.cdp.send("Target.createTarget", {url: this.startPage, newWindow: true});
targetId = res.targetId;
} catch (e) {
logger.warn("Error getting new page in window context", e);
await this.repair();
}
const target = this.pendingTargets.get(targetId);
// this shouldn't really happen, but just in case somehow ended up w/o a target, try again
if (!target) {
continue;
}
this.pendingTargets.delete(targetId);
return {page: await target.page(), count: 0, id: this.sessionId};
}
}
async createResources() {
if (this.pages.length) {
const res = this.pages.shift();
if (res.id === this.sessionId) {
return res;
} else {
// page is using stale session (eg. from crashed/previous browser instance), don't attempt to reuse
}
}
return await this.getNewPage();
}
async freeResources(resources) {
// if marked as failed, don't try to reuse
if (resources.page.__failed) {
await resources.page.close();
}
if (++resources.count > this.reuseCount) {
await resources.page.close();
} else {
this.pages.push(resources);
}
}
}

258
util/worker.js Normal file
View file

@ -0,0 +1,258 @@
//import { EventEmitter } from "node:events";
import PQueue from "p-queue";
import puppeteer from "puppeteer-core";
import { Logger, errJSON } from "./logger.js";
const logger = new Logger();
//class BrowserEmitter extends EventEmitter {}
//const browserEmitter = new BrowserEmitter();
const MAX_REUSE = 5;
// ===========================================================================
export class Worker
{
constructor(id, browser, task, puppeteerOptions, screencaster, healthChecker) {
this.id = id;
this.browser = browser;
this.task = task;
this.puppeteerOptions = puppeteerOptions;
this.screencaster = screencaster;
this.healthChecker = healthChecker;
this.reuseCount = 0;
this.page = null;
this.startPage = "about:blank?_browsertrix" + Math.random().toString(36).slice(2);
}
async initPage() {
if (this.page && ++this.reuseCount <= MAX_REUSE) {
logger.debug("Reusing page", {reuseCount: this.reuseCount}, "worker");
return this.page;
} else if (this.page) {
try {
await this.page.close();
} catch (e) {
// ignore
}
this.page = null;
}
//open page in a new tab
this.pendingTargets = new Map();
this.browser.on("targetcreated", (target) => {
if (target.url() === this.startPage) {
this.pendingTargets.set(target._targetId, target);
}
});
this.reuseCount = 1;
while (true) {
try {
logger.debug("Opening new page", {}, "worker");
const mainTarget = this.browser.target();
this.cdp = await mainTarget.createCDPSession();
let targetId;
const res = await this.cdp.send("Target.createTarget", {url: this.startPage, newWindow: true});
targetId = res.targetId;
const target = this.pendingTargets.get(targetId);
this.pendingTargets.delete(targetId);
this.page = await target.page();
this.page._workerid = this.id;
if (this.healthChecker) {
this.healthChecker.resetErrors();
}
break;
} catch (err) {
logger.warn("Error getting new page in window context", {"workerid": this.id, ...errJSON(err)}, "worker");
await sleep(500);
logger.warn("Retry getting new page");
if (this.healthChecker) {
this.healthChecker.incError();
}
}
}
}
async runTask(job) {
if (job.callbacks) {
job.callbacks.start();
}
const urlData = job.data;
await this.initPage();
const url = job.getUrl();
logger.info("Starting page", {"workerid": this.id, "page": url}, "worker");
let result;
let errorState;
await this.task({
page: this.page,
data: urlData,
});
if (errorState) {
return {
type: "error",
error: errorState,
};
}
return {
data: result,
type: "success",
};
}
}
// ===========================================================================
export class WorkerPool
{
constructor(options) {
this.maxConcurrency = options.maxConcurrency;
this.puppeteerOptions = options.puppeteerOptions;
this.crawlState = options.crawlState;
this.screencaster = options.screencaster;
this.healthChecker = options.healthChecker;
this.task = options.task;
this.browser = null;
this.workers = [];
this.workersAvailable = [];
this.workersBusy = [];
this.interrupted = false;
this.createWorkers(this.maxConcurrency);
}
async createWorkers(numWorkers = 1) {
if (!this.browser) {
this.browser = await puppeteer.launch(this.puppeteerOptions);
}
logger.info(`Creating ${numWorkers} workers`, {}, "worker");
for (let i=0; i < numWorkers; i++) {
await this.createWorker(`worker-${i+1}`);
}
}
async createWorker(id) {
const worker = new Worker(
id,
this.browser,
this.task,
this.puppeteerOptions,
this.screencaster,
this.healthChecker
);
this.workers.push(worker);
this.workersAvailable.push(worker);
}
async getAvailableWorker() {
if (this.workersAvailable.length > 0) {
const worker = this.workersAvailable.shift();
this.workersBusy.push(worker);
return worker;
}
// wait half a second and try again
await sleep(500);
return await this.getAvailableWorker();
}
freeWorker(worker) {
const workerIndex = this.workersBusy.indexOf(worker);
this.workersBusy.splice(workerIndex, 1);
this.workersAvailable.push(worker);
}
async crawlPageInWorker() {
const worker = await this.getAvailableWorker();
const job = await this.crawlState.shift();
if (!job) {
logger.debug("No jobs available - waiting for pending pages to finish", {}, "worker");
this.freeWorker(worker);
return;
}
const result = await worker.runTask(job);
if (result.type === "error") {
if (job.callbacks) {
job.callbacks.reject(result.error);
}
if (this.healthChecker) {
this.healthChecker.incError();
}
} else if (result.type === "success") {
if (this.healthChecker) {
this.healthChecker.resetErrors();
}
if (job.callbacks) {
job.callbacks.resolve(result.data);
}
}
this.freeWorker(worker);
}
async work() {
const queue = new PQueue({concurrency: this.maxConcurrency});
while (!this.interrupted) {
if ((await this.crawlState.realSize()) + (await this.crawlState.numPending()) == 0) {
break;
}
if ((await this.crawlState.realSize()) > 0) {
(async () => {
await queue.add(() => this.crawlPageInWorker());
})();
}
// wait half a second
await sleep(500);
}
await queue.onIdle();
}
interrupt() {
logger.info("Interrupting Crawl", {}, "worker");
this.interrupted = true;
}
async close() {
if (this.browser) {
try {
await this.browser.close();
/* eslint-disable no-empty */
} catch (e) {}
}
}
}
function sleep(millis) {
return new Promise(resolve => setTimeout(resolve, millis));
}

View file

@ -1266,6 +1266,13 @@ chownr@^1.1.1:
resolved "https://registry.yarnpkg.com/chownr/-/chownr-1.1.4.tgz#6fc9d7b42d32a583596337666e7d08084da2cc6b"
integrity sha512-jJ0bqzaylmJtVnNgzTeSOs8DPavpbYgEr/b0YL8/2GO3xJEhInFmhKMUnEJQjZumK7KXGFhUy89PrsJWlakBVg==
chromium-bidi@0.4.4:
version "0.4.4"
resolved "https://registry.yarnpkg.com/chromium-bidi/-/chromium-bidi-0.4.4.tgz#44f25d4fa5d2f3debc3fc3948d0657194cac4407"
integrity sha512-4BX5cSaponuvVT1+SbLYTOAgDoVtX/Khoc9UsbFJ/AsPVUeFAM3RiIDFI6XFhLYMi9WmVJqh1ZH+dRpNKkKwiQ==
dependencies:
mitt "3.0.0"
ci-info@^3.2.0:
version "3.5.0"
resolved "https://registry.yarnpkg.com/ci-info/-/ci-info-3.5.0.tgz#bfac2a29263de4c829d806b1ab478e35091e171f"
@ -1533,10 +1540,10 @@ detect-newline@^3.0.0:
resolved "https://registry.yarnpkg.com/detect-newline/-/detect-newline-3.1.0.tgz#576f5dfc63ae1a192ff192d8ad3af6308991b651"
integrity sha512-TLz+x/vEXm/Y7P7wn1EJFNLxYpUD4TgMosxY6fAVJUnJMbupHBOncxyWUG9OpTaH9EBD7uFI5LfEgmMOc54DsA==
devtools-protocol@0.0.1036444:
version "0.0.1036444"
resolved "https://registry.yarnpkg.com/devtools-protocol/-/devtools-protocol-0.0.1036444.tgz#a570d3cdde61527c82f9b03919847b8ac7b1c2b9"
integrity sha512-0y4f/T8H9lsESV9kKP1HDUXgHxCdniFeJh6Erq+FbdOEvp/Ydp9t8kcAAM5gOd17pMrTDlFWntoHtzzeTUWKNw==
devtools-protocol@0.0.1094867:
version "0.0.1094867"
resolved "https://registry.yarnpkg.com/devtools-protocol/-/devtools-protocol-0.0.1094867.tgz#2ab93908e9376bd85d4e0604aa2651258f13e374"
integrity sha512-pmMDBKiRVjh0uKK6CT1WqZmM3hBVSgD+N2MrgyV1uNizAZMw4tx6i/RTc+/uCsKSCmg0xXx7arCP/OFcIwTsiQ==
diff-sequences@^29.2.0:
version "29.2.0"
@ -1834,6 +1841,11 @@ event-target-shim@^5.0.0:
resolved "https://registry.yarnpkg.com/event-target-shim/-/event-target-shim-5.0.1.tgz#5d4d3ebdf9583d63a5333ce2deb7480ab2b05789"
integrity sha512-i/2XbnSz/uxRCU6+NdVJgKWDTM427+MqYbkQzD321DuCQJUqOuJKIA0IM2+W2xtYHdKOmZ4dR6fExsd4SXL+WQ==
eventemitter3@^4.0.7:
version "4.0.7"
resolved "https://registry.yarnpkg.com/eventemitter3/-/eventemitter3-4.0.7.tgz#2de9b68f6528d5644ef5c59526a1b4a07306169f"
integrity sha512-8guHBZCwKnFhYdHr2ysuRWErTwhoN2X8XELRlrRwpmfeY2jjuUN4taQMsULKUVo1K4DvZl+0pgfyoysHxvmvEw==
evp_bytestokey@^1.0.0, evp_bytestokey@^1.0.3:
version "1.0.3"
resolved "https://registry.yarnpkg.com/evp_bytestokey/-/evp_bytestokey-1.0.3.tgz#7fcbdb198dc71959432efe13842684e0525acb02"
@ -3226,6 +3238,11 @@ minio@7.0.26:
xml "^1.0.0"
xml2js "^0.4.15"
mitt@3.0.0:
version "3.0.0"
resolved "https://registry.yarnpkg.com/mitt/-/mitt-3.0.0.tgz#69ef9bd5c80ff6f57473e8d89326d01c414be0bd"
integrity sha512-7dX2/10ITVyqh4aOSVI9gdape+t9l2/8QxHrFmUXu4EEUpdlxl6RudZUPZoc+zuY2hk1j7XxVroIVIan/pD/SQ==
mkdirp-classic@^0.5.2:
version "0.5.3"
resolved "https://registry.yarnpkg.com/mkdirp-classic/-/mkdirp-classic-0.5.3.tgz#fa10c9115cc6d8865be221ba47ee9bed78601113"
@ -3404,6 +3421,19 @@ p-map@^2.1.0:
resolved "https://registry.yarnpkg.com/p-map/-/p-map-2.1.0.tgz#310928feef9c9ecc65b68b17693018a665cea175"
integrity sha512-y3b8Kpd8OAN444hxfBbFfj1FY/RjtTd8tzYwhUqNYXx0fXx2iX4maP4Qr6qhIKbQXI02wTLAda4fYUbDagTUFw==
p-queue@^7.3.0:
version "7.3.0"
resolved "https://registry.yarnpkg.com/p-queue/-/p-queue-7.3.0.tgz#90dfa104894b286dc2f3638961380fb6dc262e55"
integrity sha512-5fP+yVQ0qp0rEfZoDTlP2c3RYBgxvRsw30qO+VtPPc95lyvSG+x6USSh1TuLB4n96IO6I8/oXQGsTgtna4q2nQ==
dependencies:
eventemitter3 "^4.0.7"
p-timeout "^5.0.2"
p-timeout@^5.0.2:
version "5.1.0"
resolved "https://registry.yarnpkg.com/p-timeout/-/p-timeout-5.1.0.tgz#b3c691cf4415138ce2d9cfe071dba11f0fee085b"
integrity sha512-auFDyzzzGZZZdHz3BtET9VEz0SE/uMEAx7uWfGPucfzEwwe/xH0iVeZibQmANYE/hp9T2+UUZT5m+BKyrDp3Ew==
p-try@^2.0.0:
version "2.2.0"
resolved "https://registry.yarnpkg.com/p-try/-/p-try-2.2.0.tgz#cb2868540e313d61de58fafbe35ce9004d5540e6"
@ -3529,7 +3559,7 @@ pretty-format@^29.2.1:
ansi-styles "^5.0.0"
react-is "^18.0.0"
progress@2.0.3, progress@^2.0.0:
progress@^2.0.0:
version "2.0.3"
resolved "https://registry.yarnpkg.com/progress/-/progress-2.0.3.tgz#7e8cf8d8f5b8f239c1bc68beb4eb78567d572ef8"
integrity sha512-7PiHtLll5LdnKIMw100I+8xJXR5gW2QwWYkT6iJva0bXitZKa/XMrSbdmg3r2Xnaidz9Qumd0VPaMrZlF9V9sA==
@ -3586,28 +3616,22 @@ punycode@^2.1.0, punycode@^2.1.1:
resolved "https://registry.yarnpkg.com/punycode/-/punycode-2.1.1.tgz#b58b010ac40c22c5657616c8d2c2c02c7bf479ec"
integrity sha512-XRsRjdf+j5ml+y/6GKHPZbrF/8p2Yga0JPtdqTIY2Xe5ohJPD9saDJJLPvp9+NSBprVvevdXZybnj2cv8OEd0A==
"puppeteer-cluster@github:ikreymer/puppeteer-cluster#async-job-queue":
version "0.22.0"
resolved "https://codeload.github.com/ikreymer/puppeteer-cluster/tar.gz/6e4cf4fb6c14f369428cc822a3620cf9a50e66e2"
dependencies:
debug "^4.1.1"
puppeteer-core@^17.1.2:
version "17.1.2"
resolved "https://registry.yarnpkg.com/puppeteer-core/-/puppeteer-core-17.1.2.tgz#fdf109fa2d805fdb007b5abfc83728c545ac9ac0"
integrity sha512-mUndfkp581aFC9Tj0NQzoQ4kBEiYszvubkAovAfA72cO2VgiAnk4RTeQhgPekdL50+7bvU1JZp+10y2xOBOy0g==
puppeteer-core@^19.7.2:
version "19.7.2"
resolved "https://registry.yarnpkg.com/puppeteer-core/-/puppeteer-core-19.7.2.tgz#deee9ef915829b6a1d1a3a008625c29eeb251161"
integrity sha512-PvI+fXqgP0uGJxkyZcX51bnzjFA73MODZOAv0fSD35yR7tvbqwtMV3/Y+hxQ0AMMwzxkEebP6c7po/muqxJvmQ==
dependencies:
chromium-bidi "0.4.4"
cross-fetch "3.1.5"
debug "4.3.4"
devtools-protocol "0.0.1036444"
devtools-protocol "0.0.1094867"
extract-zip "2.0.1"
https-proxy-agent "5.0.1"
progress "2.0.3"
proxy-from-env "1.1.0"
rimraf "3.0.2"
tar-fs "2.1.1"
unbzip2-stream "1.4.3"
ws "8.8.1"
ws "8.11.0"
qs@~6.5.2:
version "6.5.2"
@ -4389,10 +4413,10 @@ write-file-atomic@^4.0.1:
imurmurhash "^0.1.4"
signal-exit "^3.0.7"
ws@8.8.1:
version "8.8.1"
resolved "https://registry.yarnpkg.com/ws/-/ws-8.8.1.tgz#5dbad0feb7ade8ecc99b830c1d77c913d4955ff0"
integrity sha512-bGy2JzvzkPowEJV++hF07hAD6niYSr0JzBNo/J29WsB57A2r7Wlc1UFcTR9IzrPvuNVO4B8LGqF8qcpsVOhJCA==
ws@8.11.0:
version "8.11.0"
resolved "https://registry.yarnpkg.com/ws/-/ws-8.11.0.tgz#6a0d36b8edfd9f96d8b25683db2f8d7de6e8e143"
integrity sha512-HPG3wQd9sNQoT9xHyNCXoDUa+Xw/VevmY9FoHyQ+g+rrMn4j6FB4np7Z0OhdTgjx6MgQLK7jwSy1YecU1+4Asg==
ws@^7.4.4:
version "7.4.5"