Add loader for slop data in converter.

Also alter CrawledDocument to not require String parsing of the underlying byte[] data.  This should reduce the number of large memory allocations quite significantly, hopefully reducing the GC churn a bit.
This commit is contained in:
Viktor Lofgren 2024-12-17 15:40:24 +01:00
parent f6f036b9b1
commit 3714104976
42 changed files with 402 additions and 626 deletions

View File

@ -19,6 +19,7 @@ import nu.marginalia.model.idx.WordFlags;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.List;
@ -91,7 +92,7 @@ public class DocumentProcessor {
DocumentClass documentClass,
DocumentDecorator documentDecorator,
DomainLinks externalDomainLinks,
ProcessedDocument ret) throws URISyntaxException, DisqualifiedException
ProcessedDocument ret) throws URISyntaxException, IOException, DisqualifiedException
{
var crawlerStatus = CrawlerDocumentStatus.valueOf(crawledDocument.crawlerStatus);
@ -109,7 +110,7 @@ public class DocumentProcessor {
ret.state = crawlerStatusToUrlState(crawledDocument.crawlerStatus, crawledDocument.httpStatus);
final var plugin = findPlugin(crawledDocument);
AbstractDocumentProcessorPlugin plugin = findPlugin(crawledDocument);
EdgeUrl url = new EdgeUrl(crawledDocument.url);
LinkTexts linkTexts = anchorTextKeywords.getAnchorTextKeywords(externalDomainLinks, url);

View File

@ -65,15 +65,15 @@ public class DomainProcessor {
if (sizeHint > SIDELOAD_THRESHOLD) {
// If the file is too big, we run a processing mode that doesn't
// require loading the entire dataset into RAM
return sideloadProcessing(dataStream, sizeHint);
return simpleProcessing(dataStream, sizeHint);
}
return fullProcessing(dataStream);
}
public SideloadProcessing sideloadProcessing(SerializableCrawlDataStream dataStream, int sizeHint, Collection<String> extraKeywords) {
public SimpleProcessing simpleProcessing(SerializableCrawlDataStream dataStream, int sizeHint, Collection<String> extraKeywords) {
try {
return new SideloadProcessing(dataStream, sizeHint, extraKeywords);
return new SimpleProcessing(dataStream, sizeHint, extraKeywords);
}
catch (Exception ex) {
logger.warn("Failed to process domain sideload", ex);
@ -81,9 +81,9 @@ public class DomainProcessor {
}
}
public SideloadProcessing sideloadProcessing(SerializableCrawlDataStream dataStream, int sizeHint) {
public SimpleProcessing simpleProcessing(SerializableCrawlDataStream dataStream, int sizeHint) {
try {
return new SideloadProcessing(dataStream, sizeHint);
return new SimpleProcessing(dataStream, sizeHint);
}
catch (Exception ex) {
logger.warn("Failed to process domain sideload", ex);
@ -91,93 +91,6 @@ public class DomainProcessor {
}
}
public class SideloadProcessing implements ConverterBatchWritableIf, SideloadSource {
private final SerializableCrawlDataStream dataStream;
private final ProcessedDomain domain;
private final DocumentDecorator documentDecorator;
private final Set<String> processedUrls = new HashSet<>();
private final DomainLinks externalDomainLinks;
private final LshDocumentDeduplicator deduplicator = new LshDocumentDeduplicator();
private static final ProcessingIterator.Factory iteratorFactory = ProcessingIterator.factory(8,
Integer.getInteger("java.util.concurrent.ForkJoinPool.common.parallelism", Runtime.getRuntime().availableProcessors())
);
SideloadProcessing(SerializableCrawlDataStream dataStream, int sizeHint) throws IOException {
this(dataStream, sizeHint, List.of());
}
SideloadProcessing(SerializableCrawlDataStream dataStream, int sizeHint, Collection<String> extraKeywords) throws IOException {
this.dataStream = dataStream;
if (!dataStream.hasNext() || !(dataStream.next() instanceof CrawledDomain crawledDomain))
{
throw new IllegalStateException("First record must be a domain, was " + dataStream.next().getClass().getSimpleName());
}
domain = new ProcessedDomain();
domain.sizeloadSizeAdvice = sizeHint == 0 ? 10_000 : sizeHint;
documentDecorator = new DocumentDecorator();
documentDecorator.addTerms(extraKeywords);
processDomain(crawledDomain, domain, documentDecorator);
externalDomainLinks = anchorTagsSource.getAnchorTags(domain.domain);
}
@Override
public ProcessedDomain getDomain() {
return domain;
}
@Override
public Iterator<ProcessedDocument> getDocumentsStream() {
return iteratorFactory.create((taskConsumer) -> {
while (dataStream.hasNext())
{
if (!(dataStream.next() instanceof CrawledDocument doc))
continue;
if (doc.url == null || !processedUrls.add(doc.url))
continue;
taskConsumer.accept(() -> {
var processedDoc = documentProcessor.process(doc, domain.domain, externalDomainLinks, documentDecorator);
synchronized (deduplicator) {
deduplicator.markIfDuplicate(processedDoc);
}
if (processedDoc.isProcessedFully()) {
// This is a bit sketchy, but we need to set the size and topology to something
processedDoc.details.metadata = processedDoc.details.metadata.withSizeAndTopology(
10_000, externalDomainLinks.countForUrl(processedDoc.url));
}
return processedDoc;
});
}
});
}
@Override
public void write(ConverterBatchWriter writer) throws IOException {
writer.writeSideloadSource(this);
}
@Override
public String id() {
return domain.domain.toString();
}
@Override
public void close() throws Exception {
dataStream.close();
deduplicator.close();
}
}
@Nullable
public ProcessedDomain fullProcessing(SerializableCrawlDataStream dataStream) {
try {
@ -209,7 +122,7 @@ public class DomainProcessor {
continue;
if (doc.url == null)
continue;
if (doc.documentBody.isBlank())
if (doc.documentBodyBytes.length == 0)
continue;
if (!processedUrls.add(doc.url))
continue;
@ -236,6 +149,90 @@ public class DomainProcessor {
}
}
/** The simple processing track processes documents individually, and does not perform any domain-level analysis.
* This is needed to process extremely large domains, which would otherwise eat up too much RAM.
*/
public class SimpleProcessing implements ConverterBatchWritableIf, SideloadSource {
private final SerializableCrawlDataStream dataStream;
private final ProcessedDomain domain;
private final DocumentDecorator documentDecorator;
private final Set<String> processedUrls = new HashSet<>();
private final DomainLinks externalDomainLinks;
private final LshDocumentDeduplicator deduplicator = new LshDocumentDeduplicator();
private static final ProcessingIterator.Factory iteratorFactory = ProcessingIterator.factory(8,
Integer.getInteger("java.util.concurrent.ForkJoinPool.common.parallelism", Runtime.getRuntime().availableProcessors())
);
SimpleProcessing(SerializableCrawlDataStream dataStream, int sizeHint) throws IOException {
this(dataStream, sizeHint, List.of());
}
SimpleProcessing(SerializableCrawlDataStream dataStream, int sizeHint, Collection<String> extraKeywords) throws IOException {
this.dataStream = dataStream;
if (!dataStream.hasNext() || !(dataStream.next() instanceof CrawledDomain crawledDomain))
{
throw new IllegalStateException("First record must be a domain, was " + dataStream.next().getClass().getSimpleName());
}
domain = new ProcessedDomain();
domain.sizeloadSizeAdvice = sizeHint == 0 ? 10_000 : sizeHint;
documentDecorator = new DocumentDecorator();
documentDecorator.addTerms(extraKeywords);
processDomain(crawledDomain, domain, documentDecorator);
externalDomainLinks = anchorTagsSource.getAnchorTags(domain.domain);
}
@Override
public ProcessedDomain getDomain() {
return domain;
}
@Override
public Iterator<ProcessedDocument> getDocumentsStream() {
return dataStream.map((next) -> {
if (!(next instanceof CrawledDocument doc))
return Optional.empty();
if (doc.url == null || !processedUrls.add(doc.url))
return Optional.empty();
var processedDoc = documentProcessor.process(doc, domain.domain, externalDomainLinks, documentDecorator);
synchronized (deduplicator) {
deduplicator.markIfDuplicate(processedDoc);
}
if (processedDoc.isProcessedFully()) {
// This is a bit sketchy, but we need to set the size and topology to something
processedDoc.details.metadata = processedDoc.details.metadata.withSizeAndTopology(
10_000, externalDomainLinks.countForUrl(processedDoc.url));
}
return Optional.of(processedDoc);
});
}
@Override
public void write(ConverterBatchWriter writer) throws IOException {
writer.writeSideloadSource(this);
}
@Override
public String id() {
return domain.domain.toString();
}
@Override
public void close() throws Exception {
dataStream.close();
deduplicator.close();
}
}
private void processDomain(CrawledDomain crawledDomain,
ProcessedDomain domain,
DocumentDecorator decorator)

View File

@ -24,7 +24,7 @@ public class DocumentValuator {
double scriptPenalty = getScriptPenalty(parsedDocument);
double chatGptPenalty = getChatGptContentFarmPenalty(parsedDocument);
int rawLength = crawledDocument.documentBody.length();
int rawLength = crawledDocument.documentBodyBytes.length;
if (textLength == 0) {
throw new DisqualifiedException(DisqualifiedException.DisqualificationReason.LENGTH);

View File

@ -218,7 +218,10 @@ public class FeatureExtractor {
}
}
if (features.contains(HtmlFeature.JS) && adblockSimulator.hasAds(doc.clone())) {
if (features.contains(HtmlFeature.JS)
// remove while disabled to get rid of expensive clone() call:
// adblockSimulator.hasAds(doc.clone())
) {
features.add(HtmlFeature.ADVERTISEMENT);
}

View File

@ -14,6 +14,7 @@ import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.model.html.HtmlStandard;
import javax.annotation.Nullable;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.HashSet;
import java.util.List;
@ -25,7 +26,7 @@ public abstract class AbstractDocumentProcessorPlugin {
this.languageFilter = languageFilter;
}
public abstract DetailsWithWords createDetails(CrawledDocument crawledDocument, LinkTexts linkTexts, DocumentClass documentClass) throws DisqualifiedException, URISyntaxException;
public abstract DetailsWithWords createDetails(CrawledDocument crawledDocument, LinkTexts linkTexts, DocumentClass documentClass) throws DisqualifiedException, URISyntaxException, IOException;
public abstract boolean isApplicable(CrawledDocument doc);
protected void checkDocumentLanguage(DocumentLanguageData dld) throws DisqualifiedException {
@ -86,6 +87,7 @@ public abstract class AbstractDocumentProcessorPlugin {
return this;
}
public MetaTagsBuilder addPubDate(PubDate pubDate) {
if (pubDate.year() > 1900) {

View File

@ -6,6 +6,7 @@ import nu.marginalia.converting.model.DisqualifiedException;
import nu.marginalia.converting.model.DocumentHeaders;
import nu.marginalia.converting.model.GeneratorType;
import nu.marginalia.converting.model.ProcessedDocumentDetails;
import nu.marginalia.converting.processor.AcceptableAds;
import nu.marginalia.converting.processor.DocumentClass;
import nu.marginalia.converting.processor.MetaRobotsTag;
import nu.marginalia.converting.processor.logic.*;
@ -32,11 +33,11 @@ import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.model.html.HtmlStandard;
import nu.marginalia.model.idx.DocumentFlags;
import nu.marginalia.model.idx.DocumentMetadata;
import org.jsoup.Jsoup;
import org.jsoup.nodes.Document;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.EnumSet;
import java.util.HashSet;
@ -51,7 +52,6 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
private final double minDocumentQuality;
private final FeatureExtractor featureExtractor;
private final TitleExtractor titleExtractor;
private final DocumentKeywordExtractor keywordExtractor;
private final PubDateSniffer pubDateSniffer;
@ -74,7 +74,6 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
@Named("min-document-quality") Double minDocumentQuality,
LanguageFilter languageFilter,
FeatureExtractor featureExtractor,
TitleExtractor titleExtractor,
DocumentKeywordExtractor keywordExtractor,
PubDateSniffer pubDateSniffer,
DocumentLengthLogic documentLengthLogic,
@ -89,7 +88,6 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
this.minDocumentQuality = minDocumentQuality;
this.featureExtractor = featureExtractor;
this.titleExtractor = titleExtractor;
this.keywordExtractor = keywordExtractor;
this.pubDateSniffer = pubDateSniffer;
this.metaRobotsTag = metaRobotsTag;
@ -108,19 +106,17 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
public DetailsWithWords createDetails(CrawledDocument crawledDocument,
LinkTexts linkTexts,
DocumentClass documentClass)
throws DisqualifiedException, URISyntaxException {
throws DisqualifiedException, URISyntaxException, IOException {
String documentBody = crawledDocument.documentBody;
if (languageFilter.isBlockedUnicodeRange(documentBody)) {
if (languageFilter.isBlockedUnicodeRange(crawledDocument.documentBody(512))) {
throw new DisqualifiedException(DisqualificationReason.LANGUAGE);
}
if (documentBody.length() > MAX_DOCUMENT_LENGTH_BYTES) { // 128kb
documentBody = documentBody.substring(0, MAX_DOCUMENT_LENGTH_BYTES);
}
Document doc = crawledDocument.parseBody();
Document doc = Jsoup.parse(documentBody);
if (AcceptableAds.hasAcceptableAdsTag(doc)) {
throw new DisqualifiedException(DisqualifiedException.DisqualificationReason.ACCEPTABLE_ADS);
}
if (!metaRobotsTag.allowIndexingByMetaTag(doc)) {
throw new DisqualifiedException(DisqualificationReason.FORBIDDEN);
@ -138,32 +134,33 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
}
var prunedDoc = specialization.prune(doc);
DocumentLanguageData dld = sentenceExtractorProvider.get().extractSentences(prunedDoc);
checkDocumentLanguage(dld);
var ret = new ProcessedDocumentDetails();
final int length = getLength(doc);
final HtmlStandard standard = getHtmlStandard(doc);
final double quality = documentValuator.getQuality(crawledDocument, standard, doc, length);
if (isDisqualified(documentClass, url, quality, doc.title())) {
throw new DisqualifiedException(DisqualificationReason.QUALITY);
}
DocumentLanguageData dld = sentenceExtractorProvider.get().extractSentences(prunedDoc);
checkDocumentLanguage(dld);
documentLengthLogic.validateLength(dld, specialization.lengthModifier() * documentClass.lengthLimitModifier());
var ret = new ProcessedDocumentDetails();
ret.length = length;
ret.standard = standard;
ret.title = specialization.getTitle(doc, dld, crawledDocument.url);
documentLengthLogic.validateLength(dld, specialization.lengthModifier() * documentClass.lengthLimitModifier());
final Set<HtmlFeature> features = featureExtractor.getFeatures(url, doc, documentHeaders, dld);
ret.features = features;
ret.quality = documentValuator.adjustQuality(quality, features);
ret.hashCode = dld.localitySensitiveHashCode();
if (isDisqualified(documentClass, url, quality, ret.title)) {
throw new DisqualifiedException(DisqualificationReason.QUALITY);
}
PubDate pubDate = pubDateSniffer.getPubDate(documentHeaders, url, doc, standard, true);
EnumSet<DocumentFlags> documentFlags = documentFlags(features, generatorParts.type());

View File

@ -71,7 +71,7 @@ public class PlainTextDocumentProcessorPlugin extends AbstractDocumentProcessorP
DocumentClass documentClass)
throws DisqualifiedException, URISyntaxException {
String documentBody = crawledDocument.documentBody;
String documentBody = crawledDocument.documentBody();
if (languageFilter.isBlockedUnicodeRange(documentBody)) {
throw new DisqualifiedException(DisqualifiedException.DisqualificationReason.LANGUAGE);

View File

@ -19,6 +19,7 @@ import nu.marginalia.model.idx.DocumentMetadata;
import nu.marginalia.model.idx.WordFlags;
import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
import java.time.LocalDateTime;
import java.util.EnumSet;
import java.util.List;
@ -50,7 +51,7 @@ public class SideloaderProcessing {
"OK",
"NP",
"",
body,
body.getBytes(StandardCharsets.UTF_8),
false,
null,
null

View File

@ -98,7 +98,7 @@ public class ConvertingIntegrationTest {
@Test
public void testMemexMarginaliaNuSideloadProcessing() throws IOException {
var ret = domainProcessor.sideloadProcessing(asSerializableCrawlData(readMarginaliaWorkingSet()), 100);
var ret = domainProcessor.simpleProcessing(asSerializableCrawlData(readMarginaliaWorkingSet()), 100);
assertNotNull(ret);
assertEquals("memex.marginalia.nu", ret.id());
@ -146,7 +146,7 @@ public class ConvertingIntegrationTest {
"OK",
"",
"",
readClassPathFile(p.toString()),
readClassPathFile(p.toString()).getBytes(),
false,
null,
null

View File

@ -2,6 +2,11 @@ package nu.marginalia.contenttype;
import org.apache.commons.lang3.StringUtils;
import java.nio.charset.Charset;
import java.nio.charset.IllegalCharsetNameException;
import java.nio.charset.StandardCharsets;
import java.nio.charset.UnsupportedCharsetException;
/** Content type and charset of a document
* @param contentType The content type, e.g. "text/html"
* @param charset The charset, e.g. "UTF-8"
@ -18,6 +23,32 @@ public record ContentType(String contentType, String charset) {
return new ContentType(contentType, charset);
}
public Charset asCharset() {
try {
int eqAt = charset.indexOf('=');
String charset = this.charset;
if (eqAt >= 0) {
charset = charset.substring(eqAt + 1);
}
if (Charset.isSupported(charset)) {
return Charset.forName(charset);
} else {
return StandardCharsets.UTF_8;
}
}
catch (IllegalCharsetNameException ex) {
// Fall back to UTF-8 if we don't understand what this is. It's *probably* fine? Maybe?
return StandardCharsets.UTF_8;
}
catch (UnsupportedCharsetException ex) {
// This is usually like Macintosh Latin
// (https://en.wikipedia.org/wiki/Macintosh_Latin_encoding)
//
// It's close enough to 8859-1 to serve
return StandardCharsets.ISO_8859_1;
}
}
public boolean is(String contentType) {
return this.contentType.equalsIgnoreCase(contentType);
}

View File

@ -1,9 +1,12 @@
package nu.marginalia.contenttype;
import org.jsoup.Jsoup;
import org.jsoup.nodes.Document;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.charset.Charset;
import java.nio.charset.IllegalCharsetNameException;
import java.nio.charset.StandardCharsets;
import java.nio.charset.UnsupportedCharsetException;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@ -23,24 +26,25 @@ public class DocumentBodyToString {
return new String(data, charset);
}
public static Document getParsedData(ContentType type, byte[] data, String url) throws IOException {
final Charset charset;
if (type.charset() == null || type.charset().isBlank()) {
charset = StandardCharsets.UTF_8;
} else {
charset = charsetMap.computeIfAbsent(type, DocumentBodyToString::computeCharset);
}
ByteArrayInputStream bais = new ByteArrayInputStream(data);
return Jsoup.parse(bais, charset.name(), url);
}
private static Charset computeCharset(ContentType type) {
try {
if (type.charset() == null || type.charset().isBlank())
return StandardCharsets.UTF_8;
else {
return Charset.forName(type.charset());
}
}
catch (IllegalCharsetNameException ex) {
// Fall back to UTF-8 if we don't understand what this is. It's *probably* fine? Maybe?
if (type.charset() == null || type.charset().isBlank())
return StandardCharsets.UTF_8;
}
catch (UnsupportedCharsetException ex) {
// This is usually like Macintosh Latin
// (https://en.wikipedia.org/wiki/Macintosh_Latin_encoding)
//
// It's close enough to 8859-1 to serve
return StandardCharsets.ISO_8859_1;
else {
return type.asCharset();
}
}
}

View File

@ -184,7 +184,7 @@ public class WarcRecorder implements AutoCloseable {
writer.write(item);
}
private void saveOldResponse(EdgeUrl url, String contentType, int statusCode, String documentBody, @Nullable String headers, ContentTags contentTags) {
private void saveOldResponse(EdgeUrl url, String contentType, int statusCode, byte[] documentBody, @Nullable String headers, ContentTags contentTags) {
try {
WarcDigestBuilder responseDigestBuilder = new WarcDigestBuilder();
WarcDigestBuilder payloadDigestBuilder = new WarcDigestBuilder();
@ -194,7 +194,7 @@ public class WarcRecorder implements AutoCloseable {
if (documentBody == null) {
bytes = new byte[0];
} else {
bytes = documentBody.getBytes();
bytes = documentBody;
}
// Create a synthesis of custom headers and the original headers
@ -263,7 +263,7 @@ public class WarcRecorder implements AutoCloseable {
* an E-Tag or Last-Modified header, and the server responds with a 304 Not Modified. In this
* scenario we want to record the data as it was in the previous crawl, but not re-fetch it.
*/
public void writeReferenceCopy(EdgeUrl url, String contentType, int statusCode, String documentBody, @Nullable String headers, ContentTags ctags) {
public void writeReferenceCopy(EdgeUrl url, String contentType, int statusCode, byte[] documentBody, @Nullable String headers, ContentTags ctags) {
saveOldResponse(url, contentType, statusCode, documentBody, headers, ctags);
}

View File

@ -58,7 +58,7 @@ public class CrawlDataReference implements AutoCloseable {
return null;
}
public static boolean isContentBodySame(String one, String other) {
public static boolean isContentBodySame(byte[] one, byte[] other) {
final long contentHashOne = contentHash(one);
final long contentHashOther = contentHash(other);
@ -66,7 +66,7 @@ public class CrawlDataReference implements AutoCloseable {
return EasyLSH.hammingDistance(contentHashOne, contentHashOther) < 4;
}
private static long contentHash(String content) {
private static long contentHash(byte[] content) {
EasyLSH hash = new EasyLSH();
int next = 0;
@ -74,8 +74,8 @@ public class CrawlDataReference implements AutoCloseable {
// In a naive best-effort fashion, extract the text
// content of the document and feed it into the LSH
for (int i = 0; i < content.length(); i++) {
char c = content.charAt(i);
for (byte b : content) {
char c = (char) b;
if (c == '<') {
isInTag = true;
} else if (c == '>') {

View File

@ -17,7 +17,6 @@ import nu.marginalia.link_parser.LinkParser;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.body.HttpFetchResult;
import org.jsoup.Jsoup;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -283,14 +282,14 @@ public class CrawlerRetreiver implements AutoCloseable {
else if (fetchedDoc instanceof HttpFetchResult.Result304Raw && reference.doc() != null) {
var doc = reference.doc();
warcRecorder.writeReferenceCopy(top, doc.contentType, doc.httpStatus, doc.documentBody, doc.headers, contentTags);
warcRecorder.writeReferenceCopy(top, doc.contentType, doc.httpStatus, doc.documentBodyBytes, doc.headers, contentTags);
fetchedDoc = new HttpFetchResult.Result304ReplacedWithReference(doc.url,
new ContentType(doc.contentType, "UTF-8"),
doc.documentBody);
doc.documentBodyBytes);
if (doc.documentBody != null) {
var parsed = Jsoup.parse(doc.documentBody);
if (doc.documentBodyBytes != null) {
var parsed = doc.parseBody();
crawlFrontier.enqueueLinksFromDocument(top, parsed);
crawlFrontier.addVisited(top);

View File

@ -1,6 +1,5 @@
package nu.marginalia.crawl.retreival.revisit;
import com.google.common.base.Strings;
import crawlercommons.robots.SimpleRobotRules;
import nu.marginalia.crawl.fetcher.ContentTags;
import nu.marginalia.crawl.fetcher.warc.WarcRecorder;
@ -11,7 +10,8 @@ import nu.marginalia.crawl.retreival.DomainCrawlFrontier;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.body.HttpFetchResult;
import nu.marginalia.model.crawldata.CrawledDocument;
import org.jsoup.Jsoup;
import java.io.IOException;
/** This class encapsulates the logic for re-visiting a domain that has already been crawled.
* We may use information from the previous crawl to inform the next crawl, specifically the
@ -70,7 +70,7 @@ public class CrawlerRevisitor {
// unlikely to produce anything meaningful for us.
if (doc.httpStatus != 200)
continue;
if (Strings.isNullOrEmpty(doc.documentBody))
if (!doc.hasBody())
continue;
if (!crawlFrontier.filterLink(url))
@ -117,14 +117,19 @@ public class CrawlerRevisitor {
// fashion to make sure we eventually catch changes over time
// and ensure we discover new links
// Hoover up any links from the document
crawlFrontier.enqueueLinksFromDocument(url, Jsoup.parse(doc.documentBody));
try {
// Hoover up any links from the document
crawlFrontier.enqueueLinksFromDocument(url, doc.parseBody());
}
catch (IOException ex) {
//
}
// Add a WARC record so we don't repeat this
warcRecorder.writeReferenceCopy(url,
doc.contentType,
doc.httpStatus,
doc.documentBody,
doc.documentBodyBytes,
doc.headers,
new ContentTags(doc.etagMaybe, doc.lastModifiedMaybe)
);

View File

@ -2,8 +2,6 @@ package nu.marginalia.crawl.retreival.revisit;
import nu.marginalia.crawl.fetcher.ContentTags;
import nu.marginalia.crawl.retreival.CrawlDataReference;
import nu.marginalia.model.body.DocumentBodyExtractor;
import nu.marginalia.model.body.DocumentBodyResult;
import nu.marginalia.model.body.HttpFetchResult;
import nu.marginalia.model.crawldata.CrawledDocument;
@ -35,21 +33,17 @@ public record DocumentWithReference(
return false;
if (doc == null)
return false;
if (doc.documentBody == null)
if (doc.documentBodyBytes.length == 0)
return false;
if (!(DocumentBodyExtractor.asString(resultOk) instanceof DocumentBodyResult.Ok<String> bodyOk)) {
return false;
}
return CrawlDataReference.isContentBodySame(doc.documentBody, bodyOk.body());
return CrawlDataReference.isContentBodySame(doc.documentBodyBytes, resultOk.bytesRaw());
}
public ContentTags getContentTags() {
if (null == doc)
return ContentTags.empty();
if (doc.documentBody == null || doc.httpStatus != 200)
if (doc.documentBodyBytes.length == 0 || doc.httpStatus != 200)
return ContentTags.empty();
String lastmod = doc.getLastModified();

View File

@ -2,10 +2,14 @@ package nu.marginalia.io;
import nu.marginalia.model.crawldata.SerializableCrawlData;
import org.jetbrains.annotations.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Iterator;
import java.util.Optional;
import java.util.function.Function;
/** Closable iterator exceptional over serialized crawl data
* The data may appear in any order, and the iterator must be closed.
@ -13,7 +17,7 @@ import java.util.Iterator;
* @see CrawledDomainReader
* */
public interface SerializableCrawlDataStream extends AutoCloseable {
Logger logger = LoggerFactory.getLogger(SerializableCrawlDataStream.class);
SerializableCrawlData next() throws IOException;
@ -26,6 +30,41 @@ public interface SerializableCrawlDataStream extends AutoCloseable {
@Nullable
default Path path() { return null; }
default <T> Iterator<T> map(Function<SerializableCrawlData, Optional<T>> mapper) {
return new Iterator<>() {
T next = null;
public boolean hasNext() {
if (next != null)
return true;
try {
while (SerializableCrawlDataStream.this.hasNext()) {
var val = mapper.apply(SerializableCrawlDataStream.this.next());
if (val.isPresent()) {
next = val.get();
return true;
}
}
}
catch (IOException ex) {
logger.error("Error during stream", ex);
}
return false;
}
public T next() {
if (next == null && !hasNext())
throw new IllegalStateException("No more data to read");
T ret = next;
next = null;
return ret;
}
};
}
// Dummy iterator over nothing
static SerializableCrawlDataStream empty() {
return new SerializableCrawlDataStream() {
@ -46,7 +85,6 @@ public interface SerializableCrawlDataStream extends AutoCloseable {
public boolean hasNext() { return iterator.hasNext(); }
public void close() {}
};
}
}

View File

@ -1,7 +1,6 @@
package nu.marginalia.io.crawldata.format;
import nu.marginalia.contenttype.ContentType;
import nu.marginalia.contenttype.DocumentBodyToString;
import nu.marginalia.hash.MurmurHash3_128;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeUrl;
@ -18,6 +17,7 @@ import java.nio.file.Path;
import java.util.*;
import java.util.stream.Stream;
@Deprecated
public class ParquetSerializableCrawlDataStream implements AutoCloseable, SerializableCrawlDataStream {
private static final Logger logger = LoggerFactory.getLogger(ParquetSerializableCrawlDataStream.class);
@ -124,9 +124,7 @@ public class ParquetSerializableCrawlDataStream implements AutoCloseable, Serial
}
else if (nextRecord.body != null) {
try {
bodyString = DocumentBodyToString.getStringData(
ContentType.parse(nextRecord.contentType),
nextRecord.body);
ContentType.parse(nextRecord.contentType);
} catch (Exception ex) {
logger.error("Failed to convert body to string", ex);
status = CrawlerDocumentStatus.BAD_CHARSET;
@ -147,7 +145,7 @@ public class ParquetSerializableCrawlDataStream implements AutoCloseable, Serial
status.toString(),
"",
nextRecord.headers,
bodyString,
nextRecord.body,
// this field isn't actually used, maybe we can skip calculating it?
nextRecord.cookies,
lastModified,

View File

@ -1,7 +1,6 @@
package nu.marginalia.io.crawldata.format;
import nu.marginalia.contenttype.ContentType;
import nu.marginalia.contenttype.DocumentBodyToString;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.crawldata.*;
@ -36,7 +35,14 @@ public class SlopSerializableCrawlDataStream implements AutoCloseable, Serializa
reader = new SlopCrawlDataRecord.FilteringReader(file) {
@Override
public boolean filter(String url, int status, String contentType) {
return contentType.toLowerCase().startsWith("text/");
String ctLc = contentType.toLowerCase();
if (ctLc.startsWith("text/"))
return true;
else if (ctLc.startsWith("x-marginalia/"))
return true;
return false;
}
};
}
@ -121,7 +127,6 @@ public class SlopSerializableCrawlDataStream implements AutoCloseable, Serializa
}
private void createDocumentRecord(SlopCrawlDataRecord nextRecord) {
String bodyString = "";
CrawlerDocumentStatus status = CrawlerDocumentStatus.OK;
if (nextRecord.contentType().startsWith("x-marginalia/advisory;state=content-type-failed-probe")) {
@ -136,9 +141,7 @@ public class SlopSerializableCrawlDataStream implements AutoCloseable, Serializa
}
else if (nextRecord.body() != null) {
try {
bodyString = DocumentBodyToString.getStringData(
ContentType.parse(nextRecord.contentType()),
nextRecord.body());
ContentType.parse(nextRecord.contentType());
} catch (Exception ex) {
logger.error("Failed to convert body to string", ex);
status = CrawlerDocumentStatus.BAD_CHARSET;
@ -156,7 +159,7 @@ public class SlopSerializableCrawlDataStream implements AutoCloseable, Serializa
status.toString(),
"",
nextRecord.headers(),
bodyString,
nextRecord.body(),
// this field isn't actually used, maybe we can skip calculating it?
nextRecord.cookies(),
null,

View File

@ -18,7 +18,7 @@ public class DocumentBodyExtractor {
return asBytes(fetchOk);
}
else if (result instanceof HttpFetchResult.Result304ReplacedWithReference retained) {
return new DocumentBodyResult.Ok<>(retained.contentType(), retained.body().getBytes());
return new DocumentBodyResult.Ok<>(retained.contentType(), retained.body());
}
return new DocumentBodyResult.Error<>(CrawlerDocumentStatus.ERROR, "Fetch Result Not Ok");

View File

@ -114,20 +114,10 @@ public sealed interface HttpFetchResult {
*
* @see Result304Raw for the case where the document has not yet been replaced with the reference data.
*/
record Result304ReplacedWithReference(String url, ContentType contentType, String body) implements HttpFetchResult {
record Result304ReplacedWithReference(String url, ContentType contentType, byte[] body) implements HttpFetchResult {
public boolean isOk() {
return true;
}
public Optional<Document> parseDocument() {
try {
return Optional.of(Jsoup.parse(body));
}
catch (Exception ex) {
return Optional.empty();
}
}
}
/** Fetching resulted in an exception */

View File

@ -1,8 +1,16 @@
package nu.marginalia.model.crawldata;
import nu.marginalia.contenttype.ContentType;
import nu.marginalia.contenttype.DocumentBodyToString;
import nu.marginalia.model.EdgeUrl;
import org.apache.commons.lang3.StringUtils;
import org.jetbrains.annotations.Nullable;
import org.jsoup.nodes.Document;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Objects;
public final class CrawledDocument implements SerializableCrawlData {
public String crawlId;
@ -19,8 +27,37 @@ public final class CrawledDocument implements SerializableCrawlData {
@Nullable
public String headers;
public String documentBody;
public String documentBody() {
return DocumentBodyToString.getStringData(
ContentType.parse(contentType),
documentBodyBytes);
}
public Document parseBody() throws IOException {
return DocumentBodyToString.getParsedData(
ContentType.parse(contentType),
documentBodyBytes,
url);
}
public boolean hasBody() {
return documentBodyBytes.length > 0;
}
public String documentBody(int sampleSize) {
if (sampleSize >= documentBodyBytes.length) {
return documentBody();
}
byte[] bytes = new byte[sampleSize];
System.arraycopy(documentBodyBytes, 0, bytes, 0, sampleSize);
return DocumentBodyToString.getStringData(
ContentType.parse(contentType),
bytes);
}
public byte[] documentBodyBytes;
/**
* This is not guaranteed to be set in all versions of the format,
* information may come in CrawledDomain instead
@ -30,7 +67,7 @@ public final class CrawledDocument implements SerializableCrawlData {
public String lastModifiedMaybe;
public String etagMaybe;
public CrawledDocument(String crawlId, String url, String contentType, String timestamp, int httpStatus, String crawlerStatus, String crawlerStatusDesc, @Nullable String headers, String documentBody, Boolean hasCookies, String lastModifiedMaybe, String etagMaybe) {
public CrawledDocument(String crawlId, String url, String contentType, String timestamp, int httpStatus, String crawlerStatus, String crawlerStatusDesc, @Nullable String headers, byte[] documentBodyBytes, Boolean hasCookies, String lastModifiedMaybe, String etagMaybe) {
this.crawlId = crawlId;
this.url = url;
this.contentType = contentType;
@ -39,7 +76,7 @@ public final class CrawledDocument implements SerializableCrawlData {
this.crawlerStatus = crawlerStatus;
this.crawlerStatusDesc = crawlerStatusDesc;
this.headers = headers;
this.documentBody = documentBody;
this.documentBodyBytes = Objects.requireNonNullElse(documentBodyBytes, new byte[] {});
this.hasCookies = hasCookies;
this.lastModifiedMaybe = lastModifiedMaybe;
this.etagMaybe = etagMaybe;
@ -106,7 +143,7 @@ public final class CrawledDocument implements SerializableCrawlData {
}
public String toString() {
return "CrawledDocument(crawlId=" + this.crawlId + ", url=" + this.url + ", contentType=" + this.contentType + ", timestamp=" + this.timestamp + ", httpStatus=" + this.httpStatus + ", crawlerStatus=" + this.crawlerStatus + ", crawlerStatusDesc=" + this.crawlerStatusDesc + ", headers=" + this.headers + ", documentBody=" + this.documentBody + ", hasCookies=" + this.hasCookies + ", lastModifiedMaybe=" + this.lastModifiedMaybe + ", etagMaybe=" + this.etagMaybe + ")";
return "CrawledDocument(crawlId=" + this.crawlId + ", url=" + this.url + ", contentType=" + this.contentType + ", timestamp=" + this.timestamp + ", httpStatus=" + this.httpStatus + ", crawlerStatus=" + this.crawlerStatus + ", crawlerStatusDesc=" + this.crawlerStatusDesc + ", headers=" + this.headers + ", documentBody=" + documentBody() + ", hasCookies=" + this.hasCookies + ", lastModifiedMaybe=" + this.lastModifiedMaybe + ", etagMaybe=" + this.etagMaybe + ")";
}
public static class CrawledDocumentBuilder {
@ -118,7 +155,7 @@ public final class CrawledDocument implements SerializableCrawlData {
private String crawlerStatus;
private String crawlerStatusDesc;
private @Nullable String headers;
private String documentBody;
private byte[] documentBodyBytes = new byte[0];
private String recrawlState;
private Boolean hasCookies;
private String lastModifiedMaybe;
@ -168,10 +205,13 @@ public final class CrawledDocument implements SerializableCrawlData {
}
public CrawledDocumentBuilder documentBody(String documentBody) {
this.documentBody = documentBody;
this.documentBodyBytes = documentBody.getBytes(StandardCharsets.UTF_8);
return this;
}
public CrawledDocumentBuilder documentBodyBytes(byte[] documentBodyBytes) {
this.documentBodyBytes = documentBodyBytes;
return this;
}
@Deprecated
public CrawledDocumentBuilder recrawlState(String recrawlState) {
this.recrawlState = recrawlState;
@ -194,11 +234,11 @@ public final class CrawledDocument implements SerializableCrawlData {
}
public CrawledDocument build() {
return new CrawledDocument(this.crawlId, this.url, this.contentType, this.timestamp, this.httpStatus, this.crawlerStatus, this.crawlerStatusDesc, this.headers, this.documentBody, this.hasCookies, this.lastModifiedMaybe, this.etagMaybe);
return new CrawledDocument(this.crawlId, this.url, this.contentType, this.timestamp, this.httpStatus, this.crawlerStatus, this.crawlerStatusDesc, this.headers, this.documentBodyBytes, this.hasCookies, this.lastModifiedMaybe, this.etagMaybe);
}
public String toString() {
return "CrawledDocument.CrawledDocumentBuilder(crawlId=" + this.crawlId + ", url=" + this.url + ", contentType=" + this.contentType + ", timestamp=" + this.timestamp + ", httpStatus=" + this.httpStatus + ", crawlerStatus=" + this.crawlerStatus + ", crawlerStatusDesc=" + this.crawlerStatusDesc + ", headers=" + this.headers + ", documentBody=" + this.documentBody + ", recrawlState=" + this.recrawlState + ", hasCookies=" + this.hasCookies + ", lastModifiedMaybe=" + this.lastModifiedMaybe + ", etagMaybe=" + this.etagMaybe + ")";
return "CrawledDocument.CrawledDocumentBuilder(crawlId=" + this.crawlId + ", url=" + this.url + ", contentType=" + this.contentType + ", timestamp=" + this.timestamp + ", httpStatus=" + this.httpStatus + ", crawlerStatus=" + this.crawlerStatus + ", crawlerStatusDesc=" + this.crawlerStatusDesc + ", headers=" + this.headers + ", documentBodyBytes=" + Arrays.toString(this.documentBodyBytes) + ", recrawlState=" + this.recrawlState + ", hasCookies=" + this.hasCookies + ", lastModifiedMaybe=" + this.lastModifiedMaybe + ", etagMaybe=" + this.etagMaybe + ")";
}
}
}

View File

@ -80,7 +80,7 @@ class CrawledDocumentParquetRecordFileWriterTest {
var document = (CrawledDocument) secondItem;
assertEquals("https://www.marginalia.nu/", document.url);
assertEquals("text/html", document.contentType);
assertEquals("hello world", document.documentBody);
assertEquals("hello world", document.documentBody());
assertEquals(200, document.httpStatus);
}
@ -103,7 +103,7 @@ class CrawledDocumentParquetRecordFileWriterTest {
System.out.println(doc.url);
System.out.println(doc.contentType);
System.out.println(doc.httpStatus);
System.out.println(doc.documentBody.length());
System.out.println(doc.documentBody().length());
}
}
} catch (IOException e) {

View File

@ -1,10 +1,11 @@
package nu.marginalia.slop;
import nu.marginalia.parquet.crawldata.CrawledDocumentParquetRecordFileReader;
import nu.marginalia.contenttype.ContentType;
import org.jsoup.Jsoup;
import org.junit.jupiter.api.Test;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.time.Duration;
import java.time.Instant;
@ -14,39 +15,71 @@ class SlopCrawlDataRecordTest {
@Test
public void test() throws IOException {
Files.deleteIfExists(Path.of("/tmp/steam.slop.zip"));
SlopCrawlDataRecord.convertFromParquet(
Path.of("/home/vlofgren/Downloads/_storage_crawl-data__23-10-21T15_08_43.750_3b_41_3b41e714-store.steampowered.com.parquet"),
Path.of("/tmp/steam.slop.zip")
);
// Files.deleteIfExists(Path.of("/tmp/steam.slop.zip"));
// SlopCrawlDataRecord.convertFromParquet(
// Path.of("/home/vlofgren/Downloads/_storage_crawl-data__23-10-21T15_08_43.750_3b_41_3b41e714-store.steampowered.com.parquet"),
// Path.of("/tmp/steam.slop.zip")
// );
// long st = 0;
// try (var reader = new SlopCrawlDataRecord.FilteringReader(Path.of("/tmp/steam.slop.zip")) {
// public boolean filter(String url, int status, String contentType) {
// return contentType.startsWith("text/html");
// }
// }) {
// Instant start = Instant.now();
// while (reader.hasRemaining()) {
// var next = reader.get();
// byte[] body = next.body();
//
// st += Jsoup.parse(new String(body)).title().length();
// }
// System.out.println(st + " " + Duration.between(start, Instant.now()));
// }
System.out.println("BEGIN Slop");
for (int i = 0; i < 6; i++) {
int sz = 0;
Instant start = Instant.now();
try (var reader = new SlopCrawlDataRecord.Reader(Path.of("/tmp/steam.slop.zip")) {
public boolean filter(String url, int status, String contentType) {
return contentType.startsWith("text/html");
}
}) {
while (reader.hasRemaining()) {
sz += reader.get().httpStatus();
}
long st = 0;
try (var reader = new SlopCrawlDataRecord.FilteringReader(Path.of("/tmp/steam.slop.zip")) {
public boolean filter(String url, int status, String contentType) {
return contentType.startsWith("text/html");
}
Instant end = Instant.now();
System.out.println("END Iter " + sz + " " + Duration.between(start, end));
}
System.out.println("END Slop");
System.out.println("BEGIN Parquet");
for (int i = 0; i < 6; i++) {
}) {
Instant start = Instant.now();
int sz = CrawledDocumentParquetRecordFileReader.stream(Path.of("/home/vlofgren/Downloads/_storage_crawl-data__23-10-21T15_08_43.750_3b_41_3b41e714-store.steampowered.com.parquet"))
.filter(record -> record.contentType.startsWith("text/html"))
.mapToInt(record -> record.httpStatus).sum();
Instant end = Instant.now();
System.out.println("END Iter " + sz + " " + Duration.between(start, end));
while (reader.hasRemaining()) {
var next = reader.get();
byte[] body = next.body();
st += Jsoup.parse(new ByteArrayInputStream(body), ContentType.parse(next.contentType()).asCharset().name(), next.url()).title().length();
}
System.out.println(Duration.between(start, Instant.now()));
}
System.out.println("END Parquet");
// System.out.println("BEGIN Slop");
// for (int i = 0; i < 6; i++) {
// int sz = 0;
// Instant start = Instant.now();
// try (var reader = new SlopCrawlDataRecord.Reader(Path.of("/tmp/steam.slop.zip")) {
// public boolean filter(String url, int status, String contentType) {
// return contentType.startsWith("text/html");
// }
// }) {
// while (reader.hasRemaining()) {
// sz += reader.get().httpStatus();
// }
// }
// Instant end = Instant.now();
// System.out.println("END Iter " + sz + " " + Duration.between(start, end));
// }
// System.out.println("END Slop");
//
// System.out.println("BEGIN Parquet");
// for (int i = 0; i < 6; i++) {
// Instant start = Instant.now();
// int sz = CrawledDocumentParquetRecordFileReader.stream(Path.of("/home/vlofgren/Downloads/_storage_crawl-data__23-10-21T15_08_43.750_3b_41_3b41e714-store.steampowered.com.parquet"))
// .filter(record -> record.contentType.startsWith("text/html"))
// .mapToInt(record -> record.httpStatus).sum();
// Instant end = Instant.now();
// System.out.println("END Iter " + sz + " " + Duration.between(start, end));
// }
// System.out.println("END Parquet");
}
}

View File

@ -80,7 +80,7 @@ class WarcRecorderTest {
recorder.writeReferenceCopy(new EdgeUrl("https://www.marginalia.nu/"),
"text/html",
200,
"<?doctype html><html><body>test</body></html>",
"<?doctype html><html><body>test</body></html>".getBytes(),
null,
ContentTags.empty());
}
@ -116,7 +116,7 @@ class WarcRecorderTest {
recorder.writeReferenceCopy(new EdgeUrl("https://www.marginalia.nu/"),
"text/html",
200,
"<?doctype html><html><body>test</body></html>",
"<?doctype html><html><body>test</body></html>".getBytes(),
null, ContentTags.empty());
}

View File

@ -131,7 +131,7 @@ public class CrawlerMockFetcherTest {
public HttpFetchResult fetchContent(EdgeUrl url, WarcRecorder recorder, ContentTags tags, ProbeType probeType) {
logger.info("Fetching {}", url);
if (mockData.containsKey(url)) {
byte[] bodyBytes = mockData.get(url).documentBody.getBytes();
byte[] bodyBytes = mockData.get(url).documentBodyBytes;
try {
return new HttpFetchResult.ResultOk(

View File

@ -14,7 +14,6 @@ import nu.marginalia.storage.FileStorageService;
import nu.marginalia.storage.model.FileStorage;
import nu.marginalia.storage.model.FileStorageId;
import org.apache.commons.lang3.StringUtils;
import org.jsoup.Jsoup;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -52,11 +51,9 @@ public class AtagExporter implements ExporterIf {
try (var bw = new BufferedWriter(new OutputStreamWriter(new GZIPOutputStream(Files.newOutputStream(tmpFile, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING)))))
{
Path crawlerLogFile = inputDir.resolve("crawler.log");
var tagWriter = new ATagCsvWriter(bw);
for (var item : WorkLog.iterable(crawlerLogFile)) {
for (var item : WorkLog.iterable(inputDir.resolve("crawler.log"))) {
if (Thread.interrupted()) {
throw new InterruptedException();
}
@ -89,15 +86,19 @@ public class AtagExporter implements ExporterIf {
while (stream.hasNext()) {
if (!(stream.next() instanceof CrawledDocument doc))
continue;
if (null == doc.documentBody)
if (!doc.hasBody())
continue;
if (!doc.contentType.toLowerCase().startsWith("text/html"))
continue;
var baseUrl = new EdgeUrl(doc.url);
var parsed = Jsoup.parse(doc.documentBody);
var parsed = doc.parseBody();
for (var atag : parsed.getElementsByTag("a")) {
if (!atag.hasAttr("href")) {
continue;
}
String linkText = atag.text();
if (!linkFilter.isLinkTextEligible(linkText)) {

View File

@ -12,7 +12,6 @@ import nu.marginalia.process.log.WorkLog;
import nu.marginalia.storage.FileStorageService;
import nu.marginalia.storage.model.FileStorage;
import nu.marginalia.storage.model.FileStorageId;
import org.jsoup.Jsoup;
import java.io.BufferedWriter;
import java.io.IOException;
@ -81,13 +80,13 @@ public class FeedExporter implements ExporterIf {
while (stream.hasNext()) {
if (!(stream.next() instanceof CrawledDocument doc))
continue;
if (null == doc.documentBody)
if (!doc.hasBody())
continue;
if (!doc.contentType.toLowerCase().startsWith("text/html"))
continue;
var baseUrl = new EdgeUrl(doc.url);
var parsed = Jsoup.parse(doc.documentBody);
var parsed = doc.parseBody();
List<EdgeUrl> feedUrls = new ArrayList<>();
for (var link : parsed.select("link[rel=alternate]")) {

View File

@ -15,7 +15,6 @@ import nu.marginalia.storage.FileStorageService;
import nu.marginalia.storage.model.FileStorage;
import nu.marginalia.storage.model.FileStorageId;
import nu.marginalia.util.SimpleBlockingThreadPool;
import org.jsoup.Jsoup;
import org.jsoup.nodes.Document;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -110,13 +109,13 @@ public class TermFrequencyExporter implements ExporterIf {
return;
if (!(stream.next() instanceof CrawledDocument doc)) continue;
if (doc.documentBody == null) continue;
if (!doc.hasBody()) continue;
if (!doc.contentType.toLowerCase().startsWith("text/html"))
continue;
docCount.incrementAndGet();
Document parsed = Jsoup.parse(doc.documentBody);
Document parsed = doc.parseBody();
parsed.body().filter(new DomPruningFilter(0.5));
DocumentLanguageData dld = se.extractSentences(parsed);

View File

@ -119,12 +119,16 @@ public class LiveCrawlDataSet implements AutoCloseable {
}
}
private String decompress(byte[] data) {
private String decompressStr(byte[] data) {
return new String(decompressBytes(data));
}
private byte[] decompressBytes(byte[] data) {
// gzip decompression
try (var bis = new ByteArrayInputStream(data);
var gzip = new GZIPInputStream(bis))
try (ByteArrayInputStream bis = new ByteArrayInputStream(data);
GZIPInputStream gzip = new GZIPInputStream(bis))
{
return new String(gzip.readAllBytes());
return gzip.readAllBytes();
}
catch (IOException ex) {
throw new RuntimeException(ex);
@ -177,8 +181,8 @@ public class LiveCrawlDataSet implements AutoCloseable {
dataStack = new ArrayList<>();
while (rs.next()) {
String url = rs.getString("url");
String body = decompress(rs.getBytes("body"));
String headers = decompress(rs.getBytes("headers"));
byte[] body = decompressBytes(rs.getBytes("body"));
String headers = decompressStr(rs.getBytes("headers"));
dataStack.add(new CrawledDocument(
"LIVE",

View File

@ -197,7 +197,7 @@ public class LiveCrawlerMain extends ProcessMainClass {
writer.setOrdinalOffset(67_000_000);
for (SerializableCrawlDataStream stream : hb.wrap("Processing", dataSet.getDataStreams())) {
writer.write(domainProcessor.sideloadProcessing(stream, 0, Set.of("special:live")));
writer.write(domainProcessor.simpleProcessing(stream, 0, Set.of("special:live")));
}
}

View File

@ -51,7 +51,7 @@ public class LiveCrawlDataSetTest {
case CrawledDocument document -> {
dataCount++;
Assertions.assertEquals("https://www.example.com/", document.url);
Assertions.assertEquals("test", document.documentBody);
Assertions.assertEquals("test", document.documentBody());
}
}
}

View File

@ -6,7 +6,6 @@ import nu.marginalia.converting.ConverterModule;
import nu.marginalia.io.CrawledDomainReader;
import nu.marginalia.process.log.WorkLog;
import nu.marginalia.service.module.DatabaseModule;
import nu.marginalia.tools.experiments.*;
import java.io.IOException;
import java.nio.file.Path;
@ -16,13 +15,6 @@ import java.util.Map;
public class ExperimentRunnerMain {
private static Map<String, Class<? extends Experiment>> experiments = Map.of(
"test", TestExperiment.class,
"adblock", AdblockExperiment.class,
"topic", TopicExperiment.class,
"sentence-statistics", SentenceStatisticsExperiment.class,
"site-statistics", SiteStatisticsExperiment.class,
"export-atags", ExportExternalLinksExperiment.class,
"debug-converter", DebugConverterExperiment.class
);
public static void main(String... args) throws IOException {

View File

@ -1,45 +0,0 @@
package nu.marginalia.tools.experiments;
import com.google.inject.Inject;
import nu.marginalia.converting.processor.DocumentProcessor;
import nu.marginalia.converting.processor.classifier.adblock.AdblockSimulator;
import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.model.crawldata.CrawledDomain;
import nu.marginalia.tools.LegacyExperiment;
import org.jsoup.Jsoup;
import org.jsoup.nodes.Document;
public class AdblockExperiment extends LegacyExperiment {
private final AdblockSimulator simulator;
@Inject
public AdblockExperiment(AdblockSimulator simulator) {
this.simulator = simulator;
}
@Override
public boolean process(CrawledDomain domain) {
if (domain.doc == null) return true;
for (var doc : domain.doc) {
if (DocumentProcessor.isAcceptedContentType(doc) && "OK".equals(doc.crawlerStatus)) {
processDocument(doc);
}
}
return true;
}
private void processDocument(CrawledDocument doc) {
Document parsedDocument = Jsoup.parse(doc.documentBody);
if (simulator.hasAds(parsedDocument)) {
System.out.println(doc.url);
}
}
@Override
public void onFinish() {
}
}

View File

@ -1,46 +0,0 @@
package nu.marginalia.tools.experiments;
import com.google.inject.Inject;
import nu.marginalia.converting.processor.DomainProcessor;
import nu.marginalia.converting.processor.plugin.specialization.BlogSpecialization;
import nu.marginalia.model.crawldata.CrawledDomain;
import nu.marginalia.tools.LegacyExperiment;
import org.jsoup.Jsoup;
public class DebugConverterExperiment extends LegacyExperiment {
private final DomainProcessor domainProcessor;
@Inject
public DebugConverterExperiment(DomainProcessor domainProcessor) {
this.domainProcessor = domainProcessor;
}
@Override
public boolean process(CrawledDomain domain) {
if (domain.doc == null) return true;
for (var doc : domain.doc) {
if (doc.documentBody == null) continue;
var parsed = Jsoup.parse(doc.documentBody);
var tagExtractor = new BlogSpecialization.BlogTagExtractor();
parsed.traverse(tagExtractor);
var tags = tagExtractor.getTags();
if (!tags.isEmpty()) {
System.out.println(tags);
}
}
return true;
}
@Override
public void onFinish() {
}
}

View File

@ -1,71 +0,0 @@
package nu.marginalia.tools.experiments;
import com.google.inject.Inject;
import gnu.trove.set.hash.TLongHashSet;
import nu.marginalia.hash.MurmurHash3_128;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.link_parser.LinkParser;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.tools.Experiment;
import org.jsoup.Jsoup;
import java.util.Objects;
public class ExportExternalLinksExperiment extends Experiment {
@Inject
public ExportExternalLinksExperiment() {
}
private static final LinkParser linkParser = new LinkParser();
MurmurHash3_128 hash = new MurmurHash3_128();
@Override
public boolean process(SerializableCrawlDataStream stream) {
TLongHashSet hashes = new TLongHashSet();
try {
while (stream.hasNext()) {
if (!(stream.next() instanceof CrawledDocument doc))
continue;
if (null == doc.documentBody)
continue;
var baseUrl = new EdgeUrl(doc.url);
var parsed = Jsoup.parse(doc.documentBody);
for (var atag : parsed.getElementsByTag("a")) {
String linkText = atag.text();
if (linkText.isBlank())
continue;
var linkOpt = linkParser.parseLinkPermissive(baseUrl, atag);
linkOpt
.filter(url -> !Objects.equals(url.domain, baseUrl.domain))
.filter(url -> hashes.add(hash.hashNearlyASCII(linkText) ^ hash.hashNearlyASCII(url.toString())))
.ifPresent(url ->
System.out.printf("\"%s\",\"%s\",\"%s\"\n",
csvify(url),
csvify(baseUrl.domain),
csvify(linkText)));
}
}
}
catch (Exception ex) {
ex.printStackTrace();
return false;
}
return true;
}
private static String csvify(Object field) {
return field.toString().replace("\"", "\"\"");
}
@Override
public void onFinish() {
}
}

View File

@ -1,70 +0,0 @@
package nu.marginalia.tools.experiments;
import com.google.inject.Inject;
import nu.marginalia.WmsaHome;
import nu.marginalia.converting.processor.logic.dom.DomPruningFilter;
import nu.marginalia.keyword.DocumentKeywordExtractor;
import nu.marginalia.keyword.LinkTexts;
import nu.marginalia.language.sentence.SentenceExtractor;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.crawldata.CrawledDomain;
import nu.marginalia.term_frequency_dict.TermFrequencyDict;
import nu.marginalia.tools.LegacyExperiment;
import org.jsoup.Jsoup;
import java.io.BufferedOutputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.PrintWriter;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.nio.file.Path;
public class SentenceStatisticsExperiment extends LegacyExperiment {
SentenceExtractor se = new SentenceExtractor(WmsaHome.getLanguageModels());
DocumentKeywordExtractor documentKeywordExtractor = new DocumentKeywordExtractor(new TermFrequencyDict(WmsaHome.getLanguageModels()));
Path filename;
PrintWriter writer;
@Inject
public SentenceStatisticsExperiment() throws IOException {
filename = Files.createTempFile(getClass().getSimpleName(), ".csv");
System.out.println("Writing to " + filename);
writer = new PrintWriter(new BufferedOutputStream(new FileOutputStream(filename.toFile())));
}
private void logLine(String message) {
System.out.printf("\u001b[2K\r%s", message);
}
@Override
public boolean process(CrawledDomain domain) {
if (domain.doc == null) return true;
logLine("Processing: " + domain.domain);
ByteBuffer workArea = ByteBuffer.allocate(8192);
for (var doc : domain.doc) {
if (doc.documentBody == null) continue;
var parsed = Jsoup.parse(doc.documentBody);
parsed.body().filter(new DomPruningFilter(0.5));
var dld = se.extractSentences(parsed);
var keywords = documentKeywordExtractor.extractKeywords(dld, new LinkTexts(), EdgeUrl.parse(doc.url).orElseThrow());
keywords.build(workArea);
}
return true;
}
@Override
public void onFinish() {
logLine("Done!\n");
writer.close();
}
}

View File

@ -1,43 +0,0 @@
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.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.tools.Experiment;
import java.util.Comparator;
public class SiteStatisticsExperiment extends Experiment {
private final DomainProcessor domainProcessor;
@Inject
public SiteStatisticsExperiment(DomainProcessor domainProcessor) {
this.domainProcessor = domainProcessor;
}
@Override
public boolean process(SerializableCrawlDataStream stream) {
var ret = domainProcessor.fullProcessing(stream);
ret.documents.stream()
.filter(ProcessedDocument::isProcessedFully)
.sorted(Comparator.comparing(doc -> doc.details.metadata.topology()))
.flatMap(doc -> doc.details.feedLinks.stream())
.map(EdgeUrl::toString)
.min(Comparator.comparing(String::length))
.ifPresent(url -> {
System.out.printf("\"%s\",\"%s\"\n", ret.domain, url);
});
return true;
}
@Override
public void onFinish() {
}
}

View File

@ -1,16 +0,0 @@
package nu.marginalia.tools.experiments;
import nu.marginalia.model.crawldata.CrawledDomain;
import nu.marginalia.tools.LegacyExperiment;
public class TestExperiment extends LegacyExperiment {
@Override
public boolean process(CrawledDomain domain) {
return true;
}
@Override
public void onFinish() {
System.out.println("Tada!");
}
}

View File

@ -1,64 +0,0 @@
package nu.marginalia.tools.experiments;
import com.google.inject.Inject;
import nu.marginalia.WmsaHome;
import nu.marginalia.converting.processor.classifier.topic.AdHocDetector;
import nu.marginalia.converting.processor.logic.dom.DomPruningFilter;
import nu.marginalia.language.sentence.SentenceExtractor;
import nu.marginalia.model.crawldata.CrawledDomain;
import nu.marginalia.tools.LegacyExperiment;
import org.jsoup.Jsoup;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
public class TopicExperiment extends LegacyExperiment {
AdHocDetector detector;
SentenceExtractor se = new SentenceExtractor(WmsaHome.getLanguageModels());
Path filename = null;
public void args(String... args) {
filename = Path.of(args[0]);
try {
detector = new AdHocDetector(Files.readAllLines(filename));
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Inject
public TopicExperiment() {
}
@Override
public boolean process(CrawledDomain domain) {
if (domain.doc == null) return true;
for (var doc : domain.doc) {
if (doc.documentBody == null) continue;
var parsed = Jsoup.parse(doc.documentBody);
parsed.body().filter(new DomPruningFilter(0.5));
var dld = se.extractSentences(parsed);
if (dld.totalNumWords() < 50)
continue;
if (detector.testP(dld) > 0.5) {
System.out.println("match\t" + doc.url);
}
}
return true;
}
@Override
public void onFinish() {
}
}

View File

@ -134,7 +134,7 @@ public class IntegrationTest {
</p>
</body>
</html>
""",
""".getBytes(),
"",
ContentTags.empty()
);

View File

@ -230,7 +230,7 @@ dependencyResolutionManagement {
library('jetty-util','org.eclipse.jetty','jetty-util').version('9.4.54.v20240208')
library('jetty-servlet','org.eclipse.jetty','jetty-servlet').version('9.4.54.v20240208')
library('slop', 'nu.marginalia', 'slop').version('0.0.9-org-3-SNAPSHOT')
library('slop', 'nu.marginalia', 'slop').version('0.0.9-org-4-SNAPSHOT')
bundle('jetty', ['jetty-server', 'jetty-util', 'jetty-servlet'])