(*) Remove the crawl spec abstraction

The crawl spec abstraction was used to upload lists of domains into the system for future crawling.  This was fairly clunky, and it was difficult to understand what was going to be crawled.

Since a while back, a new domains listing view has been added to the control view that allows direct access to the domains table.  This is much preferred and means the operator can directly manage domains without specs.

This commit removes the crawl spec abstraction from the code, and changes the GUI to direct to the domains list instead.
This commit is contained in:
Viktor Lofgren 2024-10-03 13:41:17 +02:00
parent ecb5eedeae
commit d84a2c183f
49 changed files with 341 additions and 1044 deletions

View File

@ -1,7 +1,9 @@
package nu.marginalia.storage.model;
public enum FileStorageType {
CRAWL_SPEC,
@Deprecated
CRAWL_SPEC, //
CRAWL_DATA,
PROCESSED_DATA,
BACKUP,

View File

@ -10,9 +10,10 @@ import nu.marginalia.service.discovery.property.ServicePartition;
import nu.marginalia.storage.model.FileStorageId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
import static nu.marginalia.functions.execution.api.ExecutorCrawlApiGrpc.*;
import static nu.marginalia.functions.execution.api.ExecutorCrawlApiGrpc.ExecutorCrawlApiBlockingStub;
@Singleton
public class ExecutorCrawlClient {
@ -36,14 +37,6 @@ public class ExecutorCrawlClient {
.build());
}
public void triggerRecrawl(int node, FileStorageId fid) {
channelPool.call(ExecutorCrawlApiBlockingStub::triggerRecrawl)
.forNode(node)
.run(RpcFileStorageId.newBuilder()
.setFileStorageId(fid.id())
.build());
}
public void triggerRecrawlSingleDomain(int node, FileStorageId fid, String domainName) {
channelPool.call(ExecutorCrawlApiBlockingStub::triggerSingleDomainRecrawl)
.forNode(node)

View File

@ -21,7 +21,6 @@ service ExecutorApi {
service ExecutorCrawlApi {
rpc triggerCrawl(RpcFileStorageId) returns (Empty) {}
rpc triggerRecrawl(RpcFileStorageId) returns (Empty) {}
rpc triggerSingleDomainRecrawl(RpcFileStorageIdWithDomainName) returns (Empty) {}
rpc triggerConvert(RpcFileStorageId) returns (Empty) {}
rpc triggerConvertAndLoad(RpcFileStorageId) returns (Empty) {}

View File

@ -4,8 +4,8 @@ import com.google.inject.Inject;
import gnu.trove.set.hash.TLongHashSet;
import lombok.SneakyThrows;
import nu.marginalia.hash.MurmurHash3_128;
import nu.marginalia.io.crawldata.CrawledDomainReader;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.link_parser.LinkParser;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.EdgeUrl;

View File

@ -2,8 +2,8 @@ package nu.marginalia.extractor;
import com.google.inject.Inject;
import lombok.SneakyThrows;
import nu.marginalia.io.crawldata.CrawledDomainReader;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.link_parser.FeedExtractor;
import nu.marginalia.link_parser.LinkParser;
import nu.marginalia.model.EdgeDomain;

View File

@ -5,7 +5,7 @@ import gnu.trove.map.hash.TLongIntHashMap;
import gnu.trove.set.hash.TLongHashSet;
import nu.marginalia.WmsaHome;
import nu.marginalia.converting.processor.logic.dom.DomPruningFilter;
import nu.marginalia.io.crawldata.CrawledDomainReader;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.language.filter.LanguageFilter;
import nu.marginalia.language.model.DocumentLanguageData;
import nu.marginalia.language.sentence.SentenceExtractor;

View File

@ -33,7 +33,6 @@ public class ExecutorActorControlService {
ConvertActor convertActor,
ConvertAndLoadActor convertAndLoadActor,
CrawlActor crawlActor,
RecrawlActor recrawlActor,
RecrawlSingleDomainActor recrawlSingleDomainActor,
RestoreBackupActor restoreBackupActor,
ConverterMonitorActor converterMonitorFSM,
@ -43,7 +42,6 @@ public class ExecutorActorControlService {
FileStorageMonitorActor fileStorageMonitorActor,
IndexConstructorMonitorActor indexConstructorMonitorActor,
TriggerAdjacencyCalculationActor triggerAdjacencyCalculationActor,
CrawlJobExtractorActor crawlJobExtractorActor,
ExportDataActor exportDataActor,
ExportAtagsActor exportAtagsActor,
ExportFeedsActor exportFeedsActor,
@ -59,7 +57,6 @@ public class ExecutorActorControlService {
this.node = baseServiceParams.configuration.node();
register(ExecutorActor.CRAWL, crawlActor);
register(ExecutorActor.RECRAWL, recrawlActor);
register(ExecutorActor.RECRAWL_SINGLE_DOMAIN, recrawlSingleDomainActor);
register(ExecutorActor.CONVERT, convertActor);
@ -75,7 +72,6 @@ public class ExecutorActorControlService {
register(ExecutorActor.MONITOR_FILE_STORAGE, fileStorageMonitorActor);
register(ExecutorActor.ADJACENCY_CALCULATION, triggerAdjacencyCalculationActor);
register(ExecutorActor.CRAWL_JOB_EXTRACTOR, crawlJobExtractorActor);
register(ExecutorActor.EXPORT_DATA, exportDataActor);
register(ExecutorActor.EXPORT_ATAGS, exportAtagsActor);

View File

@ -3,86 +3,118 @@ package nu.marginalia.actor.task;
import com.google.gson.Gson;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import nu.marginalia.actor.ExecutorActor;
import nu.marginalia.actor.ExecutorActorStateMachines;
import nu.marginalia.actor.prototype.RecordActorPrototype;
import nu.marginalia.actor.state.ActorResumeBehavior;
import nu.marginalia.actor.state.ActorStep;
import nu.marginalia.actor.state.Resume;
import nu.marginalia.mq.MqMessageState;
import nu.marginalia.mq.outbox.MqOutbox;
import nu.marginalia.mqapi.crawling.CrawlRequest;
import nu.marginalia.process.ProcessOutboxes;
import nu.marginalia.process.ProcessService;
import nu.marginalia.storage.FileStorageService;
import nu.marginalia.storage.model.FileStorageId;
import nu.marginalia.storage.model.FileStorageType;
import nu.marginalia.mq.MqMessageState;
import nu.marginalia.mq.outbox.MqOutbox;
import nu.marginalia.mqapi.crawling.CrawlRequest;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import nu.marginalia.svc.DomainListRefreshService;
import java.util.List;
import java.nio.file.Files;
import java.nio.file.Path;
@Singleton
public class CrawlActor extends RecordActorPrototype {
private final MqOutbox mqCrawlerOutbox;
private final FileStorageService storageService;
private final Logger logger = LoggerFactory.getLogger(getClass());
private final DomainListRefreshService refreshService;
private final ActorProcessWatcher processWatcher;
private final ExecutorActorStateMachines executorActorStateMachines;
public record Initial(FileStorageId storageId) implements ActorStep {}
/** Initial step
* @param storageId - the id of the storage to recrawl
* @param cascadeLoad - whether to automatically start the convert and load actor after the crawl
*/
public record Initial(FileStorageId storageId, boolean cascadeLoad) implements ActorStep {}
/** The action step */
@Resume(behavior = ActorResumeBehavior.RETRY)
public record Crawl(long messageId) implements ActorStep {}
public record Crawl(long messageId, FileStorageId fid, boolean cascadeLoad) implements ActorStep {}
@Override
public ActorStep transition(ActorStep self) throws Exception {
return switch (self) {
case Initial (FileStorageId fid) -> {
var storage = storageService.getStorage(fid);
if (storage == null) yield new Error("Bad storage id");
if (storage.type() != FileStorageType.CRAWL_SPEC) yield new Error("Bad storage type " + storage.type());
case Initial (FileStorageId fid, boolean cascadeLoad) when fid.id() < 0 -> {
// Allocate a new storage area if we aren't given one
var dataArea = storageService.allocateStorage(
FileStorageType.CRAWL_DATA,
"crawl-data",
storage.description());
"Crawl data");
storageService.relateFileStorages(storage.id(), dataArea.id());
// Send convert request
long msgId = mqCrawlerOutbox.sendAsync(
CrawlRequest.forSpec(fid, dataArea.id())
);
yield new Crawl(msgId);
// Now we can jump to the main track
yield new Initial(dataArea.id(), cascadeLoad);
}
case Crawl(long msgId) -> {
var rsp = processWatcher.waitResponse(mqCrawlerOutbox, ProcessService.ProcessId.CRAWLER, msgId);
case Initial (FileStorageId fid, boolean cascadeLoad) -> {
var crawlStorage = storageService.getStorage(fid);
if (rsp.state() != MqMessageState.OK)
if (crawlStorage == null) yield new Error("Bad storage id");
if (crawlStorage.type() != FileStorageType.CRAWL_DATA) yield new Error("Bad storage type " + crawlStorage.type());
Path crawlLogPath = crawlStorage.asPath().resolve("crawler.log");
if (Files.exists(crawlLogPath)) {
// Save the old crawl log
Path crawlLogBackup = crawlStorage.asPath().resolve("crawler.log-" + System.currentTimeMillis());
Files.move(crawlLogPath, crawlLogBackup);
}
refreshService.synchronizeDomainList();
long id = mqCrawlerOutbox.sendAsync(CrawlRequest.forFullCrawl(fid));
yield new Crawl(id, fid, cascadeLoad);
}
case Crawl (long msgId, FileStorageId fid, boolean cascadeLoad) -> {
var rsp = processWatcher.waitResponse(
mqCrawlerOutbox,
ProcessService.ProcessId.CRAWLER,
msgId);
if (rsp.state() != MqMessageState.OK) {
yield new Error("Crawler failed");
}
if (cascadeLoad) {
// Spawn the convert and load actor
executorActorStateMachines.initFrom(ExecutorActor.CONVERT_AND_LOAD,
new ConvertAndLoadActor.Initial(fid));
}
yield new End();
}
default -> new Error();
default -> new End();
};
}
@Override
public String describe() {
return "Run the crawler with the given crawl spec using no previous crawl data for a reference";
return "Run the crawler with the given crawl spec using previous crawl data for a reference";
}
@Inject
public CrawlActor(ProcessOutboxes processOutboxes,
public CrawlActor(ActorProcessWatcher processWatcher,
ProcessOutboxes processOutboxes,
FileStorageService storageService,
Gson gson,
ActorProcessWatcher processWatcher)
DomainListRefreshService refreshService,
ExecutorActorStateMachines executorActorStateMachines,
Gson gson)
{
super(gson);
this.processWatcher = processWatcher;
this.mqCrawlerOutbox = processOutboxes.getCrawlerOutbox();
this.storageService = storageService;
this.processWatcher = processWatcher;
this.refreshService = refreshService;
this.executorActorStateMachines = executorActorStateMachines;
}
}

View File

@ -1,83 +0,0 @@
package nu.marginalia.actor.task;
import com.google.gson.Gson;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import nu.marginalia.actor.prototype.RecordActorPrototype;
import nu.marginalia.actor.state.ActorStep;
import nu.marginalia.crawlspec.CrawlSpecFileNames;
import nu.marginalia.storage.FileStorageService;
import nu.marginalia.storage.model.FileStorageType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URL;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
import static nu.marginalia.crawlspec.CrawlSpecGenerator.*;
@Singleton
public class CrawlJobExtractorActor extends RecordActorPrototype {
private static final int INITIAL_URLS_PER_DOMAIN = Integer.getInteger("crawler.initialUrlsPerDomain", 250);
private final Logger logger = LoggerFactory.getLogger(getClass());
private final FileStorageService fileStorageService;
@Inject
public CrawlJobExtractorActor(Gson gson,
FileStorageService fileStorageService
) {
super(gson);
this.fileStorageService = fileStorageService;
}
public record CreateFromUrl(String description, String url) implements ActorStep {}
@Override
public ActorStep transition(ActorStep self) throws Exception {
return switch (self) {
case CreateFromUrl(String description, String url) -> {
var storage = fileStorageService.allocateStorage(FileStorageType.CRAWL_SPEC, "crawl-spec", description);
Path urlsTxt = storage.asPath().resolve("urls.txt");
try {
downloadToFile(url, urlsTxt);
}
catch (IOException ex) {
fileStorageService.flagFileForDeletion(storage.id());
yield new Error("Error downloading " + url);
}
final Path path = CrawlSpecFileNames.resolve(storage);
generateCrawlSpec(
path,
DomainSource.fromFile(urlsTxt),
KnownUrlsCountSource.fixed(INITIAL_URLS_PER_DOMAIN),
KnownUrlsListSource.justIndex()
);
yield new End();
}
default -> new Error();
};
}
private void downloadToFile(String url, Path urlsTxt) throws IOException {
try (var os = Files.newOutputStream(urlsTxt, StandardOpenOption.CREATE_NEW);
var is = new URL(url).openStream())
{
is.transferTo(os);
}
}
@Override
public String describe() {
return "Run the crawler job extractor process";
}
}

View File

@ -1,110 +0,0 @@
package nu.marginalia.actor.task;
import com.google.gson.Gson;
import com.google.inject.Inject;
import com.google.inject.Singleton;
import nu.marginalia.actor.ExecutorActor;
import nu.marginalia.actor.ExecutorActorStateMachines;
import nu.marginalia.actor.prototype.RecordActorPrototype;
import nu.marginalia.actor.state.ActorResumeBehavior;
import nu.marginalia.actor.state.ActorStep;
import nu.marginalia.actor.state.Resume;
import nu.marginalia.process.ProcessOutboxes;
import nu.marginalia.process.ProcessService;
import nu.marginalia.storage.FileStorageService;
import nu.marginalia.storage.model.FileStorageId;
import nu.marginalia.storage.model.FileStorageType;
import nu.marginalia.mq.MqMessageState;
import nu.marginalia.mq.outbox.MqOutbox;
import nu.marginalia.mqapi.crawling.CrawlRequest;
import nu.marginalia.svc.DomainListRefreshService;
import java.nio.file.Files;
import java.nio.file.Path;
@Singleton
public class RecrawlActor extends RecordActorPrototype {
private final MqOutbox mqCrawlerOutbox;
private final FileStorageService storageService;
private final DomainListRefreshService refreshService;
private final ActorProcessWatcher processWatcher;
private final ExecutorActorStateMachines executorActorStateMachines;
/** Initial step
* @param storageId - the id of the storage to recrawl
* @param cascadeLoad - whether to automatically start the convert and load actor after the crawl
*/
public record Initial(FileStorageId storageId, boolean cascadeLoad) implements ActorStep {}
/** The action step */
@Resume(behavior = ActorResumeBehavior.RETRY)
public record Crawl(long messageId, FileStorageId fid, boolean cascadeLoad) implements ActorStep {}
@Override
public ActorStep transition(ActorStep self) throws Exception {
return switch (self) {
case Initial (FileStorageId fid, boolean cascadeLoad) -> {
var crawlStorage = storageService.getStorage(fid);
if (crawlStorage == null) yield new Error("Bad storage id");
if (crawlStorage.type() != FileStorageType.CRAWL_DATA) yield new Error("Bad storage type " + crawlStorage.type());
Path crawlLogPath = crawlStorage.asPath().resolve("crawler.log");
if (Files.exists(crawlLogPath)) {
// Save the old crawl log
Path crawlLogBackup = crawlStorage.asPath().resolve("crawler.log-" + System.currentTimeMillis());
Files.move(crawlLogPath, crawlLogBackup);
}
refreshService.synchronizeDomainList();
long id = mqCrawlerOutbox.sendAsync(CrawlRequest.forRecrawl(fid));
yield new Crawl(id, fid, cascadeLoad);
}
case Crawl (long msgId, FileStorageId fid, boolean cascadeLoad) -> {
var rsp = processWatcher.waitResponse(
mqCrawlerOutbox,
ProcessService.ProcessId.CRAWLER,
msgId);
if (rsp.state() != MqMessageState.OK) {
yield new Error("Crawler failed");
}
if (cascadeLoad) {
// Spawn the convert and load actor
executorActorStateMachines.initFrom(ExecutorActor.CONVERT_AND_LOAD,
new ConvertAndLoadActor.Initial(fid));
}
yield new End();
}
default -> new End();
};
}
@Override
public String describe() {
return "Run the crawler with the given crawl spec using previous crawl data for a reference";
}
@Inject
public RecrawlActor(ActorProcessWatcher processWatcher,
ProcessOutboxes processOutboxes,
FileStorageService storageService,
DomainListRefreshService refreshService,
ExecutorActorStateMachines executorActorStateMachines,
Gson gson)
{
super(gson);
this.processWatcher = processWatcher;
this.mqCrawlerOutbox = processOutboxes.getCrawlerOutbox();
this.storageService = storageService;
this.refreshService = refreshService;
this.executorActorStateMachines = executorActorStateMachines;
}
}

View File

@ -4,7 +4,10 @@ import com.google.inject.Inject;
import io.grpc.stub.StreamObserver;
import nu.marginalia.actor.ExecutorActor;
import nu.marginalia.actor.ExecutorActorControlService;
import nu.marginalia.actor.task.*;
import nu.marginalia.actor.task.ConvertActor;
import nu.marginalia.actor.task.ConvertAndLoadActor;
import nu.marginalia.actor.task.CrawlActor;
import nu.marginalia.actor.task.RecrawlSingleDomainActor;
import nu.marginalia.functions.execution.api.*;
import nu.marginalia.service.server.DiscoverableService;
import nu.marginalia.storage.model.FileStorageId;
@ -27,21 +30,7 @@ public class ExecutorCrawlGrpcService
public void triggerCrawl(RpcFileStorageId request, StreamObserver<Empty> responseObserver) {
try {
actorControlService.startFrom(ExecutorActor.CRAWL,
new CrawlActor.Initial(FileStorageId.of(request.getFileStorageId())));
responseObserver.onNext(Empty.getDefaultInstance());
responseObserver.onCompleted();
}
catch (Exception e) {
responseObserver.onError(e);
}
}
@Override
public void triggerRecrawl(RpcFileStorageId request, StreamObserver<Empty> responseObserver) {
try {
actorControlService.startFrom(ExecutorActor.RECRAWL,
new RecrawlActor.Initial(FileStorageId.of(request.getFileStorageId()), false));
new CrawlActor.Initial(FileStorageId.of(request.getFileStorageId()), false));
responseObserver.onNext(Empty.getDefaultInstance());
responseObserver.onCompleted();
@ -113,21 +102,4 @@ public class ExecutorCrawlGrpcService
}
}
@Override
public void createCrawlSpecFromDownload(RpcCrawlSpecFromDownload request, StreamObserver<Empty> responseObserver) {
try {
actorControlService.startFrom(ExecutorActor.CRAWL_JOB_EXTRACTOR,
new CrawlJobExtractorActor.CreateFromUrl(
request.getDescription(),
request.getUrl())
);
responseObserver.onNext(Empty.getDefaultInstance());
responseObserver.onCompleted();
}
catch (Exception e) {
responseObserver.onError(e);
}
}
}

View File

@ -14,8 +14,8 @@ import nu.marginalia.converting.sideload.SideloadSourceFactory;
import nu.marginalia.converting.writer.ConverterBatchWritableIf;
import nu.marginalia.converting.writer.ConverterBatchWriter;
import nu.marginalia.converting.writer.ConverterWriter;
import nu.marginalia.io.crawldata.CrawledDomainReader;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.mq.MessageQueueFactory;
import nu.marginalia.mq.MqMessage;
import nu.marginalia.mq.inbox.MqInboxResponse;

View File

@ -16,7 +16,7 @@ import nu.marginalia.converting.writer.ConverterBatchWritableIf;
import nu.marginalia.converting.writer.ConverterBatchWriter;
import nu.marginalia.geoip.GeoIpDictionary;
import nu.marginalia.geoip.sources.AsnTable;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.crawl.DomainIndexingState;
import nu.marginalia.model.crawldata.CrawledDocument;

View File

@ -5,7 +5,7 @@ import com.google.inject.Guice;
import com.google.inject.Injector;
import nu.marginalia.converting.model.ProcessedDocument;
import nu.marginalia.converting.processor.DomainProcessor;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.crawl.DomainIndexingState;
import nu.marginalia.model.crawl.PubDate;

View File

@ -12,13 +12,13 @@ import nu.marginalia.crawl.fetcher.HttpFetcherImpl;
import nu.marginalia.crawl.fetcher.warc.WarcRecorder;
import nu.marginalia.crawl.retreival.CrawlerRetreiver;
import nu.marginalia.crawl.retreival.DomainProber;
import nu.marginalia.crawl.spec.CrawlSpecProvider;
import nu.marginalia.io.crawldata.format.ParquetSerializableCrawlDataStream;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.crawl.DomainIndexingState;
import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.model.crawldata.CrawledDomain;
import nu.marginalia.model.crawldata.SerializableCrawlData;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import nu.marginalia.parquet.crawldata.CrawledDocumentParquetRecordFileWriter;
import org.junit.jupiter.api.*;
import org.slf4j.Logger;
@ -77,11 +77,7 @@ public class CrawlingThenConvertingIntegrationTest {
@Test
public void testInvalidDomain() throws IOException {
// Attempt to fetch an invalid domain
var specs = CrawlSpecRecord.builder()
.domain("invalid.invalid.invalid")
.crawlDepth(10)
.urls(List.of()) // add specific URLs to crawl here
.build();
var specs = new CrawlSpecProvider.CrawlSpecRecord("invalid.invalid.invalid", 10);
CrawledDomain crawlData = crawl(specs);
@ -97,11 +93,7 @@ public class CrawlingThenConvertingIntegrationTest {
@Test
public void testRedirectingDomain() throws IOException {
// Attempt to fetch an invalid domain
var specs = CrawlSpecRecord.builder()
.domain("memex.marginalia.nu")
.crawlDepth(10)
.urls(List.of()) // add specific URLs to crawl here
.build();
var specs = new CrawlSpecProvider.CrawlSpecRecord("memex.marginalia.nu", 10);
CrawledDomain crawlData = crawl(specs);
@ -120,11 +112,7 @@ public class CrawlingThenConvertingIntegrationTest {
@Test
public void testBlockedDomain() throws IOException {
// Attempt to fetch an invalid domain
var specs = CrawlSpecRecord.builder()
.domain("search.marginalia.nu")
.crawlDepth(10)
.urls(List.of()) // add specific URLs to crawl here
.build();
var specs = new CrawlSpecProvider.CrawlSpecRecord("search.marginalia.nu", 10);
CrawledDomain crawlData = crawl(specs, d->false); // simulate blocking by blacklisting everything
@ -140,11 +128,7 @@ public class CrawlingThenConvertingIntegrationTest {
@Test
public void crawlSunnyDay() throws IOException {
var specs = CrawlSpecRecord.builder()
.domain("www.marginalia.nu")
.crawlDepth(10)
.urls(List.of()) // add specific URLs to crawl here
.build();
var specs = new CrawlSpecProvider.CrawlSpecRecord("www.marginalia.nu", 10);
CrawledDomain domain = crawl(specs);
assertFalse(domain.doc.isEmpty());
@ -177,14 +161,12 @@ public class CrawlingThenConvertingIntegrationTest {
@Test
public void crawlContentTypes() throws IOException {
var specs = CrawlSpecRecord.builder()
.domain("www.marginalia.nu")
.crawlDepth(5)
.urls(List.of(
var specs = new CrawlSpecProvider.CrawlSpecRecord("www.marginalia.nu", 10,
List.of(
"https://www.marginalia.nu/sanic.png",
"https://www.marginalia.nu/invalid"
))
.build();
)
);
CrawledDomain domain = crawl(specs);
assertFalse(domain.doc.isEmpty());
@ -217,13 +199,9 @@ public class CrawlingThenConvertingIntegrationTest {
@Test
public void crawlRobotsTxt() throws IOException {
var specs = CrawlSpecRecord.builder()
.domain("search.marginalia.nu")
.crawlDepth(5)
.urls(List.of(
"https://search.marginalia.nu/search?q=hello+world"
))
.build();
var specs = new CrawlSpecProvider.CrawlSpecRecord("search.marginalia.nu", 5,
List.of("https://search.marginalia.nu/search?q=hello+world")
);
CrawledDomain domain = crawl(specs);
assertFalse(domain.doc.isEmpty());
@ -260,18 +238,18 @@ public class CrawlingThenConvertingIntegrationTest {
return null; // unreachable
}
}
private CrawledDomain crawl(CrawlSpecRecord specs) throws IOException {
private CrawledDomain crawl(CrawlSpecProvider.CrawlSpecRecord specs) throws IOException {
return crawl(specs, domain -> true);
}
private CrawledDomain crawl(CrawlSpecRecord specs, Predicate<EdgeDomain> domainBlacklist) throws IOException {
private CrawledDomain crawl(CrawlSpecProvider.CrawlSpecRecord specs, Predicate<EdgeDomain> domainBlacklist) throws IOException {
List<SerializableCrawlData> data = new ArrayList<>();
try (var recorder = new WarcRecorder(fileName)) {
new CrawlerRetreiver(httpFetcher, new DomainProber(domainBlacklist), specs, recorder).crawlDomain();
}
CrawledDocumentParquetRecordFileWriter.convertWarc(specs.domain,
CrawledDocumentParquetRecordFileWriter.convertWarc(specs.domain(),
new UserAgent("test", "test"),
fileName, fileName2);

View File

@ -17,15 +17,11 @@ import nu.marginalia.crawl.retreival.CrawlDataReference;
import nu.marginalia.crawl.retreival.CrawlerRetreiver;
import nu.marginalia.crawl.retreival.DomainProber;
import nu.marginalia.crawl.spec.CrawlSpecProvider;
import nu.marginalia.crawl.spec.DbCrawlSpecProvider;
import nu.marginalia.crawl.spec.ParquetCrawlSpecProvider;
import nu.marginalia.crawl.warc.WarcArchiverFactory;
import nu.marginalia.crawl.warc.WarcArchiverIf;
import nu.marginalia.crawlspec.CrawlSpecFileNames;
import nu.marginalia.io.crawldata.CrawledDomainReader;
import nu.marginalia.io.crawldata.CrawlerOutputFile;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.io.CrawlerOutputFile;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import nu.marginalia.mq.MessageQueueFactory;
import nu.marginalia.mq.MqMessage;
import nu.marginalia.mq.inbox.MqInboxResponse;
@ -66,7 +62,7 @@ public class CrawlerMain extends ProcessMainClass {
private final MessageQueueFactory messageQueueFactory;
private final DomainProber domainProber;
private final FileStorageService fileStorageService;
private final DbCrawlSpecProvider dbCrawlSpecProvider;
private final CrawlSpecProvider crawlSpecProvider;
private final AnchorTagsSourceFactory anchorTagsSourceFactory;
private final WarcArchiverFactory warcArchiverFactory;
private final Gson gson;
@ -89,7 +85,7 @@ public class CrawlerMain extends ProcessMainClass {
MessageQueueFactory messageQueueFactory, DomainProber domainProber,
FileStorageService fileStorageService,
ProcessConfiguration processConfiguration,
DbCrawlSpecProvider dbCrawlSpecProvider,
CrawlSpecProvider crawlSpecProvider,
AnchorTagsSourceFactory anchorTagsSourceFactory,
WarcArchiverFactory warcArchiverFactory,
Gson gson) {
@ -98,7 +94,7 @@ public class CrawlerMain extends ProcessMainClass {
this.messageQueueFactory = messageQueueFactory;
this.domainProber = domainProber;
this.fileStorageService = fileStorageService;
this.dbCrawlSpecProvider = dbCrawlSpecProvider;
this.crawlSpecProvider = crawlSpecProvider;
this.anchorTagsSourceFactory = anchorTagsSourceFactory;
this.warcArchiverFactory = warcArchiverFactory;
this.gson = gson;
@ -148,7 +144,7 @@ public class CrawlerMain extends ProcessMainClass {
crawler.runForSingleDomain(instructions.targetDomainName, instructions.outputDir);
}
else {
crawler.run(instructions.specProvider, instructions.outputDir);
crawler.run(instructions.outputDir);
}
instructions.ok();
} catch (Exception ex) {
@ -164,12 +160,12 @@ public class CrawlerMain extends ProcessMainClass {
System.exit(0);
}
public void run(CrawlSpecProvider specProvider, Path outputDir) throws Exception {
public void run(Path outputDir) throws Exception {
heartbeat.start();
// First a validation run to ensure the file is all good to parse
totalTasks = specProvider.totalCount();
totalTasks = crawlSpecProvider.totalCount();
if (totalTasks == 0) {
// This is an error state, and we should make noise about it
throw new IllegalStateException("No crawl tasks found, refusing to continue");
@ -178,18 +174,18 @@ public class CrawlerMain extends ProcessMainClass {
try (WorkLog workLog = new WorkLog(outputDir.resolve("crawler.log"));
WarcArchiverIf warcArchiver = warcArchiverFactory.get(outputDir);
AnchorTagsSource anchorTagsSource = anchorTagsSourceFactory.create(specProvider.getDomains())
AnchorTagsSource anchorTagsSource = anchorTagsSourceFactory.create(crawlSpecProvider.getDomains())
) {
// Set the number of tasks done to the number of tasks that are already finished,
// (this happens when the process is restarted after a crash or a shutdown)
tasksDone.set(workLog.countFinishedJobs());
// Process the crawl tasks
try (var specStream = specProvider.stream()) {
try (var specStream = crawlSpecProvider.stream()) {
specStream
.takeWhile((e) -> abortMonitor.isAlive())
.filter(e -> !workLog.isJobFinished(e.domain))
.filter(e -> processingIds.put(e.domain, "") == null)
.filter(e -> !workLog.isJobFinished(e.domain()))
.filter(e -> processingIds.put(e.domain(), "") == null)
.map(e -> new CrawlTask(e, anchorTagsSource, outputDir, warcArchiver, workLog))
.forEach(pool::submitQuietly);
}
@ -226,7 +222,7 @@ public class CrawlerMain extends ProcessMainClass {
WarcArchiverIf warcArchiver = warcArchiverFactory.get(outputDir);
AnchorTagsSource anchorTagsSource = anchorTagsSourceFactory.create(List.of(new EdgeDomain(targetDomainName)))
) {
var spec = new CrawlSpecRecord(targetDomainName, 1000, null);
var spec = new CrawlSpecProvider.CrawlSpecRecord(targetDomainName, 1000, null);
var task = new CrawlTask(spec, anchorTagsSource, outputDir, warcArchiver, workLog);
task.run();
}
@ -240,7 +236,7 @@ public class CrawlerMain extends ProcessMainClass {
class CrawlTask implements SimpleBlockingThreadPool.Task {
private final CrawlSpecRecord specification;
private final CrawlSpecProvider.CrawlSpecRecord specification;
private final String domain;
private final String id;
@ -250,7 +246,7 @@ public class CrawlerMain extends ProcessMainClass {
private final WarcArchiverIf warcArchiver;
private final WorkLog workLog;
CrawlTask(CrawlSpecRecord specification,
CrawlTask(CrawlSpecProvider.CrawlSpecRecord specification,
AnchorTagsSource anchorTagsSource,
Path outputDir,
WarcArchiverIf warcArchiver,
@ -262,7 +258,7 @@ public class CrawlerMain extends ProcessMainClass {
this.warcArchiver = warcArchiver;
this.workLog = workLog;
this.domain = specification.domain;
this.domain = specification.domain();
this.id = Integer.toHexString(domain.hashCode());
}
@ -280,7 +276,7 @@ public class CrawlerMain extends ProcessMainClass {
Files.deleteIfExists(tempFile);
}
var domainLock = domainLocks.getSemaphore(new EdgeDomain(specification.domain));
var domainLock = domainLocks.getSemaphore(new EdgeDomain(specification.domain()));
try (var warcRecorder = new WarcRecorder(newWarcFile); // write to a temp file for now
var retriever = new CrawlerRetreiver(fetcher, domainProber, specification, warcRecorder);
@ -336,7 +332,7 @@ public class CrawlerMain extends ProcessMainClass {
try {
return new CrawlDataReference(CrawledDomainReader.createDataStream(outputDir, domain, id));
} catch (IOException e) {
logger.debug("Failed to read previous crawl data for {}", specification.domain);
logger.debug("Failed to read previous crawl data for {}", specification.domain());
return new CrawlDataReference();
}
}
@ -346,22 +342,19 @@ public class CrawlerMain extends ProcessMainClass {
private static class CrawlRequest {
private final CrawlSpecProvider specProvider;
private final Path outputDir;
private final MqMessage message;
private final MqSingleShotInbox inbox;
private final String targetDomainName;
CrawlRequest(CrawlSpecProvider specProvider,
String targetDomainName,
CrawlRequest(String targetDomainName,
Path outputDir,
MqMessage message,
MqSingleShotInbox inbox)
{
this.message = message;
this.inbox = inbox;
this.specProvider = specProvider;
this.outputDir = outputDir;
this.targetDomainName = targetDomainName;
}
@ -387,26 +380,9 @@ public class CrawlerMain extends ProcessMainClass {
var request = gson.fromJson(msg.payload(), nu.marginalia.mqapi.crawling.CrawlRequest.class);
CrawlSpecProvider specProvider;
if (request.specStorage != null) {
var specData = fileStorageService.getStorage(request.specStorage);
var parquetProvider = new ParquetCrawlSpecProvider(CrawlSpecFileNames.resolve(specData));
// Ensure the parquet domains are loaded into the database to avoid
// rare data-loss scenarios
dbCrawlSpecProvider.ensureParquetDomainsLoaded(parquetProvider);
specProvider = parquetProvider;
}
else {
specProvider = dbCrawlSpecProvider;
}
var crawlData = fileStorageService.getStorage(request.crawlStorage);
return new CrawlRequest(
specProvider,
request.targetDomainName,
crawlData.asPath(),
msg,

View File

@ -2,7 +2,7 @@ package nu.marginalia.crawl.retreival;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.lsh.EasyLSH;
import nu.marginalia.model.crawldata.CrawledDocument;
import org.slf4j.Logger;

View File

@ -11,12 +11,12 @@ import nu.marginalia.crawl.logic.LinkFilterSelector;
import nu.marginalia.crawl.retreival.revisit.CrawlerRevisitor;
import nu.marginalia.crawl.retreival.revisit.DocumentWithReference;
import nu.marginalia.crawl.retreival.sitemap.SitemapFetcher;
import nu.marginalia.crawl.spec.CrawlSpecProvider;
import nu.marginalia.ip_blocklist.UrlBlocklist;
import nu.marginalia.link_parser.LinkParser;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.body.HttpFetchResult;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import org.jsoup.Jsoup;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -26,7 +26,6 @@ import java.net.InetAddress;
import java.net.UnknownHostException;
import java.nio.file.Path;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
@ -55,16 +54,16 @@ public class CrawlerRetreiver implements AutoCloseable {
public CrawlerRetreiver(HttpFetcher fetcher,
DomainProber domainProber,
CrawlSpecRecord specs,
CrawlSpecProvider.CrawlSpecRecord specs,
WarcRecorder warcRecorder)
{
this.warcRecorder = warcRecorder;
this.fetcher = fetcher;
this.domainProber = domainProber;
domain = specs.domain;
domain = specs.domain();
crawlFrontier = new DomainCrawlFrontier(new EdgeDomain(domain), Objects.requireNonNullElse(specs.urls, List.of()), specs.crawlDepth);
crawlFrontier = new DomainCrawlFrontier(new EdgeDomain(domain), specs.urls(), specs.crawlDepth());
crawlerRevisitor = new CrawlerRevisitor(crawlFrontier, this, warcRecorder);
sitemapFetcher = new SitemapFetcher(crawlFrontier, fetcher.createSitemapRetriever());

View File

@ -1,16 +1,143 @@
package nu.marginalia.crawl.spec;
import com.google.inject.Inject;
import com.zaxxer.hikari.HikariDataSource;
import lombok.Builder;
import lombok.SneakyThrows;
import nu.marginalia.ProcessConfiguration;
import nu.marginalia.db.DomainBlacklist;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.stream.Stream;
public interface CrawlSpecProvider {
int totalCount() throws Exception;
Stream<CrawlSpecRecord> stream();
public class CrawlSpecProvider {
private final HikariDataSource dataSource;
private final ProcessConfiguration processConfiguration;
private final DomainBlacklist blacklist;
default List<EdgeDomain> getDomains() {
return stream().map(CrawlSpecRecord::getDomain).map(EdgeDomain::new).toList();
private List<CrawlSpecRecord> domains;
private static final Logger logger = LoggerFactory.getLogger(CrawlSpecProvider.class);
private static final double URL_GROWTH_FACTOR = Double.parseDouble(System.getProperty("crawler.crawlSetGrowthFactor", "1.25"));
private static final int MIN_URLS_PER_DOMAIN = Integer.getInteger("crawler.minUrlsPerDomain", 100);
private static final int MID_URLS_PER_DOMAIN = Integer.getInteger("crawler.minUrlsPerDomain", 2_000);
private static final int MAX_URLS_PER_DOMAIN = Integer.getInteger("crawler.maxUrlsPerDomain", 10_000);
@Inject
public CrawlSpecProvider(HikariDataSource dataSource,
ProcessConfiguration processConfiguration,
DomainBlacklist blacklist
) {
this.dataSource = dataSource;
this.processConfiguration = processConfiguration;
this.blacklist = blacklist;
}
// Load the domains into memory to ensure the crawler is resilient to database blips
private List<CrawlSpecRecord> loadData() throws Exception {
var domains = new ArrayList<CrawlSpecRecord>();
logger.info("Loading domains to be crawled");
blacklist.waitUntilLoaded();
List<Integer> domainIds = new ArrayList<>(10_000);
try (var conn = dataSource.getConnection();
var assignFreeDomains = conn.prepareStatement("UPDATE EC_DOMAIN SET NODE_AFFINITY=? WHERE NODE_AFFINITY=0");
var query = conn.prepareStatement("""
SELECT DOMAIN_NAME, COALESCE(KNOWN_URLS, 0), EC_DOMAIN.ID
FROM EC_DOMAIN
LEFT JOIN DOMAIN_METADATA ON EC_DOMAIN.ID=DOMAIN_METADATA.ID
WHERE NODE_AFFINITY=? OR NODE_AFFINITY=0
""")
)
{
// Assign any domains with node_affinity=0 to this node. We must do this now, before we start crawling
// to avoid race conditions with other crawl runs. We don't want multiple crawlers to crawl the same domain.
assignFreeDomains.setInt(1, processConfiguration.node());
assignFreeDomains.executeUpdate();
// Fetch the domains to be crawled
query.setInt(1, processConfiguration.node());
query.setFetchSize(10_000);
var rs = query.executeQuery();
while (rs.next()) {
// Skip blacklisted domains
int id = rs.getInt(3);
if (blacklist.isBlacklisted(id))
continue;
domainIds.add(id);
int urls = rs.getInt(2);
double growthFactor;
if (urls < MID_URLS_PER_DOMAIN) {
growthFactor = Math.max(2.5, URL_GROWTH_FACTOR);
}
else {
growthFactor = URL_GROWTH_FACTOR;
}
int urlsToFetch = Math.clamp((int) (growthFactor * rs.getInt(2)), MIN_URLS_PER_DOMAIN, MAX_URLS_PER_DOMAIN);
var record = new CrawlSpecRecord(
rs.getString(1),
urlsToFetch,
List.of()
);
domains.add(record);
}
}
logger.info("Loaded {} domains", domains.size());
// Shuffle the domains to ensure we get a good mix of domains in each crawl,
// so that e.g. the big domains don't get all crawled at once, or we end up
// crawling the same server in parallel from different subdomains...
Collections.shuffle(domains);
return domains;
}
public List<EdgeDomain> getDomains() {
return stream().map(CrawlSpecRecord::domain).map(EdgeDomain::new).toList();
}
public int totalCount() throws Exception {
if (domains == null) {
domains = loadData();
}
return domains.size();
}
@SneakyThrows
public Stream<CrawlSpecRecord> stream() {
if (domains == null) {
domains = loadData();
}
return domains.stream();
}
@Builder
public record CrawlSpecRecord(@NotNull String domain,
int crawlDepth,
@NotNull List<String> urls) {
public CrawlSpecRecord(String domain, int crawlDepth) {
this(domain, crawlDepth, List.of());
}
}
}

View File

@ -1,164 +0,0 @@
package nu.marginalia.crawl.spec;
import com.google.inject.Inject;
import com.zaxxer.hikari.HikariDataSource;
import lombok.SneakyThrows;
import nu.marginalia.ProcessConfiguration;
import nu.marginalia.db.DomainBlacklist;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.stream.Stream;
public class DbCrawlSpecProvider implements CrawlSpecProvider {
private final HikariDataSource dataSource;
private final ProcessConfiguration processConfiguration;
private final DomainBlacklist blacklist;
private List<CrawlSpecRecord> domains;
private static final Logger logger = LoggerFactory.getLogger(DbCrawlSpecProvider.class);
private static final double URL_GROWTH_FACTOR = Double.parseDouble(System.getProperty("crawler.crawlSetGrowthFactor", "1.25"));
private static final int MIN_URLS_PER_DOMAIN = Integer.getInteger("crawler.minUrlsPerDomain", 100);
private static final int MID_URLS_PER_DOMAIN = Integer.getInteger("crawler.minUrlsPerDomain", 2_000);
private static final int MAX_URLS_PER_DOMAIN = Integer.getInteger("crawler.maxUrlsPerDomain", 10_000);
@Inject
public DbCrawlSpecProvider(HikariDataSource dataSource,
ProcessConfiguration processConfiguration,
DomainBlacklist blacklist
) {
this.dataSource = dataSource;
this.processConfiguration = processConfiguration;
this.blacklist = blacklist;
}
// Load the domains into memory to ensure the crawler is resilient to database blips
private List<CrawlSpecRecord> loadData() throws Exception {
var domains = new ArrayList<CrawlSpecRecord>();
logger.info("Loading domains to be crawled");
blacklist.waitUntilLoaded();
List<Integer> domainIds = new ArrayList<>(10_000);
try (var conn = dataSource.getConnection();
var assignFreeDomains = conn.prepareStatement("UPDATE EC_DOMAIN SET NODE_AFFINITY=? WHERE NODE_AFFINITY=0");
var query = conn.prepareStatement("""
SELECT DOMAIN_NAME, COALESCE(KNOWN_URLS, 0), EC_DOMAIN.ID
FROM EC_DOMAIN
LEFT JOIN DOMAIN_METADATA ON EC_DOMAIN.ID=DOMAIN_METADATA.ID
WHERE NODE_AFFINITY=? OR NODE_AFFINITY=0
""")
)
{
// Assign any domains with node_affinity=0 to this node. We must do this now, before we start crawling
// to avoid race conditions with other crawl runs. We don't want multiple crawlers to crawl the same domain.
assignFreeDomains.setInt(1, processConfiguration.node());
assignFreeDomains.executeUpdate();
// Fetch the domains to be crawled
query.setInt(1, processConfiguration.node());
query.setFetchSize(10_000);
var rs = query.executeQuery();
while (rs.next()) {
// Skip blacklisted domains
int id = rs.getInt(3);
if (blacklist.isBlacklisted(id))
continue;
domainIds.add(id);
int urls = rs.getInt(2);
double growthFactor;
if (urls < MID_URLS_PER_DOMAIN) {
growthFactor = Math.max(2.5, URL_GROWTH_FACTOR);
}
else {
growthFactor = URL_GROWTH_FACTOR;
}
int urlsToFetch = Math.clamp((int) (growthFactor * rs.getInt(2)), MIN_URLS_PER_DOMAIN, MAX_URLS_PER_DOMAIN);
var record = new CrawlSpecRecord(
rs.getString(1),
urlsToFetch,
List.of()
);
domains.add(record);
}
}
logger.info("Loaded {} domains", domains.size());
// Shuffle the domains to ensure we get a good mix of domains in each crawl,
// so that e.g. the big domains don't get all crawled at once, or we end up
// crawling the same server in parallel from different subdomains...
Collections.shuffle(domains);
return domains;
}
/** Ensure that the domains in the parquet spec provider are loaded into
* the database. This mitigates the risk of certain footguns, such
* re-crawling before performing the 'Load' operation, which would
* otherwise result in the crawler not being able to find the domain
* in the database through the DbCrawlSpecProvider, and thus not
* being able to crawl it.
* */
public void ensureParquetDomainsLoaded(ParquetCrawlSpecProvider parquetCrawlSpecProvider) throws Exception {
// This is a bit of an unhealthy mix of concerns, but it's for the Greater Good (TM)
try (var conn = dataSource.getConnection();
var stmt = conn.prepareStatement("""
INSERT IGNORE INTO EC_DOMAIN(DOMAIN_NAME, DOMAIN_TOP, NODE_AFFINITY)
VALUES (?, ?, ?)
"""))
{
parquetCrawlSpecProvider.stream().forEach(record -> {
try {
var domain = new EdgeDomain(record.getDomain());
stmt.setString(1, record.domain);
stmt.setString(2, domain.topDomain);
stmt.setInt(3, processConfiguration.node());
stmt.addBatch();
} catch (Exception e) {
throw new RuntimeException(e);
}
});
stmt.executeBatch();
}
}
@Override
public int totalCount() throws Exception {
if (domains == null) {
domains = loadData();
}
return domains.size();
}
@SneakyThrows
@Override
public Stream<CrawlSpecRecord> stream() {
if (domains == null) {
domains = loadData();
}
return domains.stream();
}
}

View File

@ -1,37 +0,0 @@
package nu.marginalia.crawl.spec;
import lombok.SneakyThrows;
import nu.marginalia.io.crawlspec.CrawlSpecRecordParquetFileReader;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import java.io.IOException;
import java.nio.file.Path;
import java.util.List;
import java.util.stream.Stream;
public class ParquetCrawlSpecProvider implements CrawlSpecProvider {
private final List<Path> files;
public ParquetCrawlSpecProvider(List<Path> files) {
this.files = files;
}
@Override
public int totalCount() throws IOException {
int total = 0;
for (var specs : files) {
total += CrawlSpecRecordParquetFileReader.count(specs);
}
return total;
}
@Override
public Stream<CrawlSpecRecord> stream() {
return files.stream().flatMap(this::streamQuietly);
}
@SneakyThrows
private Stream<CrawlSpecRecord> streamQuietly(Path file) {
return CrawlSpecRecordParquetFileReader.stream(file);
}
}

View File

@ -1,34 +0,0 @@
package nu.marginalia.crawlspec;
import nu.marginalia.storage.model.FileStorage;
import nu.marginalia.storage.model.FileStorageType;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
public class CrawlSpecFileNames {
public static Path resolve(Path base) {
return base.resolve("crawl-spec.parquet");
}
public static Path resolve(FileStorage storage) {
if (storage.type() != FileStorageType.CRAWL_SPEC)
throw new IllegalArgumentException("Provided file storage is of unexpected type " +
storage.type() + ", expected CRAWL_SPEC");
return resolve(storage.asPath());
}
public static List<Path> resolve(List<FileStorage> storageList) {
List<Path> ret = new ArrayList<>();
for (var storage : storageList) {
if (storage.type() != FileStorageType.CRAWL_SPEC)
throw new IllegalArgumentException("Provided file storage is of unexpected type " +
storage.type() + ", expected CRAWL_SPEC");
ret.add(resolve(storage));
}
return ret;
}
}

View File

@ -1,71 +0,0 @@
package nu.marginalia.crawlspec;
import nu.marginalia.io.crawlspec.CrawlSpecRecordParquetFileWriter;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.sql.SQLException;
import java.util.List;
public class CrawlSpecGenerator {
public static void generateCrawlSpec(Path output,
DomainSource domains,
KnownUrlsCountSource counts,
KnownUrlsListSource listSource)
throws IOException, SQLException
{
try (var writer = new CrawlSpecRecordParquetFileWriter(output)) {
for (String domain : domains.getDomainNames()) {
domain = domain.toLowerCase();
writer.write(CrawlSpecRecord
.builder()
.crawlDepth(counts.getKnownUrlCount(domain))
.urls(listSource.getKnownUrls(domain))
.domain(domain)
.build());
}
}
}
public interface DomainSource {
List<String> getDomainNames() throws IOException, SQLException;
static DomainSource fromFile(Path file) {
return () -> {
var lines = Files.readAllLines(file);
lines.replaceAll(s ->
s.split("#", 2)[0]
.trim()
.toLowerCase()
);
lines.removeIf(String::isBlank);
return lines;
};
}
}
public interface KnownUrlsCountSource {
int getKnownUrlCount(String domainName) throws SQLException;
static KnownUrlsCountSource fixed(int value) {
return domainName -> value;
}
}
public interface KnownUrlsListSource {
List<String> getKnownUrls(String domainName) throws SQLException;
static KnownUrlsListSource justIndex() {
return domainName -> List.of(
"http://" + domainName + "/",
"https://" + domainName + "/"
);
}
}
}

View File

@ -1,4 +1,4 @@
package nu.marginalia.io.crawldata;
package nu.marginalia.io;
import nu.marginalia.io.crawldata.format.ParquetSerializableCrawlDataStream;
import org.slf4j.Logger;

View File

@ -1,4 +1,4 @@
package nu.marginalia.io.crawldata;
package nu.marginalia.io;
import org.apache.logging.log4j.util.Strings;

View File

@ -1,4 +1,4 @@
package nu.marginalia.io.crawldata;
package nu.marginalia.io;
import nu.marginalia.model.crawldata.SerializableCrawlData;
import org.jetbrains.annotations.Nullable;

View File

@ -4,7 +4,7 @@ import lombok.SneakyThrows;
import nu.marginalia.contenttype.ContentType;
import nu.marginalia.contenttype.DocumentBodyToString;
import nu.marginalia.hash.MurmurHash3_128;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.crawldata.*;
import nu.marginalia.parquet.crawldata.CrawledDocumentParquetRecord;

View File

@ -1,26 +0,0 @@
package nu.marginalia.io.crawlspec;
import blue.strategic.parquet.HydratorSupplier;
import blue.strategic.parquet.ParquetReader;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import org.jetbrains.annotations.NotNull;
import java.io.IOException;
import java.nio.file.Path;
import java.util.stream.Stream;
public class CrawlSpecRecordParquetFileReader {
@NotNull
public static Stream<CrawlSpecRecord> stream(Path path) throws IOException {
return ParquetReader.streamContent(path.toFile(),
HydratorSupplier.constantly(CrawlSpecRecord.newHydrator()));
}
public static int count(Path path) throws IOException {
try (var stream = stream(path)) {
// FIXME This can be done in a more performant way by using another hydrator that only reads a single field
return (int) stream.count();
}
}
}

View File

@ -1,24 +0,0 @@
package nu.marginalia.io.crawlspec;
import blue.strategic.parquet.ParquetWriter;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import java.io.IOException;
import java.nio.file.Path;
public class CrawlSpecRecordParquetFileWriter implements AutoCloseable {
private final ParquetWriter<CrawlSpecRecord> writer;
public CrawlSpecRecordParquetFileWriter(Path file) throws IOException {
writer = ParquetWriter.writeFile(CrawlSpecRecord.schema,
file.toFile(), CrawlSpecRecord.newDehydrator());
}
public void write(CrawlSpecRecord domainData) throws IOException {
writer.write(domainData);
}
public void close() throws IOException {
writer.close();
}
}

View File

@ -1,90 +0,0 @@
package nu.marginalia.model.crawlspec;
import blue.strategic.parquet.Dehydrator;
import blue.strategic.parquet.Hydrator;
import blue.strategic.parquet.ValueWriter;
import lombok.*;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.Types;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import java.util.ArrayList;
import java.util.List;
import static org.apache.parquet.schema.LogicalTypeAnnotation.stringType;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*;
@Getter
@Setter
@NoArgsConstructor
@AllArgsConstructor
@EqualsAndHashCode
@Builder
@ToString
public class CrawlSpecRecord {
@NotNull
public String domain;
/** Limit for how many documents will be crawled */
public int crawlDepth;
/** List of known URLs */
@Nullable
public List<String> urls;
public static Hydrator<CrawlSpecRecord, CrawlSpecRecord> newHydrator() {
return new CrawlSpecRecordHydrator();
}
public static Dehydrator<CrawlSpecRecord> newDehydrator() {
return CrawlSpecRecord::dehydrate;
}
public static MessageType schema = new MessageType(
CrawlSpecRecord.class.getSimpleName(),
Types.required(BINARY).as(stringType()).named("domain"),
Types.required(INT32).named("crawlDepth"),
Types.repeated(BINARY).as(stringType()).named("urls")
);
public void dehydrate(ValueWriter valueWriter) {
valueWriter.write("domain", domain);
valueWriter.write("crawlDepth", crawlDepth);
valueWriter.writeList("urls", urls);
}
public CrawlSpecRecord add(String heading, Object value) {
switch (heading) {
case "domain" -> domain = (String) value;
case "crawlDepth" -> crawlDepth = (Integer) value;
case "urls" -> {
if (urls == null)
urls = new ArrayList<>();
urls.add((String) value);
}
}
return this;
}
}
class CrawlSpecRecordHydrator implements Hydrator<CrawlSpecRecord, CrawlSpecRecord> {
@Override
public CrawlSpecRecord start() {
return new CrawlSpecRecord();
}
@Override
public CrawlSpecRecord add(CrawlSpecRecord target, String heading, Object value) {
return target.add(heading, value);
}
@Override
public CrawlSpecRecord finish(CrawlSpecRecord target) {
return target;
}
}

View File

@ -9,13 +9,13 @@ import nu.marginalia.crawl.fetcher.SitemapRetriever;
import nu.marginalia.crawl.fetcher.warc.WarcRecorder;
import nu.marginalia.crawl.retreival.CrawlerRetreiver;
import nu.marginalia.crawl.retreival.DomainProber;
import nu.marginalia.crawl.spec.CrawlSpecProvider;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.body.HttpFetchResult;
import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.model.crawldata.CrawlerDocumentStatus;
import nu.marginalia.model.crawldata.SerializableCrawlData;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import nu.marginalia.test.CommonTestData;
import okhttp3.Headers;
import org.junit.jupiter.api.AfterEach;
@ -68,7 +68,7 @@ public class CrawlerMockFetcherTest {
}
void crawl(CrawlSpecRecord spec) throws IOException {
void crawl(CrawlSpecProvider.CrawlSpecRecord spec) throws IOException {
try (var recorder = new WarcRecorder()) {
new CrawlerRetreiver(fetcherMock, new DomainProber(d -> true), spec, recorder)
.crawlDomain();
@ -83,7 +83,7 @@ public class CrawlerMockFetcherTest {
registerUrlClasspathData(new EdgeUrl("https://startrek.website/c/startrek"), "mock-crawl-data/lemmy/c_startrek.html");
registerUrlClasspathData(new EdgeUrl("https://startrek.website/post/108995"), "mock-crawl-data/lemmy/108995.html");
crawl(new CrawlSpecRecord("startrek.website", 10, new ArrayList<>()));
crawl(new CrawlSpecProvider.CrawlSpecRecord("startrek.website", 10, new ArrayList<>()));
}
@Test
@ -92,7 +92,7 @@ public class CrawlerMockFetcherTest {
registerUrlClasspathData(new EdgeUrl("https://en.wikipedia.org/"), "mock-crawl-data/mediawiki/index.html");
crawl(new CrawlSpecRecord("en.wikipedia.org", 10, new ArrayList<>()));
crawl(new CrawlSpecProvider.CrawlSpecRecord("en.wikipedia.org", 10, new ArrayList<>()));
}
@Test
@ -103,7 +103,7 @@ public class CrawlerMockFetcherTest {
registerUrlClasspathData(new EdgeUrl("https://community.tt-rss.org/t/telegram-channel-to-idle-on/3501"), "mock-crawl-data/discourse/telegram.html");
registerUrlClasspathData(new EdgeUrl("https://community.tt-rss.org/t/combined-mode-but-grid/4489"), "mock-crawl-data/discourse/grid.html");
crawl(new CrawlSpecRecord("community.tt-rss.org", 10, new ArrayList<>()));
crawl(new CrawlSpecProvider.CrawlSpecRecord("community.tt-rss.org", 10, new ArrayList<>()));
}
class MockFetcher implements HttpFetcher {

View File

@ -8,14 +8,14 @@ import nu.marginalia.crawl.fetcher.HttpFetcher;
import nu.marginalia.crawl.fetcher.HttpFetcherImpl;
import nu.marginalia.crawl.fetcher.warc.WarcRecorder;
import nu.marginalia.crawl.retreival.*;
import nu.marginalia.io.crawldata.CrawledDomainReader;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.crawl.spec.CrawlSpecProvider;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.model.crawldata.CrawledDomain;
import nu.marginalia.model.crawldata.SerializableCrawlData;
import nu.marginalia.model.crawlspec.CrawlSpecRecord;
import nu.marginalia.parquet.crawldata.CrawledDocumentParquetRecordFileWriter;
import org.jetbrains.annotations.NotNull;
import org.junit.jupiter.api.*;
@ -76,7 +76,7 @@ class CrawlerRetreiverTest {
@Test
public void testWarcOutput() throws IOException {
var specs = CrawlSpecRecord
var specs = CrawlSpecProvider.CrawlSpecRecord
.builder()
.crawlDepth(5)
.domain("www.marginalia.nu")
@ -119,7 +119,7 @@ class CrawlerRetreiverTest {
@SneakyThrows
@Test
public void testResync() throws IOException {
var specs = CrawlSpecRecord
var specs = CrawlSpecProvider.CrawlSpecRecord
.builder()
.crawlDepth(5)
.domain("www.marginalia.nu")
@ -168,7 +168,7 @@ class CrawlerRetreiverTest {
@Test
public void testWithKnownDomains() throws IOException {
var specs = CrawlSpecRecord
var specs = CrawlSpecProvider.CrawlSpecRecord
.builder()
.crawlDepth(5)
.domain("www.marginalia.nu")
@ -212,7 +212,7 @@ class CrawlerRetreiverTest {
@Test
public void testRedirect() throws IOException, URISyntaxException {
var specs = CrawlSpecRecord
var specs = CrawlSpecProvider.CrawlSpecRecord
.builder()
.crawlDepth(3)
.domain("www.marginalia.nu")
@ -270,7 +270,7 @@ class CrawlerRetreiverTest {
@Test
public void testEmptySet() throws IOException {
var specs = CrawlSpecRecord
var specs = CrawlSpecProvider.CrawlSpecRecord
.builder()
.crawlDepth(5)
.domain("www.marginalia.nu")
@ -318,7 +318,7 @@ class CrawlerRetreiverTest {
@Test
public void testRecrawl() throws IOException {
var specs = CrawlSpecRecord
var specs = CrawlSpecProvider.CrawlSpecRecord
.builder()
.crawlDepth(12)
.domain("www.marginalia.nu")
@ -378,7 +378,7 @@ class CrawlerRetreiverTest {
@Test
public void testRecrawlWithResync() throws IOException {
var specs = CrawlSpecRecord
var specs = CrawlSpecProvider.CrawlSpecRecord
.builder()
.crawlDepth(12)
.domain("www.marginalia.nu")
@ -466,7 +466,7 @@ class CrawlerRetreiverTest {
}
}
private void doCrawlWithReferenceStream(CrawlSpecRecord specs, SerializableCrawlDataStream stream) {
private void doCrawlWithReferenceStream(CrawlSpecProvider.CrawlSpecRecord specs, SerializableCrawlDataStream stream) {
try (var recorder = new WarcRecorder(tempFileWarc2)) {
new CrawlerRetreiver(httpFetcher, new DomainProber(d -> true), specs, recorder).crawlDomain(new DomainLinks(),
new CrawlDataReference(stream));
@ -477,7 +477,7 @@ class CrawlerRetreiverTest {
}
@NotNull
private DomainCrawlFrontier doCrawl(Path tempFileWarc1, CrawlSpecRecord specs) {
private DomainCrawlFrontier doCrawl(Path tempFileWarc1, CrawlSpecProvider.CrawlSpecRecord specs) {
try (var recorder = new WarcRecorder(tempFileWarc1)) {
var crawler = new CrawlerRetreiver(httpFetcher, new DomainProber(d -> true), specs, recorder);
crawler.crawlDomain();

View File

@ -3,16 +3,9 @@ package nu.marginalia.mqapi.crawling;
import lombok.AllArgsConstructor;
import nu.marginalia.storage.model.FileStorageId;
import java.util.List;
/** A request to start a crawl */
@AllArgsConstructor
public class CrawlRequest {
/** (optional) Crawl spec(s) for sourcing domains to crawl. If not set,
* the EC_DOMAIN table will be consulted and domains with the corresponding
* node affinity will be used.
*/
public List<FileStorageId> specStorage;
/** (optional) Name of a single domain to be re-crawled */
public String targetDomainName;
@ -22,16 +15,12 @@ public class CrawlRequest {
*/
public FileStorageId crawlStorage;
public static CrawlRequest forSpec(FileStorageId specStorage, FileStorageId crawlStorage) {
return new CrawlRequest(List.of(specStorage), null, crawlStorage);
}
public static CrawlRequest forSingleDomain(String targetDomainName, FileStorageId crawlStorage) {
return new CrawlRequest(null, targetDomainName, crawlStorage);
return new CrawlRequest(targetDomainName, crawlStorage);
}
public static CrawlRequest forRecrawl(FileStorageId crawlStorage) {
return new CrawlRequest(null, null, crawlStorage);
public static CrawlRequest forFullCrawl(FileStorageId crawlStorage) {
return new CrawlRequest(null, crawlStorage);
}
}

View File

@ -10,9 +10,6 @@ import java.time.LocalDate;
import java.time.format.DateTimeFormatter;
public record FileStorageWithActions(FileStorage storage) {
public boolean isCrawlable() {
return storage.type() == FileStorageType.CRAWL_SPEC;
}
public boolean isStatusNew() {
return storage.state() == FileStorageState.NEW;
}

View File

@ -83,12 +83,9 @@ public class ControlNodeActionsService {
Spark.post("/nodes/:node/actions/download-sample-data", this::downloadSampleData,
redirectControl.renderRedirectAcknowledgement("Downloading", "..")
);
Spark.post("/nodes/:id/actions/new-crawl", this::triggerNewCrawl,
Spark.post("/nodes/:id/actions/new-crawl", this::triggerCrawl,
redirectControl.renderRedirectAcknowledgement("Crawling", "..")
);
Spark.post("/nodes/:id/actions/recrawl", this::triggerAutoRecrawl,
redirectControl.renderRedirectAcknowledgement("Recrawling", "..")
);
Spark.post("/nodes/:id/actions/recrawl-single-domain", this::triggerSingleDomainRecrawl,
redirectControl.renderRedirectAcknowledgement("Recrawling", "..")
);
@ -101,9 +98,6 @@ public class ControlNodeActionsService {
Spark.post("/nodes/:id/actions/restore-backup", this::triggerRestoreBackup,
redirectControl.renderRedirectAcknowledgement("Restoring", "..")
);
Spark.post("/nodes/:id/actions/new-crawl-specs", this::createNewSpecsAction,
redirectControl.renderRedirectAcknowledgement("Creating", "../actions?view=new-crawl")
);
Spark.post("/nodes/:id/actions/export-db-data", this::exportDbData,
redirectControl.renderRedirectAcknowledgement("Exporting", "..")
);
@ -205,14 +199,14 @@ public class ControlNodeActionsService {
return "";
}
private Object triggerAutoRecrawl(Request request, Response response) throws SQLException {
private Object triggerCrawl(Request request, Response response) throws SQLException {
int nodeId = Integer.parseInt(request.params("id"));
var toCrawl = parseSourceFileStorageId(request.queryParams("source"));
changeActiveStorage(nodeId, FileStorageType.CRAWL_DATA, toCrawl);
crawlClient.triggerRecrawl(
crawlClient.triggerCrawl(
nodeId,
toCrawl
);
@ -235,18 +229,6 @@ public class ControlNodeActionsService {
return "";
}
private Object triggerNewCrawl(Request request, Response response) throws SQLException {
int nodeId = Integer.parseInt(request.params("id"));
var toCrawl = parseSourceFileStorageId(request.queryParams("source"));
changeActiveStorage(nodeId, FileStorageType.CRAWL_SPEC, toCrawl);
crawlClient.triggerCrawl(nodeId, toCrawl);
return "";
}
private Object triggerProcess(Request request, Response response) throws SQLException {
int nodeId = Integer.parseInt(request.params("id"));
boolean isAutoload = "on".equalsIgnoreCase(request.queryParams("autoload"));
@ -308,21 +290,6 @@ public class ControlNodeActionsService {
}
}
private Object createNewSpecsAction(Request request, Response response) {
final String description = request.queryParams("description");
final String url = request.queryParams("url");
int nodeId = Integer.parseInt(request.params("id"));
if (url == null || url.isBlank()) {
throw new ControlValidationError("No url specified", "A url must be specified", "..");
}
crawlClient.createCrawlSpecFromDownload(nodeId, description, url);
return "";
}
private Object exportDbData(Request req, Response rsp) {
exportClient.exportData(Integer.parseInt(req.params("id")));

View File

@ -175,9 +175,7 @@ public class ControlNodeService {
"allCrawlData",
fileStorageService.getEachFileStorage(nodeId, FileStorageType.CRAWL_DATA),
"allProcessedData",
fileStorageService.getEachFileStorage(nodeId, FileStorageType.PROCESSED_DATA),
"allCrawlSpecs",
fileStorageService.getEachFileStorage(nodeId, FileStorageType.CRAWL_SPEC)
fileStorageService.getEachFileStorage(nodeId, FileStorageType.PROCESSED_DATA)
);
}
@ -201,7 +199,6 @@ public class ControlNodeService {
case "backup" -> FileStorageType.BACKUP;
case "crawl" -> FileStorageType.CRAWL_DATA;
case "processed" -> FileStorageType.PROCESSED_DATA;
case "specs" -> FileStorageType.CRAWL_SPEC;
case "exports" -> FileStorageType.EXPORT;
default -> throw new IllegalArgumentException(view);
};

View File

@ -8,7 +8,10 @@
{{> control/partials/nav}}
<div class="container">
<h1 class="my-3">Domains</h1>
<div class="my-3 p-3 border bg-light">
This is a listing of all domains known by the search engine. Additional domains can be added via the
<a href="/domain/new">Add Domains</a> utility.
</div>
<table class="table">
<form method="get">
<tr>

View File

@ -1,28 +0,0 @@
<h1 class="my-3">Create New Specification</h1>
<div class="my-3 p-3 border bg-light">
<p>A crawling specification is a list of domain names to crawl.</p>
<p>
If you are just looking to test the software, feel free to use <a href="https://downloads.marginalia.nu/domain-list-test.txt">this
short list of marginalia-related websites</a>, that are safe to crawl repeatedly without causing any problems.
</p>
<p><span class="text-danger">Warning</span> Ensure <a href="?view=download-sample-data">downloaded sample data</a> has not been loaded onto this instance
before performing this action, otherwise those domains will also be crawled while re-crawling in the future!</p>
</div>
<form method="post" action="actions/new-crawl-specs">
<div class="form">
<label class="form-label" for="description">Description</label>
<input class="form-control" type="text" name="description" id="description" maxlength="255">
<div class="form-text" id="basic-addon4">This is how you'll be able to find the
specification later so give it a good and descriptive name</div>
<div id="spec-url-options" class="py-3">
<label class="form-label" for="url">URL to list of domain names</label>
<input class="form-control" type="text" name="url" id="url" />
</div>
<button type="submit" class="btn btn-primary">Create</button>
</div>
</form>

View File

@ -1,32 +1,21 @@
<h1 class="my-3">New Crawl</h1>
{{#unless allCrawlSpecs}}
No crawling specifications have been created. You must create a crawl spec before you can crawl.
<a href="?view=new-crawl-specs">Create a crawl spec</a>.
{{/unless}}
{{#if allCrawlSpecs}}
<div class="my-3 p-3 border bg-light">
This will perform a new crawl on node {{node.id}} based on the crawl spec you select below.
Additional specifications can be created <a href="?view=new-crawl-specs">with this form</a>.
</div>
<div class="my-3 p-3 border">
<p><em class="text-danger">IMPORTANT!</em> Be sure you've read and understood the
<a href="https://github.com/MarginaliaSearch/MarginaliaSearch/blob/master/doc/crawling.md">crawling documentation</a>
before you begin a crawl. You will be accessing real servers from your connection, and you may end up on IP greylists
that temporarily block your access to those servers for up to a few weeks; on rare occasions permanently. The crawler
is generally polite and well-behaved, but it is still a crawler, and it will be accessing a lot of pages.
</p>
<p>
You run the risk of getting in trouble with your ISP, the law, your partner, your landlord, and/or your parents;
depending on terms of service, jurisdiction, marital status, living conditions and age.
<p>
This will crawl on node {{node.id}}.
</p><p>
If existing crawl data is provided, the crawler may perform a smart crawl that uses the existing data
as a starting point. It will use the existing crawl data to perform conditional GETs on the domains in the
crawl spec. If the document has changed, it will be re-crawled. If it has not changed, it will be skipped,
and the previous data will be retained. This is both faster and easier on the target server.
</p>
This action is performed at your own risk.
</p>
<p><span class="text-danger">Warning</span> Ensure <a href="?view=download-sample-data">downloaded sample data</a>
has not been loaded onto this instance before performing this action, otherwise those domains will also be crawled!</p>
<p>Use the <a href="/domain">Domains</a> view to inspect and manage known domains. </p>
</div>
<form method="post" action="actions/new-crawl">
<p>Select the crawl data to use:</p>
<table class="table">
<tr>
<th>Use</th>
@ -34,8 +23,7 @@
<th>Description</th>
<th>Details</th>
</tr>
{{#each allCrawlSpecs}}
{{#each allCrawlData}}
<tr>
<td><input {{#if active}}checked{{/if}} {{#if new}}disabled{{/if}} {{#if delete}}disabled{{/if}} class="form-check-input" type="radio" name="source" id="{{id}}" value="{{id}}"></td>
<td><label for="{{id}}" class="form-check-label" >{{path}}</label></td>
@ -46,11 +34,19 @@
<td><a href="/nodes/{{node}}/storage/details?fid={{id}}">[Details]</a></td>
</tr>
{{/each}}
<tr>
<td>
<input class="form-check-input" type="radio" name="source" id="-1" value="-1">
</td>
<td colspan="3">
<label for="-1" class="form-check-label"><em>Crawl without referencing existing crawl data, using the <a href="/domain">domains in the database</a>.</em></label>
</td>
</tr>
</table>
<button
class="btn btn-primary me-md-2"
onclick="return confirm('Confirm crawling on node {{node.id}}');"
onclick="return confirm('Confirm recrawl of the selected data on node {{node.id}}');"
type="submit">
Trigger New Crawl</button>
Trigger Recrawl</button>
</form>
{{/if}}

View File

@ -1,53 +0,0 @@
<h1 class="my-3">Recrawl</h1>
{{#unless allCrawlData}}
<p>
No <a href="storage/crawl">crawl data</a> currently exists on this node. Recrawling is not possible.
First, create a crawl spec and do an initial crawl. Then, you can recrawl.
</p>
{{/unless}}
{{#if allCrawlData}}
<div class="my-3 p-3 border bg-light">
<p>
This will perform a re-crawl on node {{node.id}}.
</p><p>
A re-crawl is a smart crawl that uses the existing crawl data
as a starting point. It will use the existing crawl data to perform conditional GETs on the domains in the
crawl spec. If the document has changed, it will be re-crawled. If it has not changed, it will be skipped,
and the previous data will be retained. This is both faster and easier on the target server.
</p>
<p><span class="text-danger">Warning</span> Ensure <a href="?view=download-sample-data">downloaded sample data</a>
has not been loaded onto this instance before performing this action, otherwise those domains will also be crawled!</p>
</div>
<form method="post" action="actions/recrawl">
<p>Select the crawl data to use:</p>
<table class="table">
<tr>
<th>Use</th>
<th>Path</th>
<th>Description</th>
<th>Details</th>
</tr>
{{#each allCrawlData}}
<tr>
<td><input {{#if active}}checked{{/if}} {{#if new}}disabled{{/if}} {{#if delete}}disabled{{/if}} class="form-check-input" type="radio" name="source" id="{{id}}" value="{{id}}"></td>
<td><label for="{{id}}" class="form-check-label" >{{path}}</label></td>
<td>{{description}}
<span class="text-danger">{{#if new}}[CREATING]{{/if}}</span>
<span class="text-danger">{{#if delete}}[DELETING]{{/if}}</span>
</td>
<td><a href="/nodes/{{node}}/storage/details?fid={{id}}">[Details]</a></td>
</tr>
{{/each}}
</table>
<button
class="btn btn-primary me-md-2"
onclick="return confirm('Confirm recrawl of the selected data on node {{node.id}}');"
type="submit">
Trigger Recrawl</button>
</form>
{{/if}}

View File

@ -10,8 +10,6 @@
{{> control/node/partial-node-nav }}
{{#if view.new-crawl}} {{> control/node/actions/partial-new-crawl }} {{/if}}
{{#if view.new-crawl-specs}} {{> control/node/actions/partial-new-crawl-specs }} {{/if}}
{{#if view.re-crawl}} {{> control/node/actions/partial-recrawl }} {{/if}}
{{#if view.process}} {{> control/node/actions/partial-process }} {{/if}}
{{#if view.load}} {{> control/node/actions/partial-load }} {{/if}}
{{#if view.repartition}} {{> control/node/actions/partial-repartition }} {{/if}}

View File

@ -14,7 +14,6 @@
<a class="nav-link dropdown-toggle {{#if tab.actions}}active{{/if}}" data-bs-toggle="dropdown" href="#" role="button" aria-expanded="false">Actions</a>
<ul class="dropdown-menu">
<li><a class="dropdown-item" href="/nodes/{{node.id}}/actions?view=new-crawl">New Crawl</a></li>
<li><a class="dropdown-item" href="/nodes/{{node.id}}/actions?view=re-crawl">Re-crawl</a></li>
<li><hr class="dropdown-divider"></li>
<li><a class="dropdown-item" href="/nodes/{{node.id}}/actions?view=process">Process Crawl Data</a></li>
<li><a class="dropdown-item" href="/nodes/{{node.id}}/actions?view=load">Load Processed Data</a></li>

View File

@ -11,9 +11,6 @@
<li class="nav-item">
<a href="backup" {{#unless view.backup}}class="nav-link"{{/unless}} {{#if view.backup}}aria-current="page" class="active nav-link"{{/if}}>Backups</a>
</li>
<li class="nav-item">
<a href="specs" {{#unless view.specs}}class="nav-link"{{/unless}} {{#if view.specs}}aria-current="page" class="active nav-link"{{/if}}>Specs</a>
</li>
<li class="nav-item">
<a href="exports" {{#unless view.exports}}class="nav-link"{{/unless}} {{#if view.exports}}aria-current="page" class="active nav-link"{{/if}}>Exports</a>
</li>

View File

@ -1,6 +1,6 @@
package nu.marginalia.tools;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.SerializableCrawlDataStream;
import java.io.IOException;
import java.util.HashSet;

View File

@ -3,7 +3,7 @@ package nu.marginalia.tools;
import com.google.inject.Guice;
import com.google.inject.Injector;
import nu.marginalia.converting.ConverterModule;
import nu.marginalia.io.crawldata.CrawledDomainReader;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.process.log.WorkLog;
import nu.marginalia.service.module.DatabaseModule;
import nu.marginalia.tools.experiments.*;

View File

@ -1,6 +1,6 @@
package nu.marginalia.tools;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.model.crawldata.CrawledDomain;

View File

@ -4,7 +4,7 @@ import com.google.inject.Inject;
import gnu.trove.set.hash.TLongHashSet;
import lombok.SneakyThrows;
import nu.marginalia.hash.MurmurHash3_128;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.link_parser.LinkParser;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.crawldata.CrawledDocument;

View File

@ -3,7 +3,7 @@ package nu.marginalia.tools.experiments;
import com.google.inject.Inject;
import nu.marginalia.converting.model.ProcessedDocument;
import nu.marginalia.converting.processor.DomainProcessor;
import nu.marginalia.io.crawldata.SerializableCrawlDataStream;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.tools.Experiment;

View File

@ -24,7 +24,7 @@ import nu.marginalia.index.index.StatefulIndex;
import nu.marginalia.index.journal.IndexJournal;
import nu.marginalia.index.model.SearchParameters;
import nu.marginalia.index.searchset.SearchSetAny;
import nu.marginalia.io.crawldata.CrawledDomainReader;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.linkdb.docs.DocumentDbReader;
import nu.marginalia.linkdb.docs.DocumentDbWriter;
import nu.marginalia.loading.LoaderIndexJournalWriter;