mirror of
https://github.com/MarginaliaSearch/MarginaliaSearch.git
synced 2025-02-23 21:18:58 +00:00
(converter, WIP) Refactor converter to not have to load everything into RAM.
This commit is contained in:
parent
7470c170b1
commit
a56953c798
@ -31,11 +31,9 @@ public class CrawledDomainReader {
|
|||||||
public CrawledDomainReader() {
|
public CrawledDomainReader() {
|
||||||
}
|
}
|
||||||
|
|
||||||
public Iterator<SerializableCrawlData> createIterator(Path basePath, CrawlingSpecification spec) throws IOException {
|
public Iterator<SerializableCrawlData> createIterator(Path fullPath) throws IOException {
|
||||||
|
|
||||||
final var path = CrawlerOutputFile.getOutputFile(basePath, spec.id, spec.domain);
|
BufferedReader br = new BufferedReader(new InputStreamReader(new ZstdInputStream(new FileInputStream(fullPath.toFile()))));
|
||||||
|
|
||||||
BufferedReader br = new BufferedReader(new InputStreamReader(new ZstdInputStream(new FileInputStream(path.toFile()))));
|
|
||||||
|
|
||||||
return new Iterator<>() {
|
return new Iterator<>() {
|
||||||
SerializableCrawlData next;
|
SerializableCrawlData next;
|
||||||
@ -71,6 +69,11 @@ public class CrawledDomainReader {
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Iterator<SerializableCrawlData> createIterator(Path basePath, CrawlingSpecification spec) throws IOException {
|
||||||
|
|
||||||
|
return createIterator(CrawlerOutputFile.getOutputFile(basePath, spec.id, spec.domain));
|
||||||
|
}
|
||||||
|
|
||||||
public CrawledDomain read(Path path) throws IOException {
|
public CrawledDomain read(Path path) throws IOException {
|
||||||
DomainDataAssembler domainData = new DomainDataAssembler();
|
DomainDataAssembler domainData = new DomainDataAssembler();
|
||||||
|
@ -21,7 +21,7 @@ public class CrawledDocument implements SerializableCrawlData {
|
|||||||
public String crawlerStatusDesc;
|
public String crawlerStatusDesc;
|
||||||
|
|
||||||
public String headers;
|
public String headers;
|
||||||
public BigString documentBody;
|
public String documentBody;
|
||||||
public String documentBodyHash;
|
public String documentBodyHash;
|
||||||
|
|
||||||
public String canonicalUrl;
|
public String canonicalUrl;
|
||||||
@ -35,9 +35,4 @@ public class CrawledDocument implements SerializableCrawlData {
|
|||||||
return SERIAL_IDENTIFIER;
|
return SERIAL_IDENTIFIER;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Remove all large data from this object to save memory */
|
|
||||||
public void dispose() {
|
|
||||||
documentBody = null;
|
|
||||||
headers = null;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -5,22 +5,18 @@ import lombok.NoArgsConstructor;
|
|||||||
import lombok.ToString;
|
import lombok.ToString;
|
||||||
import nu.marginalia.crawling.io.CrawledDomainReader;
|
import nu.marginalia.crawling.io.CrawledDomainReader;
|
||||||
import nu.marginalia.crawling.model.CrawledDomain;
|
import nu.marginalia.crawling.model.CrawledDomain;
|
||||||
|
import nu.marginalia.crawling.model.SerializableCrawlData;
|
||||||
import nu.marginalia.crawling.model.spec.CrawlerSpecificationLoader;
|
import nu.marginalia.crawling.model.spec.CrawlerSpecificationLoader;
|
||||||
import nu.marginalia.crawling.model.spec.CrawlingSpecification;
|
import nu.marginalia.crawling.model.spec.CrawlingSpecification;
|
||||||
import nu.marginalia.process.log.WorkLog;
|
import nu.marginalia.process.log.WorkLog;
|
||||||
import nu.marginalia.process.log.WorkLogEntry;
|
|
||||||
import org.jetbrains.annotations.NotNull;
|
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.io.FileNotFoundException;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.file.Files;
|
import java.nio.file.Files;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.function.Consumer;
|
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
import java.util.stream.Stream;
|
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
@AllArgsConstructor @NoArgsConstructor @ToString
|
@AllArgsConstructor @NoArgsConstructor @ToString
|
||||||
@ -122,4 +118,30 @@ public class CrawlPlan {
|
|||||||
return reader.readOptionally(path);
|
return reader.readOptionally(path);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public Iterable<Iterator<SerializableCrawlData>> crawlDataIterable(Predicate<String> idPredicate) {
|
||||||
|
final CrawledDomainReader reader = new CrawledDomainReader();
|
||||||
|
|
||||||
|
return WorkLog.iterableMap(crawl.getLogFile(),
|
||||||
|
entry -> {
|
||||||
|
if (!idPredicate.test(entry.id())) {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
var path = getCrawledFilePath(entry.path());
|
||||||
|
|
||||||
|
if (!Files.exists(path)) {
|
||||||
|
logger.warn("File not found: {}", path);
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
return Optional.of(reader.createIterator(path));
|
||||||
|
}
|
||||||
|
catch (IOException ex) {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -4,6 +4,7 @@ import com.google.gson.Gson;
|
|||||||
import com.google.inject.Guice;
|
import com.google.inject.Guice;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
|
import nu.marginalia.crawling.model.SerializableCrawlData;
|
||||||
import nu.marginalia.db.storage.FileStorageService;
|
import nu.marginalia.db.storage.FileStorageService;
|
||||||
import nu.marginalia.mq.MessageQueueFactory;
|
import nu.marginalia.mq.MessageQueueFactory;
|
||||||
import nu.marginalia.mq.MqMessage;
|
import nu.marginalia.mq.MqMessage;
|
||||||
@ -23,12 +24,12 @@ import org.slf4j.LoggerFactory;
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.sql.SQLException;
|
import java.sql.SQLException;
|
||||||
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.function.Predicate;
|
|
||||||
|
|
||||||
import static nu.marginalia.mqapi.ProcessInboxNames.CONVERTER_INBOX;
|
import static nu.marginalia.mqapi.ProcessInboxNames.CONVERTER_INBOX;
|
||||||
|
|
||||||
@ -101,20 +102,14 @@ public class ConverterMain {
|
|||||||
int totalDomains = plan.countCrawledDomains();
|
int totalDomains = plan.countCrawledDomains();
|
||||||
AtomicInteger processedDomains = new AtomicInteger(0);
|
AtomicInteger processedDomains = new AtomicInteger(0);
|
||||||
|
|
||||||
var pipe = new ParallelPipe<CrawledDomain, ProcessingInstructions>("Converter", 16, 4, 2) {
|
var pipe = new ParallelPipe<Iterator<SerializableCrawlData>, ProcessingInstructions>("Converter", 16, 4, 2) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected ProcessingInstructions onProcess(CrawledDomain domainData) {
|
protected ProcessingInstructions onProcess(Iterator<SerializableCrawlData> dataStream) {
|
||||||
Thread.currentThread().setName("Converter:Processor["+domainData.domain+"] - " + domainData.size());
|
var processed = processor.process(dataStream);
|
||||||
try {
|
var compiled = compiler.compile(processed);
|
||||||
var processed = processor.process(domainData);
|
|
||||||
var compiled = compiler.compile(processed);
|
|
||||||
|
|
||||||
return new ProcessingInstructions(domainData.id, compiled);
|
return new ProcessingInstructions(processed.id, compiled);
|
||||||
}
|
|
||||||
finally {
|
|
||||||
Thread.currentThread().setName("Converter:Processor[IDLE]");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -140,7 +135,7 @@ public class ConverterMain {
|
|||||||
processedDomains.set(processLog.countFinishedJobs());
|
processedDomains.set(processLog.countFinishedJobs());
|
||||||
heartbeat.setProgress(processedDomains.get() / (double) totalDomains);
|
heartbeat.setProgress(processedDomains.get() / (double) totalDomains);
|
||||||
|
|
||||||
for (var domain : plan.domainsIterable(id -> !processLog.isJobFinished(id)))
|
for (var domain : plan.crawlDataIterable(id -> !processLog.isJobFinished(id)))
|
||||||
{
|
{
|
||||||
pipe.accept(domain);
|
pipe.accept(domain);
|
||||||
}
|
}
|
||||||
|
@ -3,7 +3,6 @@ package nu.marginalia.converting.compiler;
|
|||||||
import nu.marginalia.converting.instruction.Instruction;
|
import nu.marginalia.converting.instruction.Instruction;
|
||||||
import nu.marginalia.converting.instruction.instructions.LoadKeywords;
|
import nu.marginalia.converting.instruction.instructions.LoadKeywords;
|
||||||
import nu.marginalia.converting.instruction.instructions.LoadProcessedDocument;
|
import nu.marginalia.converting.instruction.instructions.LoadProcessedDocument;
|
||||||
import nu.marginalia.converting.instruction.instructions.LoadProcessedDocumentWithError;
|
|
||||||
import nu.marginalia.converting.model.ProcessedDocument;
|
import nu.marginalia.converting.model.ProcessedDocument;
|
||||||
import nu.marginalia.model.crawl.HtmlFeature;
|
import nu.marginalia.model.crawl.HtmlFeature;
|
||||||
|
|
||||||
|
@ -6,6 +6,8 @@ import nu.marginalia.converting.instruction.instructions.LoadUrl;
|
|||||||
import nu.marginalia.converting.model.ProcessedDocument;
|
import nu.marginalia.converting.model.ProcessedDocument;
|
||||||
import nu.marginalia.model.EdgeDomain;
|
import nu.marginalia.model.EdgeDomain;
|
||||||
import nu.marginalia.model.EdgeUrl;
|
import nu.marginalia.model.EdgeUrl;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
@ -15,30 +17,39 @@ import java.util.Set;
|
|||||||
public class UrlsCompiler {
|
public class UrlsCompiler {
|
||||||
|
|
||||||
private static final int MAX_INTERNAL_LINKS = 25;
|
private static final int MAX_INTERNAL_LINKS = 25;
|
||||||
|
private final Logger logger = LoggerFactory.getLogger(getClass());
|
||||||
|
|
||||||
public void compile(List<Instruction> ret, List<ProcessedDocument> documents) {
|
public void compile(List<Instruction> ret, List<ProcessedDocument> documents) {
|
||||||
Set<EdgeUrl> seenUrls = new HashSet<>(documents.size()*4);
|
Set<EdgeUrl> seenUrls = new HashSet<>(documents.size()*4);
|
||||||
Set<EdgeDomain> seenDomains = new HashSet<>(documents.size());
|
Set<EdgeDomain> seenDomains = new HashSet<>(documents.size());
|
||||||
|
|
||||||
for (var doc : documents) {
|
for (var doc : documents) {
|
||||||
|
if (doc.url == null) {
|
||||||
|
logger.warn("Discovered document with null URL");
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
seenUrls.add(doc.url);
|
seenUrls.add(doc.url);
|
||||||
|
|
||||||
if (doc.details != null) {
|
if (doc.details == null) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
for (var url : doc.details.linksExternal) {
|
// Add *some* external links; to avoid loading too many and gunking up the database with nonsense,
|
||||||
if (seenDomains.add(url.domain)) {
|
// only permit this once per external domain per crawled domain
|
||||||
seenUrls.add(url);
|
for (var url : doc.details.linksExternal) {
|
||||||
}
|
if (seenDomains.add(url.domain)) {
|
||||||
|
seenUrls.add(url);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
if (doc.isOk()) {
|
if (doc.isOk()) {
|
||||||
// Don't load more than a few from linksInternal, grows too big for no reason
|
// Don't load more than a few from linksInternal, grows too big for no reason
|
||||||
var linksToAdd = new ArrayList<>(doc.details.linksInternal);
|
var linksToAdd = new ArrayList<>(doc.details.linksInternal);
|
||||||
if (linksToAdd.size() > MAX_INTERNAL_LINKS) {
|
if (linksToAdd.size() > MAX_INTERNAL_LINKS) {
|
||||||
linksToAdd.subList(MAX_INTERNAL_LINKS, linksToAdd.size()).clear();
|
linksToAdd.subList(MAX_INTERNAL_LINKS, linksToAdd.size()).clear();
|
||||||
}
|
|
||||||
seenUrls.addAll(linksToAdd);
|
|
||||||
}
|
}
|
||||||
|
seenUrls.addAll(linksToAdd);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -6,7 +6,6 @@ import nu.marginalia.model.crawl.DomainIndexingState;
|
|||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.OptionalDouble;
|
|
||||||
|
|
||||||
@ToString
|
@ToString
|
||||||
public class ProcessedDomain {
|
public class ProcessedDomain {
|
||||||
@ -16,17 +15,7 @@ public class ProcessedDomain {
|
|||||||
public DomainIndexingState state;
|
public DomainIndexingState state;
|
||||||
public EdgeDomain redirect;
|
public EdgeDomain redirect;
|
||||||
public String ip;
|
public String ip;
|
||||||
|
public String id;
|
||||||
public OptionalDouble averageQuality() {
|
|
||||||
if (documents == null) {
|
|
||||||
return OptionalDouble.empty();
|
|
||||||
}
|
|
||||||
return documents.stream()
|
|
||||||
.map(ProcessedDocument::quality)
|
|
||||||
.filter(OptionalDouble::isPresent)
|
|
||||||
.mapToDouble(OptionalDouble::getAsDouble)
|
|
||||||
.average();
|
|
||||||
}
|
|
||||||
|
|
||||||
public int size() {
|
public int size() {
|
||||||
return Optional.ofNullable(documents).map(List::size).orElse(1);
|
return Optional.ofNullable(documents).map(List::size).orElse(1);
|
||||||
|
@ -2,7 +2,6 @@ package nu.marginalia.converting.processor;
|
|||||||
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import nu.marginalia.crawling.model.CrawledDocument;
|
import nu.marginalia.crawling.model.CrawledDocument;
|
||||||
import nu.marginalia.crawling.model.CrawledDomain;
|
|
||||||
import nu.marginalia.crawling.model.CrawlerDocumentStatus;
|
import nu.marginalia.crawling.model.CrawlerDocumentStatus;
|
||||||
import nu.marginalia.model.crawl.UrlIndexingState;
|
import nu.marginalia.model.crawl.UrlIndexingState;
|
||||||
import nu.marginalia.converting.model.DisqualifiedException;
|
import nu.marginalia.converting.model.DisqualifiedException;
|
||||||
@ -38,11 +37,14 @@ public class DocumentProcessor {
|
|||||||
processorPlugins.add(plainTextDocumentProcessorPlugin);
|
processorPlugins.add(plainTextDocumentProcessorPlugin);
|
||||||
}
|
}
|
||||||
|
|
||||||
public ProcessedDocument process(CrawledDocument crawledDocument, CrawledDomain crawledDomain) {
|
public ProcessedDocument process(CrawledDocument crawledDocument) {
|
||||||
ProcessedDocument ret = new ProcessedDocument();
|
ProcessedDocument ret = new ProcessedDocument();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
processDocument(crawledDocument, crawledDomain, ret);
|
// We must always provide the URL, even if we don't process the document
|
||||||
|
ret.url = getDocumentUrl(crawledDocument);
|
||||||
|
|
||||||
|
processDocument(crawledDocument, ret);
|
||||||
}
|
}
|
||||||
catch (DisqualifiedException ex) {
|
catch (DisqualifiedException ex) {
|
||||||
ret.state = UrlIndexingState.DISQUALIFIED;
|
ret.state = UrlIndexingState.DISQUALIFIED;
|
||||||
@ -53,13 +55,12 @@ public class DocumentProcessor {
|
|||||||
ret.state = UrlIndexingState.DISQUALIFIED;
|
ret.state = UrlIndexingState.DISQUALIFIED;
|
||||||
ret.stateReason = DisqualifiedException.DisqualificationReason.PROCESSING_EXCEPTION.toString();
|
ret.stateReason = DisqualifiedException.DisqualificationReason.PROCESSING_EXCEPTION.toString();
|
||||||
logger.info("Failed to convert " + crawledDocument.url, ex);
|
logger.info("Failed to convert " + crawledDocument.url, ex);
|
||||||
ex.printStackTrace();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void processDocument(CrawledDocument crawledDocument, CrawledDomain crawledDomain, ProcessedDocument ret) throws URISyntaxException, DisqualifiedException {
|
private void processDocument(CrawledDocument crawledDocument, ProcessedDocument ret) throws URISyntaxException, DisqualifiedException {
|
||||||
|
|
||||||
var crawlerStatus = CrawlerDocumentStatus.valueOf(crawledDocument.crawlerStatus);
|
var crawlerStatus = CrawlerDocumentStatus.valueOf(crawledDocument.crawlerStatus);
|
||||||
if (crawlerStatus != CrawlerDocumentStatus.OK) {
|
if (crawlerStatus != CrawlerDocumentStatus.OK) {
|
||||||
@ -74,15 +75,11 @@ public class DocumentProcessor {
|
|||||||
throw new DisqualifiedException(DisqualifiedException.DisqualificationReason.CONTENT_TYPE);
|
throw new DisqualifiedException(DisqualifiedException.DisqualificationReason.CONTENT_TYPE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
ret.url = getDocumentUrl(crawledDocument);
|
|
||||||
ret.state = crawlerStatusToUrlState(crawledDocument.crawlerStatus, crawledDocument.httpStatus);
|
ret.state = crawlerStatusToUrlState(crawledDocument.crawlerStatus, crawledDocument.httpStatus);
|
||||||
|
|
||||||
final var plugin = findPlugin(crawledDocument);
|
final var plugin = findPlugin(crawledDocument);
|
||||||
|
|
||||||
AbstractDocumentProcessorPlugin.DetailsWithWords detailsWithWords = plugin.createDetails(crawledDomain, crawledDocument);
|
AbstractDocumentProcessorPlugin.DetailsWithWords detailsWithWords = plugin.createDetails(crawledDocument);
|
||||||
|
|
||||||
crawledDocument.dispose();
|
|
||||||
|
|
||||||
ret.details = detailsWithWords.details();
|
ret.details = detailsWithWords.details();
|
||||||
ret.words = detailsWithWords.words();
|
ret.words = detailsWithWords.words();
|
||||||
|
@ -1,18 +1,18 @@
|
|||||||
package nu.marginalia.converting.processor;
|
package nu.marginalia.converting.processor;
|
||||||
|
|
||||||
import com.google.common.base.Strings;
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
|
import nu.marginalia.converting.model.ProcessedDocument;
|
||||||
import nu.marginalia.converting.processor.logic.links.LinkGraph;
|
import nu.marginalia.converting.processor.logic.links.LinkGraph;
|
||||||
import nu.marginalia.crawling.model.CrawledDocument;
|
import nu.marginalia.crawling.model.*;
|
||||||
import nu.marginalia.crawling.model.CrawledDomain;
|
|
||||||
import nu.marginalia.crawling.model.CrawlerDocumentStatus;
|
|
||||||
import nu.marginalia.crawling.model.CrawlerDomainStatus;
|
|
||||||
import nu.marginalia.model.crawl.DomainIndexingState;
|
import nu.marginalia.model.crawl.DomainIndexingState;
|
||||||
import nu.marginalia.converting.model.ProcessedDomain;
|
import nu.marginalia.converting.model.ProcessedDomain;
|
||||||
import nu.marginalia.model.EdgeDomain;
|
import nu.marginalia.model.EdgeDomain;
|
||||||
import nu.marginalia.model.EdgeUrl;
|
import nu.marginalia.model.EdgeUrl;
|
||||||
import nu.marginalia.converting.processor.logic.links.TopKeywords;
|
import nu.marginalia.converting.processor.logic.links.TopKeywords;
|
||||||
import nu.marginalia.converting.processor.logic.LshDocumentDeduplicator;
|
import nu.marginalia.converting.processor.logic.LshDocumentDeduplicator;
|
||||||
|
import nu.marginalia.model.crawl.HtmlFeature;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
import java.util.*;
|
import java.util.*;
|
||||||
|
|
||||||
@ -21,6 +21,8 @@ public class DomainProcessor {
|
|||||||
private final SiteWords siteWords;
|
private final SiteWords siteWords;
|
||||||
private final LshDocumentDeduplicator documentDeduplicator;
|
private final LshDocumentDeduplicator documentDeduplicator;
|
||||||
|
|
||||||
|
private final Logger logger = LoggerFactory.getLogger(getClass());
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public DomainProcessor(DocumentProcessor documentProcessor,
|
public DomainProcessor(DocumentProcessor documentProcessor,
|
||||||
SiteWords siteWords,
|
SiteWords siteWords,
|
||||||
@ -30,44 +32,85 @@ public class DomainProcessor {
|
|||||||
this.documentDeduplicator = documentDeduplicator;
|
this.documentDeduplicator = documentDeduplicator;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ProcessedDomain process(CrawledDomain crawledDomain) {
|
public ProcessedDomain process(Iterator<SerializableCrawlData> dataStream) {
|
||||||
var ret = new ProcessedDomain();
|
var ret = new ProcessedDomain();
|
||||||
|
List<ProcessedDocument> docs = new ArrayList<>();
|
||||||
|
|
||||||
|
boolean cookies = false;
|
||||||
|
String ip = "";
|
||||||
|
while (dataStream.hasNext()) {
|
||||||
|
var data = dataStream.next();
|
||||||
|
|
||||||
ret.domain = new EdgeDomain(crawledDomain.domain);
|
if (data instanceof CrawledDomain crawledDomain) {
|
||||||
ret.ip = crawledDomain.ip;
|
ret.domain = new EdgeDomain(crawledDomain.domain);
|
||||||
|
ret.ip = crawledDomain.ip;
|
||||||
|
ret.id = crawledDomain.id;
|
||||||
|
|
||||||
if (crawledDomain.redirectDomain != null) {
|
cookies = Objects.requireNonNullElse(crawledDomain.cookies, Collections.emptyList()).size() > 0;
|
||||||
ret.redirect = new EdgeDomain(crawledDomain.redirectDomain);
|
ip = crawledDomain.ip;
|
||||||
}
|
|
||||||
|
|
||||||
if (crawledDomain.doc != null) {
|
if (crawledDomain.redirectDomain != null) {
|
||||||
ret.documents = new ArrayList<>(crawledDomain.doc.size());
|
ret.redirect = new EdgeDomain(crawledDomain.redirectDomain);
|
||||||
|
|
||||||
fixBadCanonicalTags(crawledDomain.doc);
|
|
||||||
|
|
||||||
for (var doc : crawledDomain.doc) {
|
|
||||||
var processedDoc = documentProcessor.process(doc, crawledDomain);
|
|
||||||
|
|
||||||
if (processedDoc.url != null) {
|
|
||||||
ret.documents.add(processedDoc);
|
|
||||||
}
|
}
|
||||||
|
ret.documents = docs;
|
||||||
|
ret.state = getState(crawledDomain.crawlerStatus);
|
||||||
}
|
}
|
||||||
|
else if (data instanceof CrawledDocument doc) {
|
||||||
|
try {
|
||||||
|
if (doc.url == null)
|
||||||
|
continue;
|
||||||
|
fixBadCanonicalTag(doc);
|
||||||
|
|
||||||
documentDeduplicator.deduplicate(ret.documents);
|
docs.add(documentProcessor.process(doc));
|
||||||
|
}
|
||||||
calculateStatistics(ret);
|
catch (Exception ex) {
|
||||||
}
|
logger.warn("Failed to process " + doc.url, ex);
|
||||||
else {
|
}
|
||||||
ret.documents = Collections.emptyList();
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
ret.state = getState(crawledDomain.crawlerStatus);
|
// Add late keywords and features from domain-level information
|
||||||
|
|
||||||
|
List<String> terms = new ArrayList<>();
|
||||||
|
terms.add("ip:"+ip);
|
||||||
|
if (cookies)
|
||||||
|
terms.add(HtmlFeature.COOKIES.getKeyword());
|
||||||
|
|
||||||
|
for (var document : ret.documents) {
|
||||||
|
if (document.details == null)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
if (cookies)
|
||||||
|
document.details.features.add(HtmlFeature.COOKIES);
|
||||||
|
|
||||||
|
document.words.addAllSyntheticTerms(terms);
|
||||||
|
}
|
||||||
|
|
||||||
|
documentDeduplicator.deduplicate(ret.documents);
|
||||||
|
calculateStatistics(ret);
|
||||||
|
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void fixBadCanonicalTag(CrawledDocument doc) {
|
||||||
|
// Some sites have a canonical tag that points to a different domain,
|
||||||
|
// but our loader can not support this, so we point these back to the
|
||||||
|
// original url.
|
||||||
|
|
||||||
|
var canonicalOpt = EdgeUrl.parse(doc.canonicalUrl);
|
||||||
|
if (canonicalOpt.isEmpty()) return;
|
||||||
|
|
||||||
|
var urlOpt = EdgeUrl.parse(doc.url);
|
||||||
|
if (urlOpt.isEmpty()) return;
|
||||||
|
|
||||||
|
var urlActual = urlOpt.get();
|
||||||
|
var canonicalActual = canonicalOpt.get();
|
||||||
|
|
||||||
|
if (!Objects.equals(urlActual.domain, canonicalActual.domain)) {
|
||||||
|
doc.canonicalUrl = doc.url;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void calculateStatistics(ProcessedDomain ret) {
|
private void calculateStatistics(ProcessedDomain ret) {
|
||||||
LinkGraph linkGraph = new LinkGraph();
|
LinkGraph linkGraph = new LinkGraph();
|
||||||
TopKeywords topKeywords = new TopKeywords();
|
TopKeywords topKeywords = new TopKeywords();
|
||||||
@ -91,61 +134,6 @@ public class DomainProcessor {
|
|||||||
siteWords.flagAdjacentWords(topKeywords, invertedLinkGraph, ret);
|
siteWords.flagAdjacentWords(topKeywords, invertedLinkGraph, ret);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private void fixBadCanonicalTags(List<CrawledDocument> docs) {
|
|
||||||
Map<String, Set<String>> seenCanonicals = new HashMap<>();
|
|
||||||
Set<String> seenUrls = new HashSet<>();
|
|
||||||
|
|
||||||
// Sometimes sites set a blanket canonical link to their root page
|
|
||||||
// this removes such links from consideration
|
|
||||||
|
|
||||||
for (var document : docs) {
|
|
||||||
if (!Strings.isNullOrEmpty(document.canonicalUrl)
|
|
||||||
&& !Objects.equals(document.canonicalUrl, document.url)) {
|
|
||||||
seenCanonicals.computeIfAbsent(document.canonicalUrl, url -> new HashSet<>()).add(document.documentBodyHash);
|
|
||||||
}
|
|
||||||
seenUrls.add(document.url);
|
|
||||||
}
|
|
||||||
|
|
||||||
for (var document : docs) {
|
|
||||||
if (!Strings.isNullOrEmpty(document.canonicalUrl)
|
|
||||||
&& !Objects.equals(document.canonicalUrl, document.url)
|
|
||||||
&& seenCanonicals.getOrDefault(document.canonicalUrl, Collections.emptySet()).size() > 1) {
|
|
||||||
|
|
||||||
if (seenUrls.add(document.canonicalUrl)) {
|
|
||||||
document.canonicalUrl = document.url;
|
|
||||||
}
|
|
||||||
else {
|
|
||||||
document.crawlerStatus = CrawlerDocumentStatus.BAD_CANONICAL.name();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
for (var document : docs) {
|
|
||||||
if (!Strings.isNullOrEmpty(document.canonicalUrl)
|
|
||||||
&& !Objects.equals(document.canonicalUrl, document.url)
|
|
||||||
&& seenCanonicals.getOrDefault(document.canonicalUrl, Collections.emptySet()).size() > 1) {
|
|
||||||
document.canonicalUrl = document.url;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Ignore canonical URL if it points to a different domain
|
|
||||||
// ... this confuses the hell out of the loader
|
|
||||||
for (var document : docs) {
|
|
||||||
if (Strings.isNullOrEmpty(document.canonicalUrl))
|
|
||||||
continue;
|
|
||||||
|
|
||||||
Optional<EdgeUrl> cUrl = EdgeUrl.parse(document.canonicalUrl);
|
|
||||||
Optional<EdgeUrl> dUrl = EdgeUrl.parse(document.url);
|
|
||||||
|
|
||||||
if (cUrl.isPresent() && dUrl.isPresent()
|
|
||||||
&& !Objects.equals(cUrl.get().domain, dUrl.get().domain))
|
|
||||||
{
|
|
||||||
document.canonicalUrl = document.url;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private DomainIndexingState getState(String crawlerStatus) {
|
private DomainIndexingState getState(String crawlerStatus) {
|
||||||
return switch (CrawlerDomainStatus.valueOf(crawlerStatus)) {
|
return switch (CrawlerDomainStatus.valueOf(crawlerStatus)) {
|
||||||
case OK -> DomainIndexingState.ACTIVE;
|
case OK -> DomainIndexingState.ACTIVE;
|
||||||
|
@ -65,7 +65,7 @@ public class FeatureExtractor {
|
|||||||
this.googleAnwersSpamDetector = googleAnwersSpamDetector;
|
this.googleAnwersSpamDetector = googleAnwersSpamDetector;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Set<HtmlFeature> getFeatures(CrawledDomain domain, Document doc, DocumentLanguageData dld) {
|
public Set<HtmlFeature> getFeatures(Document doc, DocumentLanguageData dld) {
|
||||||
final Set<HtmlFeature> features = new HashSet<>();
|
final Set<HtmlFeature> features = new HashSet<>();
|
||||||
|
|
||||||
final Elements scriptTags = doc.getElementsByTag("script");
|
final Elements scriptTags = doc.getElementsByTag("script");
|
||||||
@ -279,9 +279,6 @@ public class FeatureExtractor {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!domain.cookies.isEmpty())
|
|
||||||
features.add(HtmlFeature.COOKIES);
|
|
||||||
|
|
||||||
if (recipeDetector.testP(dld) > 0.5)
|
if (recipeDetector.testP(dld) > 0.5)
|
||||||
features.add(HtmlFeature.CATEGORY_FOOD);
|
features.add(HtmlFeature.CATEGORY_FOOD);
|
||||||
// these should be mutually exclusive
|
// these should be mutually exclusive
|
||||||
|
@ -19,7 +19,7 @@ import java.util.*;
|
|||||||
public abstract class AbstractDocumentProcessorPlugin {
|
public abstract class AbstractDocumentProcessorPlugin {
|
||||||
protected LanguageFilter languageFilter = new LanguageFilter();
|
protected LanguageFilter languageFilter = new LanguageFilter();
|
||||||
|
|
||||||
public abstract DetailsWithWords createDetails(CrawledDomain crawledDomain, CrawledDocument crawledDocument) throws DisqualifiedException, URISyntaxException;
|
public abstract DetailsWithWords createDetails(CrawledDocument crawledDocument) throws DisqualifiedException, URISyntaxException;
|
||||||
public abstract boolean isApplicable(CrawledDocument doc);
|
public abstract boolean isApplicable(CrawledDocument doc);
|
||||||
|
|
||||||
protected void checkDocumentLanguage(DocumentLanguageData dld) throws DisqualifiedException {
|
protected void checkDocumentLanguage(DocumentLanguageData dld) throws DisqualifiedException {
|
||||||
@ -44,12 +44,6 @@ public abstract class AbstractDocumentProcessorPlugin {
|
|||||||
tagWords.add(key + ":" + value.toString().toLowerCase());
|
tagWords.add(key + ":" + value.toString().toLowerCase());
|
||||||
}
|
}
|
||||||
|
|
||||||
public MetaTagsBuilder addDomainCrawlData(CrawledDomain domain) {
|
|
||||||
add("ip", domain.ip);
|
|
||||||
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public MetaTagsBuilder addUrl(EdgeUrl url) {
|
public MetaTagsBuilder addUrl(EdgeUrl url) {
|
||||||
add("proto", url.proto);
|
add("proto", url.proto);
|
||||||
add("site", url.domain);
|
add("site", url.domain);
|
||||||
|
@ -94,10 +94,10 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DetailsWithWords createDetails(CrawledDomain crawledDomain, CrawledDocument crawledDocument)
|
public DetailsWithWords createDetails(CrawledDocument crawledDocument)
|
||||||
throws DisqualifiedException, URISyntaxException {
|
throws DisqualifiedException, URISyntaxException {
|
||||||
|
|
||||||
String documentBody = crawledDocument.documentBody.decode();
|
String documentBody = crawledDocument.documentBody;
|
||||||
|
|
||||||
if (languageFilter.isBlockedUnicodeRange(documentBody)) {
|
if (languageFilter.isBlockedUnicodeRange(documentBody)) {
|
||||||
throw new DisqualifiedException(DisqualificationReason.LANGUAGE);
|
throw new DisqualifiedException(DisqualificationReason.LANGUAGE);
|
||||||
@ -141,7 +141,7 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
|
|||||||
throw new DisqualifiedException(DisqualificationReason.QUALITY);
|
throw new DisqualifiedException(DisqualificationReason.QUALITY);
|
||||||
}
|
}
|
||||||
|
|
||||||
final Set<HtmlFeature> features = featureExtractor.getFeatures(crawledDomain, doc, dld);
|
final Set<HtmlFeature> features = featureExtractor.getFeatures(doc, dld);
|
||||||
ret.features = features;
|
ret.features = features;
|
||||||
ret.hashCode = dld.localitySensitiveHashCode();
|
ret.hashCode = dld.localitySensitiveHashCode();
|
||||||
|
|
||||||
@ -159,7 +159,6 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
|
|||||||
ret.generator = generatorParts.type();
|
ret.generator = generatorParts.type();
|
||||||
|
|
||||||
var tagWords = new MetaTagsBuilder()
|
var tagWords = new MetaTagsBuilder()
|
||||||
.addDomainCrawlData(crawledDomain)
|
|
||||||
.addPubDate(pubDate)
|
.addPubDate(pubDate)
|
||||||
.addUrl(url)
|
.addUrl(url)
|
||||||
.addFeatures(features)
|
.addFeatures(features)
|
||||||
|
@ -55,10 +55,10 @@ public class PlainTextDocumentProcessorPlugin extends AbstractDocumentProcessorP
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public DetailsWithWords createDetails(CrawledDomain crawledDomain, CrawledDocument crawledDocument)
|
public DetailsWithWords createDetails(CrawledDocument crawledDocument)
|
||||||
throws DisqualifiedException, URISyntaxException {
|
throws DisqualifiedException, URISyntaxException {
|
||||||
|
|
||||||
String documentBody = crawledDocument.documentBody.decode();
|
String documentBody = crawledDocument.documentBody;
|
||||||
|
|
||||||
if (languageFilter.isBlockedUnicodeRange(documentBody)) {
|
if (languageFilter.isBlockedUnicodeRange(documentBody)) {
|
||||||
throw new DisqualifiedException(DisqualifiedException.DisqualificationReason.LANGUAGE);
|
throw new DisqualifiedException(DisqualifiedException.DisqualificationReason.LANGUAGE);
|
||||||
@ -97,7 +97,6 @@ public class PlainTextDocumentProcessorPlugin extends AbstractDocumentProcessorP
|
|||||||
DocumentKeywordsBuilder words = keywordExtractor.extractKeywords(dld, url);
|
DocumentKeywordsBuilder words = keywordExtractor.extractKeywords(dld, url);
|
||||||
|
|
||||||
var tagWords = new MetaTagsBuilder()
|
var tagWords = new MetaTagsBuilder()
|
||||||
.addDomainCrawlData(crawledDomain)
|
|
||||||
.addPubDate(pubDate)
|
.addPubDate(pubDate)
|
||||||
.addUrl(url)
|
.addUrl(url)
|
||||||
.addFeatures(ret.features)
|
.addFeatures(ret.features)
|
||||||
|
@ -9,6 +9,7 @@ import nu.marginalia.converting.model.ProcessedDocument;
|
|||||||
import nu.marginalia.converting.processor.DomainProcessor;
|
import nu.marginalia.converting.processor.DomainProcessor;
|
||||||
import nu.marginalia.crawling.model.CrawledDocument;
|
import nu.marginalia.crawling.model.CrawledDocument;
|
||||||
import nu.marginalia.crawling.model.CrawledDomain;
|
import nu.marginalia.crawling.model.CrawledDomain;
|
||||||
|
import nu.marginalia.crawling.model.SerializableCrawlData;
|
||||||
import nu.marginalia.model.EdgeDomain;
|
import nu.marginalia.model.EdgeDomain;
|
||||||
import nu.marginalia.model.crawl.DomainIndexingState;
|
import nu.marginalia.model.crawl.DomainIndexingState;
|
||||||
import nu.marginalia.model.crawl.PubDate;
|
import nu.marginalia.model.crawl.PubDate;
|
||||||
@ -40,18 +41,17 @@ public class ConvertingIntegrationTest {
|
|||||||
public void testEmptyDomain() {
|
public void testEmptyDomain() {
|
||||||
var docs = new ArrayList<CrawledDocument>();
|
var docs = new ArrayList<CrawledDocument>();
|
||||||
|
|
||||||
var ret = domainProcessor.process(
|
var domain = new CrawledDomain("123", "memex.marginalia.nu", null, "OK", "-", "127.0.0.1",
|
||||||
new CrawledDomain("123", "memex.marginalia.nu", null, "OK", "-", "127.0.0.1",
|
docs, Collections.emptyList());
|
||||||
docs, Collections.emptyList()));
|
var ret = domainProcessor.process(asSerializableCrawlData(domain));
|
||||||
|
|
||||||
assertEquals(ret.state, DomainIndexingState.ACTIVE);
|
assertEquals(ret.state, DomainIndexingState.ACTIVE);
|
||||||
assertEquals(ret.domain, new EdgeDomain("memex.marginalia.nu"));
|
assertEquals(ret.domain, new EdgeDomain("memex.marginalia.nu"));
|
||||||
assertTrue(ret.documents.isEmpty());
|
assertTrue(ret.documents.isEmpty());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMemexMarginaliaNuDateInternalConsistency() throws IOException {
|
public void testMemexMarginaliaNuDateInternalConsistency() throws IOException {
|
||||||
var ret = domainProcessor.process(readMarginaliaWorkingSet());
|
var ret = domainProcessor.process(asSerializableCrawlData(readMarginaliaWorkingSet()));
|
||||||
ret.documents.stream().filter(ProcessedDocument::isProcessedFully).forEach(doc -> {
|
ret.documents.stream().filter(ProcessedDocument::isProcessedFully).forEach(doc -> {
|
||||||
int year = PubDate.fromYearByte(doc.details.metadata.year());
|
int year = PubDate.fromYearByte(doc.details.metadata.year());
|
||||||
Integer yearMeta = doc.details.pubYear;
|
Integer yearMeta = doc.details.pubYear;
|
||||||
@ -64,7 +64,7 @@ public class ConvertingIntegrationTest {
|
|||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMemexMarginaliaNu() throws IOException {
|
public void testMemexMarginaliaNu() throws IOException {
|
||||||
var ret = domainProcessor.process(readMarginaliaWorkingSet());
|
var ret = domainProcessor.process(asSerializableCrawlData(readMarginaliaWorkingSet()));
|
||||||
assertEquals(ret.state, DomainIndexingState.ACTIVE);
|
assertEquals(ret.state, DomainIndexingState.ACTIVE);
|
||||||
assertEquals(ret.domain, new EdgeDomain("memex.marginalia.nu"));
|
assertEquals(ret.domain, new EdgeDomain("memex.marginalia.nu"));
|
||||||
|
|
||||||
@ -110,7 +110,7 @@ public class ConvertingIntegrationTest {
|
|||||||
"OK",
|
"OK",
|
||||||
"",
|
"",
|
||||||
"",
|
"",
|
||||||
BigString.encode(readClassPathFile(p.toString())),
|
readClassPathFile(p.toString()),
|
||||||
Double.toString(Math.random()),
|
Double.toString(Math.random()),
|
||||||
"https://memex.marginalia.nu/" + file,
|
"https://memex.marginalia.nu/" + file,
|
||||||
null,
|
null,
|
||||||
@ -133,4 +133,13 @@ public class ConvertingIntegrationTest {
|
|||||||
return new String(Objects.requireNonNull(ClassLoader.getSystemResourceAsStream(s)).readAllBytes());
|
return new String(Objects.requireNonNull(ClassLoader.getSystemResourceAsStream(s)).readAllBytes());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private Iterator<SerializableCrawlData> asSerializableCrawlData(CrawledDomain domain) {
|
||||||
|
List<SerializableCrawlData> data = new ArrayList<>();
|
||||||
|
if (domain.doc != null) {
|
||||||
|
data.addAll(domain.doc);
|
||||||
|
}
|
||||||
|
data.add(domain);
|
||||||
|
return data.iterator();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -55,7 +55,11 @@ public class CrawlingThenConvertingIntegrationTest {
|
|||||||
|
|
||||||
CrawledDomain domain = crawl(specs);
|
CrawledDomain domain = crawl(specs);
|
||||||
|
|
||||||
var output = domainProcessor.process(domain);
|
List<SerializableCrawlData> data = new ArrayList<>();
|
||||||
|
data.add(domain);
|
||||||
|
data.addAll(domain.doc);
|
||||||
|
|
||||||
|
var output = domainProcessor.process(data.iterator());
|
||||||
|
|
||||||
for (var doc : output.documents) {
|
for (var doc : output.documents) {
|
||||||
if (doc.isOk()) {
|
if (doc.isOk()) {
|
||||||
|
@ -43,8 +43,7 @@ public class CrawlDataReference {
|
|||||||
return EasyLSH.hammingDistance(contentHashOne, contentHashOther) < 4;
|
return EasyLSH.hammingDistance(contentHashOne, contentHashOther) < 4;
|
||||||
}
|
}
|
||||||
|
|
||||||
private long contentHash(BigString documentBody) {
|
private long contentHash(String content) {
|
||||||
String content = documentBody.decode();
|
|
||||||
EasyLSH hash = new EasyLSH();
|
EasyLSH hash = new EasyLSH();
|
||||||
int next = 0;
|
int next = 0;
|
||||||
|
|
||||||
|
@ -323,7 +323,7 @@ public class CrawlerRetreiver {
|
|||||||
return;
|
return;
|
||||||
|
|
||||||
// Sniff the software based on the sample document
|
// Sniff the software based on the sample document
|
||||||
var doc = Jsoup.parse(sample.documentBody.decode());
|
var doc = Jsoup.parse(sample.documentBody);
|
||||||
crawlFrontier.setLinkFilter(linkFilterSelector.selectFilter(doc));
|
crawlFrontier.setLinkFilter(linkFilterSelector.selectFilter(doc));
|
||||||
|
|
||||||
for (var link : doc.getElementsByTag("link")) {
|
for (var link : doc.getElementsByTag("link")) {
|
||||||
@ -400,11 +400,9 @@ public class CrawlerRetreiver {
|
|||||||
CrawledDocument doc = reference.replaceOn304(fetchedDoc);
|
CrawledDocument doc = reference.replaceOn304(fetchedDoc);
|
||||||
|
|
||||||
if (doc.documentBody != null) {
|
if (doc.documentBody != null) {
|
||||||
var decoded = doc.documentBody.decode();
|
doc.documentBodyHash = createHash(doc.documentBody);
|
||||||
|
|
||||||
doc.documentBodyHash = createHash(decoded);
|
var parsedDoc = Jsoup.parse(doc.documentBody);
|
||||||
|
|
||||||
var parsedDoc = Jsoup.parse(decoded);
|
|
||||||
EdgeUrl url = new EdgeUrl(doc.url);
|
EdgeUrl url = new EdgeUrl(doc.url);
|
||||||
|
|
||||||
findLinks(url, parsedDoc);
|
findLinks(url, parsedDoc);
|
||||||
|
@ -295,7 +295,7 @@ public class HttpFetcherImpl implements HttpFetcher {
|
|||||||
.canonicalUrl(canonical)
|
.canonicalUrl(canonical)
|
||||||
.httpStatus(rsp.code())
|
.httpStatus(rsp.code())
|
||||||
.url(responseUrl.toString())
|
.url(responseUrl.toString())
|
||||||
.documentBody(BigString.encode(strData))
|
.documentBody(strData)
|
||||||
.build();
|
.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -402,7 +402,7 @@ public class HttpFetcherImpl implements HttpFetcher {
|
|||||||
|
|
||||||
private SimpleRobotRules parseRobotsTxt(CrawledDocument doc) {
|
private SimpleRobotRules parseRobotsTxt(CrawledDocument doc) {
|
||||||
return robotsParser.parseContent(doc.url,
|
return robotsParser.parseContent(doc.url,
|
||||||
doc.documentBody.decode().getBytes(),
|
doc.documentBody.getBytes(),
|
||||||
doc.contentType,
|
doc.contentType,
|
||||||
userAgent);
|
userAgent);
|
||||||
}
|
}
|
||||||
|
@ -43,13 +43,12 @@ public class CrawlerMockFetcherTest {
|
|||||||
.contentType("text/html")
|
.contentType("text/html")
|
||||||
.httpStatus(200)
|
.httpStatus(200)
|
||||||
.crawlerStatus(CrawlerDocumentStatus.OK.name())
|
.crawlerStatus(CrawlerDocumentStatus.OK.name())
|
||||||
.documentBody(BigString.encode(documentData))
|
.documentBody(documentData)
|
||||||
.build());
|
.build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@SneakyThrows
|
@SneakyThrows
|
||||||
private void registerUrlClasspathData(EdgeUrl url, String path) {
|
private void registerUrlClasspathData(EdgeUrl url, String path) {
|
||||||
var data = BigString.encode(CommonTestData.loadTestData(path));
|
|
||||||
|
|
||||||
mockData.put(url, CrawledDocument.builder()
|
mockData.put(url, CrawledDocument.builder()
|
||||||
.crawlId("1")
|
.crawlId("1")
|
||||||
@ -57,7 +56,7 @@ public class CrawlerMockFetcherTest {
|
|||||||
.contentType("text/html")
|
.contentType("text/html")
|
||||||
.httpStatus(200)
|
.httpStatus(200)
|
||||||
.crawlerStatus(CrawlerDocumentStatus.OK.name())
|
.crawlerStatus(CrawlerDocumentStatus.OK.name())
|
||||||
.documentBody(data)
|
.documentBody(CommonTestData.loadTestData(path))
|
||||||
.build());
|
.build());
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -32,7 +32,7 @@ public class AdblockExperiment extends Experiment {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void processDocument(CrawledDocument doc) {
|
private void processDocument(CrawledDocument doc) {
|
||||||
Document parsedDocument = Jsoup.parse(doc.documentBody.decode());
|
Document parsedDocument = Jsoup.parse(doc.documentBody);
|
||||||
|
|
||||||
if (simulator.hasAds(parsedDocument)) {
|
if (simulator.hasAds(parsedDocument)) {
|
||||||
System.out.println(doc.url);
|
System.out.println(doc.url);
|
||||||
|
@ -26,7 +26,7 @@ public class DebugConverterExperiment extends Experiment {
|
|||||||
for (var doc : domain.doc) {
|
for (var doc : domain.doc) {
|
||||||
if (doc.documentBody == null) continue;
|
if (doc.documentBody == null) continue;
|
||||||
|
|
||||||
var parsed = Jsoup.parse(doc.documentBody.decode());
|
var parsed = Jsoup.parse(doc.documentBody);
|
||||||
|
|
||||||
var tagExtractor = new BlogSpecialization.BlogTagExtractor();
|
var tagExtractor = new BlogSpecialization.BlogTagExtractor();
|
||||||
parsed.traverse(tagExtractor);
|
parsed.traverse(tagExtractor);
|
||||||
|
@ -41,7 +41,7 @@ public class SentenceStatisticsExperiment extends Experiment {
|
|||||||
for (var doc : domain.doc) {
|
for (var doc : domain.doc) {
|
||||||
if (doc.documentBody == null) continue;
|
if (doc.documentBody == null) continue;
|
||||||
|
|
||||||
var parsed = Jsoup.parse(doc.documentBody.decode());
|
var parsed = Jsoup.parse(doc.documentBody);
|
||||||
|
|
||||||
parsed.body().filter(new DomPruningFilter(0.5));
|
parsed.body().filter(new DomPruningFilter(0.5));
|
||||||
|
|
||||||
|
@ -31,12 +31,12 @@ public class SiteStatisticsExperiment extends Experiment {
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean process(CrawledDomain domain) {
|
public boolean process(CrawledDomain domain) {
|
||||||
var ret = domainProcessor.process(domain);
|
// var ret = domainProcessor.process(domain);
|
||||||
|
//
|
||||||
ret.documents.stream()
|
// ret.documents.stream()
|
||||||
.filter(ProcessedDocument::isProcessedFully)
|
// .filter(ProcessedDocument::isProcessedFully)
|
||||||
.sorted(Comparator.comparing(doc -> doc.details.metadata.topology()))
|
// .sorted(Comparator.comparing(doc -> doc.details.metadata.topology()))
|
||||||
.forEach(doc -> System.out.println(doc.url + ":" + doc.details.metadata));
|
// .forEach(doc -> System.out.println(doc.url + ":" + doc.details.metadata));
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
@ -37,7 +37,7 @@ public class TopicExperiment extends Experiment {
|
|||||||
for (var doc : domain.doc) {
|
for (var doc : domain.doc) {
|
||||||
if (doc.documentBody == null) continue;
|
if (doc.documentBody == null) continue;
|
||||||
|
|
||||||
var parsed = Jsoup.parse(doc.documentBody.decode());
|
var parsed = Jsoup.parse(doc.documentBody);
|
||||||
|
|
||||||
parsed.body().filter(new DomPruningFilter(0.5));
|
parsed.body().filter(new DomPruningFilter(0.5));
|
||||||
var dld = se.extractSentences(parsed);
|
var dld = se.extractSentences(parsed);
|
||||||
|
@ -58,7 +58,7 @@ public class TermFrequencyExtractor {
|
|||||||
continue;
|
continue;
|
||||||
docCount.incrementAndGet();
|
docCount.incrementAndGet();
|
||||||
|
|
||||||
Document parsed = Jsoup.parse(doc.documentBody.decode());
|
Document parsed = Jsoup.parse(doc.documentBody);
|
||||||
parsed.body().filter(new DomPruningFilter(0.5));
|
parsed.body().filter(new DomPruningFilter(0.5));
|
||||||
|
|
||||||
DocumentLanguageData dld = se.get().extractSentences(parsed);
|
DocumentLanguageData dld = se.get().extractSentences(parsed);
|
||||||
|
2
run/env/service.env
vendored
2
run/env/service.env
vendored
@ -1,4 +1,4 @@
|
|||||||
WMSA_HOME=run/
|
WMSA_HOME=run/
|
||||||
CONTROL_SERVICE_OPTS="-DdistPath=/dist"
|
CONTROL_SERVICE_OPTS="-DdistPath=/dist"
|
||||||
CONVERTER_OPTS="-ea -Xmx16G -XX:-CompactStrings -XX:+UseParallelGC -XX:GCTimeRatio=14 -XX:ParallelGCThreads=15"
|
CONVERTER_OPTS="-ea -Xmx16G -XX:-CompactStrings -XX:+UseParallelGC -XX:GCTimeRatio=14 -XX:ParallelGCThreads=15"
|
||||||
CRAWLER_OPTS="-Dbigstring.disabled=true -Xmx16G -XX:+UseParallelGC -XX:GCTimeRatio=14 -XX:ParallelGCThreads=15"
|
CRAWLER_OPTS="-Xmx16G -XX:+UseParallelGC -XX:GCTimeRatio=14 -XX:ParallelGCThreads=15"
|
Loading…
Reference in New Issue
Block a user