1
1
mirror of https://github.com/MarginaliaSearch/MarginaliaSearch.git synced 2025-10-05 21:22:39 +02:00

Compare commits

...

20 Commits

Author SHA1 Message Date
Viktor Lofgren
594df64b20 (domain-info) Use appropriate sqlite database when fetching feed status 2025-01-02 20:20:36 +01:00
Viktor Lofgren
78eb1417a7 (service) Only block on SingleNodeChannelPool creation in QueryClient
The code was always blocking for up to 5s while waiting for the remote end to become available, meaning some services would stall for several seconds on start-up for no sensible reason.

This should make most services start faster as a result.
2025-01-02 18:42:01 +01:00
Viktor Lofgren
67edc8f90d (domain-info) Only flag domains with rss feed items as having a feed 2025-01-02 17:41:52 +01:00
Viktor Lofgren
5f576b7d0c (query-parser) Strip leading underlines
This addresses issue #140, where __builtin_ffs gives no results.
2025-01-02 14:39:03 +01:00
Viktor Lofgren
0b65164f60 (chore) Fix broken test 2025-01-01 18:06:29 +01:00
Viktor Lofgren
9be477de33 (domain-info) Add a feed flag to domain info
This is a bit of a sketchy solution that requires both assistant services to run on the same physical machine.
2025-01-01 18:02:33 +01:00
Viktor Lofgren
710af4999a (feed-fetcher) Add " entity mapping in feed fetcher 2025-01-01 15:45:17 +01:00
Viktor Lofgren
baeb4a46cd (search) Reintroduce query rewriting for recipes, add rules for wikis and forums 2024-12-31 16:05:00 +01:00
Viktor Lofgren
5e2a8e9f27 (deploy) Add capability of adding tags to deploy script 2024-12-31 16:04:13 +01:00
Viktor
cc1a5bdf90 Merge pull request #138 from MarginaliaSearch/vlofgren-patch-1
Update ROADMAP.md
2024-12-31 14:41:02 +01:00
Viktor
7f7b1ffaba Update ROADMAP.md 2024-12-31 14:40:34 +01:00
Viktor Lofgren
0ea8092350 (search) Add link promoting the redesign beta 2024-12-30 15:47:13 +01:00
Viktor Lofgren
483d29497e (deploy) Add hashbang to deploy script 2024-12-30 15:47:13 +01:00
Viktor Lofgren
bae44497fe (crawler) Add a new system property crawler.maxFetchSize
This gives the same upper limit to the live crawler and the big boy crawler, though the live crawler will reject items too large, and the big crawler will truncate at that point.
2024-12-30 15:10:11 +01:00
Viktor Lofgren
0d59202aca (crawler) Do not remove W/-prefix on weak e-tags
The server expects to get them back prefixed, as we received them.
2024-12-27 20:56:42 +01:00
Viktor Lofgren
0ca43f0c9c (live-crawler) Improve live crawler short-circuit logic
We should not wait until we've fetched robots.txt to decide whether we have any data to fetch!  This makes the live crawler very slow and leads to unnecessary requests.
2024-12-27 20:54:42 +01:00
Viktor Lofgren
3bc99639a0 (feed-fetcher) Make feed fetcher requests conditional
Add `If-None-Match` and `If-Modified-Since` headers as appropriate to the feed fetcher's requests.  On well-configured web servers, this should short-circuit the request and reduce the amount of bandwidth and processing that is necessary.

A new table was added to the FeedDb to hold one etag per domain.

If-Modified-Since semantics are based on the creation date for the feed database, which should serve as a cutoff date for the earliest update we can have received.

This completes the changes for Issue #136.
2024-12-27 15:10:15 +01:00
Viktor Lofgren
927bc0b63c (live-crawler) Add Accept-Encoding: gzip to outbound requests
This change adds `Accept-Encoding: gzip` to all outbound requests from the live crawler and feed fetcher, and the corresponding decoding logic for the compressed response data.

The change addresses issue #136, save for making the fetcher's requests conditional.
2024-12-27 03:59:34 +01:00
Viktor Lofgren
d968801dc1 (converter) Drop feed data from SlopDomainRecord
Also remove feed extraction from converter.  This is the crawler's responsibility now.
2024-12-26 17:57:08 +01:00
Viktor Lofgren
89db69d360 (crawler) Correct feed URLs in domain state db
Discovered feed URLs were given a double slash after their domain name in the DB.  This will go away in the URL normalizer, so the URLs are still viable, but the commit fixes the issue regardless.
2024-12-26 15:18:31 +01:00
36 changed files with 611 additions and 403 deletions

View File

@@ -8,20 +8,10 @@ be implemented as well.
Major goals:
* Reach 1 billion pages indexed
* Improve technical ability of indexing and search. Although this area has improved a bit, the
search engine is still not very good at dealing with longer queries.
## Proper Position Index (COMPLETED 2024-09)
The search engine uses a fixed width bit mask to indicate word positions. It has the benefit
of being very fast to evaluate and works well for what it is, but is inaccurate and has the
drawback of making support for quoted search terms inaccurate and largely reliant on indexing
word n-grams known beforehand. This limits the ability to interpret longer queries.
The positions mask should be supplemented or replaced with a more accurate (e.g.) gamma coded positions
list, as is the civilized way of doing this.
Completed with PR [#99](https://github.com/MarginaliaSearch/MarginaliaSearch/pull/99)
* Improve technical ability of indexing and search. ~~Although this area has improved a bit, the
search engine is still not very good at dealing with longer queries.~~ (As of PR [#129](https://github.com/MarginaliaSearch/MarginaliaSearch/pull/129), this has improved significantly. There is still more work to be done )
## Hybridize crawler w/ Common Crawl data
@@ -37,8 +27,7 @@ Retaining the ability to independently crawl the web is still strongly desirable
## Safe Search
The search engine has a bit of a problem showing spicy content mixed in with the results. It would be desirable
to have a way to filter this out. It's likely something like a URL blacklist (e.g. [UT1](https://dsi.ut-capitole.fr/blacklists/index_en.php) )
The search engine has a bit of a problem showing spicy content mixed in with the results. It would be desirable to have a way to filter this out. It's likely something like a URL blacklist (e.g. [UT1](https://dsi.ut-capitole.fr/blacklists/index_en.php) )
combined with naive bayesian filter would go a long way, or something more sophisticated...?
## Web Design Overhaul
@@ -55,15 +44,6 @@ associated with each language added, at least a models file or two, as well as s
It would be very helpful to find a speaker of a large language other than English to help in the fine tuning.
## Finalize RSS support (COMPLETED 2024-11)
Marginalia has experimental RSS preview support for a few domains. This works well and
it should be extended to all domains. It would also be interesting to offer search of the
RSS data itself, or use the RSS set to feed a special live index that updates faster than the
main dataset.
Completed with PR [#122](https://github.com/MarginaliaSearch/MarginaliaSearch/pull/122) and PR [#125](https://github.com/MarginaliaSearch/MarginaliaSearch/pull/125)
## Support for binary formats like PDF
The crawler needs to be modified to retain them, and the conversion logic needs to parse them.
@@ -80,5 +60,27 @@ This looks like a good idea that wouldn't just help clean up the search filters
website, but might be cheap enough we might go as far as to offer a number of ad-hoc custom search
filter for any API consumer.
I've talked to the stract dev and he does not think it's a good idea to mimic their optics language,
which is quite ad-hoc, but instead to work together to find some new common description language for this.
I've talked to the stract dev and he does not think it's a good idea to mimic their optics language, which is quite ad-hoc, but instead to work together to find some new common description language for this.
# Completed
## Proper Position Index (COMPLETED 2024-09)
The search engine uses a fixed width bit mask to indicate word positions. It has the benefit
of being very fast to evaluate and works well for what it is, but is inaccurate and has the
drawback of making support for quoted search terms inaccurate and largely reliant on indexing
word n-grams known beforehand. This limits the ability to interpret longer queries.
The positions mask should be supplemented or replaced with a more accurate (e.g.) gamma coded positions
list, as is the civilized way of doing this.
Completed with PR [#99](https://github.com/MarginaliaSearch/MarginaliaSearch/pull/99)
## Finalize RSS support (COMPLETED 2024-11)
Marginalia has experimental RSS preview support for a few domains. This works well and
it should be extended to all domains. It would also be interesting to offer search of the
RSS data itself, or use the RSS set to feed a special live index that updates faster than the
main dataset.
Completed with PR [#122](https://github.com/MarginaliaSearch/MarginaliaSearch/pull/122) and PR [#125](https://github.com/MarginaliaSearch/MarginaliaSearch/pull/125)

View File

@@ -32,7 +32,6 @@ public class GrpcSingleNodeChannelPool<STUB> extends ServiceChangeMonitor {
private final Function<InstanceAddress, ManagedChannel> channelConstructor;
private final Function<ManagedChannel, STUB> stubConstructor;
public GrpcSingleNodeChannelPool(ServiceRegistryIf serviceRegistryIf,
ServiceKey<? extends PartitionTraits.Unicast> serviceKey,
Function<InstanceAddress, ManagedChannel> channelConstructor,
@@ -48,8 +47,6 @@ public class GrpcSingleNodeChannelPool<STUB> extends ServiceChangeMonitor {
serviceRegistryIf.registerMonitor(this);
onChange();
awaitChannel(Duration.ofSeconds(5));
}

View File

@@ -101,6 +101,7 @@ message RpcSimilarDomain {
bool active = 6;
bool screenshot = 7;
LINK_TYPE linkType = 8;
bool feed = 9;
enum LINK_TYPE {
BACKWARD = 0;

View File

@@ -9,6 +9,7 @@ import gnu.trove.map.hash.TIntIntHashMap;
import gnu.trove.set.TIntSet;
import gnu.trove.set.hash.TIntHashSet;
import it.unimi.dsi.fastutil.ints.Int2DoubleArrayMap;
import nu.marginalia.WmsaHome;
import nu.marginalia.api.domains.RpcSimilarDomain;
import nu.marginalia.api.domains.model.SimilarDomain;
import nu.marginalia.api.linkgraph.AggregateLinkGraphClient;
@@ -17,10 +18,14 @@ import org.roaringbitmap.RoaringBitmap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.file.Path;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
@@ -32,12 +37,13 @@ public class SimilarDomainsService {
private final HikariDataSource dataSource;
private final AggregateLinkGraphClient linkGraphClient;
private volatile TIntIntHashMap domainIdToIdx = new TIntIntHashMap(100_000);
private final TIntIntHashMap domainIdToIdx = new TIntIntHashMap(100_000);
private volatile int[] domainIdxToId;
public volatile Int2DoubleArrayMap[] relatedDomains;
public volatile TIntList[] domainNeighbors = null;
public volatile RoaringBitmap screenshotDomains = null;
public volatile RoaringBitmap feedDomains = null;
public volatile RoaringBitmap activeDomains = null;
public volatile RoaringBitmap indexedDomains = null;
public volatile TIntDoubleHashMap domainRanks = null;
@@ -82,6 +88,7 @@ public class SimilarDomainsService {
domainNames = new String[domainIdToIdx.size()];
domainNeighbors = new TIntList[domainIdToIdx.size()];
screenshotDomains = new RoaringBitmap();
feedDomains = new RoaringBitmap();
activeDomains = new RoaringBitmap();
indexedDomains = new RoaringBitmap();
relatedDomains = new Int2DoubleArrayMap[domainIdToIdx.size()];
@@ -145,10 +152,12 @@ public class SimilarDomainsService {
activeDomains.add(idx);
}
updateScreenshotInfo();
logger.info("Loaded {} domains", domainRanks.size());
isReady = true;
// We can defer these as they only populate a roaringbitmap, and will degrade gracefully when not complete
updateScreenshotInfo();
updateFeedInfo();
}
}
catch (SQLException throwables) {
@@ -156,6 +165,42 @@ public class SimilarDomainsService {
}
}
private void updateFeedInfo() {
Set<String> feedsDomainNames = new HashSet<>(500_000);
Path readerDbPath = WmsaHome.getDataPath().resolve("rss-feeds.db").toAbsolutePath();
String dbUrl = "jdbc:sqlite:" + readerDbPath;
logger.info("Opening feed db at " + dbUrl);
try (var conn = DriverManager.getConnection(dbUrl);
var stmt = conn.createStatement()) {
var rs = stmt.executeQuery("""
select
json_extract(feed, '$.domain') as domain
from feed
where json_array_length(feed, '$.items') > 0
""");
while (rs.next()) {
feedsDomainNames.add(rs.getString(1));
}
}
catch (SQLException ex) {
logger.error("Failed to read RSS feed items", ex);
}
for (int idx = 0; idx < domainNames.length; idx++) {
String name = domainNames[idx];
if (name == null) {
continue;
}
if (feedsDomainNames.contains(name)) {
feedDomains.add(idx);
}
}
}
private void updateScreenshotInfo() {
try (var connection = dataSource.getConnection()) {
try (var stmt = connection.createStatement()) {
@@ -254,6 +299,7 @@ public class SimilarDomainsService {
.setIndexed(indexedDomains.contains(idx))
.setActive(activeDomains.contains(idx))
.setScreenshot(screenshotDomains.contains(idx))
.setFeed(feedDomains.contains(idx))
.setLinkType(RpcSimilarDomain.LINK_TYPE.valueOf(linkType.name()))
.build());
@@ -369,6 +415,7 @@ public class SimilarDomainsService {
.setIndexed(indexedDomains.contains(idx))
.setActive(activeDomains.contains(idx))
.setScreenshot(screenshotDomains.contains(idx))
.setFeed(feedDomains.contains(idx))
.setLinkType(RpcSimilarDomain.LINK_TYPE.valueOf(linkType.name()))
.build());

View File

@@ -24,6 +24,7 @@ dependencies {
implementation project(':code:libraries:message-queue')
implementation project(':code:execution:api')
implementation project(':code:processes:crawling-process:ft-content-type')
implementation libs.jsoup
implementation libs.rssreader

View File

@@ -8,6 +8,7 @@ import nu.marginalia.rss.model.FeedDefinition;
import nu.marginalia.rss.model.FeedItems;
import nu.marginalia.service.module.ServiceConfiguration;
import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -127,6 +128,26 @@ public class FeedDb {
return FeedItems.none();
}
@Nullable
public String getEtag(EdgeDomain domain) {
if (!feedDbEnabled) {
throw new IllegalStateException("Feed database is disabled on this node");
}
// Capture the current reader to avoid concurrency issues
FeedDbReader reader = this.reader;
try {
if (reader != null) {
return reader.getEtag(domain);
}
}
catch (Exception e) {
logger.error("Error getting etag for " + domain, e);
}
return null;
}
public Optional<String> getFeedAsJson(String domain) {
if (!feedDbEnabled) {
throw new IllegalStateException("Feed database is disabled on this node");
@@ -214,7 +235,7 @@ public class FeedDb {
public Instant getFetchTime() {
if (!Files.exists(readerDbPath)) {
return Instant.ofEpochMilli(0);
return Instant.EPOCH;
}
try {
@@ -224,7 +245,23 @@ public class FeedDb {
}
catch (IOException ex) {
logger.error("Failed to read the creatiom time of {}", readerDbPath);
return Instant.ofEpochMilli(0);
return Instant.EPOCH;
}
}
public boolean hasData() {
if (!feedDbEnabled) {
throw new IllegalStateException("Feed database is disabled on this node");
}
// Capture the current reader to avoid concurrency issues
FeedDbReader reader = this.reader;
if (reader != null) {
return reader.hasData();
}
return false;
}
}

View File

@@ -8,6 +8,7 @@ import nu.marginalia.rss.model.FeedItems;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.nio.file.Path;
import java.sql.Connection;
import java.sql.DriverManager;
@@ -32,6 +33,7 @@ public class FeedDbReader implements AutoCloseable {
try (var stmt = connection.createStatement()) {
stmt.executeUpdate("CREATE TABLE IF NOT EXISTS feed (domain TEXT PRIMARY KEY, feed JSON)");
stmt.executeUpdate("CREATE TABLE IF NOT EXISTS errors (domain TEXT PRIMARY KEY, cnt INT DEFAULT 0)");
stmt.executeUpdate("CREATE TABLE IF NOT EXISTS etags (domain TEXT PRIMARY KEY, etag TEXT)");
}
}
@@ -106,6 +108,22 @@ public class FeedDbReader implements AutoCloseable {
return FeedItems.none();
}
@Nullable
public String getEtag(EdgeDomain domain) {
try (var stmt = connection.prepareStatement("SELECT etag FROM etags WHERE DOMAIN = ?")) {
stmt.setString(1, domain.toString());
var rs = stmt.executeQuery();
if (rs.next()) {
return rs.getString(1);
}
} catch (SQLException e) {
logger.error("Error getting etag for " + domain, e);
}
return null;
}
private FeedItems deserialize(String string) {
return gson.fromJson(string, FeedItems.class);
}
@@ -141,4 +159,18 @@ public class FeedDbReader implements AutoCloseable {
}
public boolean hasData() {
try (var stmt = connection.prepareStatement("SELECT 1 FROM feed LIMIT 1")) {
var rs = stmt.executeQuery();
if (rs.next()) {
return rs.getBoolean(1);
}
else {
return false;
}
}
catch (SQLException ex) {
return false;
}
}
}

View File

@@ -20,6 +20,7 @@ public class FeedDbWriter implements AutoCloseable {
private final Connection connection;
private final PreparedStatement insertFeedStmt;
private final PreparedStatement insertErrorStmt;
private final PreparedStatement insertEtagStmt;
private final Path dbPath;
private volatile boolean closed = false;
@@ -34,10 +35,12 @@ public class FeedDbWriter implements AutoCloseable {
try (var stmt = connection.createStatement()) {
stmt.executeUpdate("CREATE TABLE IF NOT EXISTS feed (domain TEXT PRIMARY KEY, feed JSON)");
stmt.executeUpdate("CREATE TABLE IF NOT EXISTS errors (domain TEXT PRIMARY KEY, cnt INT DEFAULT 0)");
stmt.executeUpdate("CREATE TABLE IF NOT EXISTS etags (domain TEXT PRIMARY KEY, etag TEXT)");
}
insertFeedStmt = connection.prepareStatement("INSERT INTO feed (domain, feed) VALUES (?, ?)");
insertErrorStmt = connection.prepareStatement("INSERT INTO errors (domain, cnt) VALUES (?, ?)");
insertEtagStmt = connection.prepareStatement("INSERT INTO etags (domain, etag) VALUES (?, ?)");
}
public Path getDbPath() {
@@ -56,6 +59,20 @@ public class FeedDbWriter implements AutoCloseable {
}
}
public synchronized void saveEtag(String domain, String etag) {
if (etag == null || etag.isBlank())
return;
try {
insertEtagStmt.setString(1, domain.toLowerCase());
insertEtagStmt.setString(2, etag);
insertEtagStmt.executeUpdate();
}
catch (SQLException e) {
logger.error("Error saving etag for " + domain, e);
}
}
public synchronized void setErrorCount(String domain, int count) {
try {
insertErrorStmt.setString(1, domain);

View File

@@ -5,6 +5,8 @@ import com.apptasticsoftware.rssreader.RssReader;
import com.google.inject.Inject;
import com.opencsv.CSVReader;
import nu.marginalia.WmsaHome;
import nu.marginalia.contenttype.ContentType;
import nu.marginalia.contenttype.DocumentBodyToString;
import nu.marginalia.executor.client.ExecutorClient;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.nodecfg.NodeConfigurationService;
@@ -32,9 +34,7 @@ import java.net.http.HttpRequest;
import java.net.http.HttpResponse;
import java.nio.charset.StandardCharsets;
import java.sql.SQLException;
import java.time.Duration;
import java.time.LocalDateTime;
import java.time.ZonedDateTime;
import java.time.*;
import java.time.format.DateTimeFormatter;
import java.util.*;
import java.util.concurrent.Executors;
@@ -59,7 +59,6 @@ public class FeedFetcherService {
private final DomainLocks domainLocks = new DomainLocks();
private volatile boolean updating;
private boolean deterministic = false;
@Inject
public FeedFetcherService(FeedDb feedDb,
@@ -91,11 +90,6 @@ public class FeedFetcherService {
REFRESH
};
/** Disable random-based heuristics. This is meant for testing */
public void setDeterministic() {
this.deterministic = true;
}
public void updateFeeds(UpdateMode updateMode) throws IOException {
if (updating) // Prevent concurrent updates
{
@@ -135,37 +129,37 @@ public class FeedFetcherService {
for (var feed : definitions) {
executor.submitQuietly(() -> {
try {
var oldData = feedDb.getFeed(new EdgeDomain(feed.domain()));
EdgeDomain domain = new EdgeDomain(feed.domain());
var oldData = feedDb.getFeed(domain);
// If we have existing data, we might skip updating it with a probability that increases with time,
// this is to avoid hammering the feeds that are updated very rarely and save some time and resources
// on our end
@Nullable
String ifModifiedSinceDate = switch(updateMode) {
case REFRESH -> getIfModifiedSinceDate(feedDb);
case CLEAN -> null;
};
/* Disable for now:
if (!oldData.isEmpty()) {
Duration duration = feed.durationSinceUpdated();
long daysSinceUpdate = duration.toDays();
if (deterministic || (daysSinceUpdate > 2 && ThreadLocalRandom.current()
.nextInt(1, 1 + (int) Math.min(10, daysSinceUpdate) / 2) > 1)) {
// Skip updating this feed, just write the old data back instead
writer.saveFeed(oldData);
return;
}
}
*/
@Nullable
String ifNoneMatchTag = switch (updateMode) {
case REFRESH -> feedDb.getEtag(domain);
case CLEAN -> null;
};
FetchResult feedData;
try (DomainLocks.DomainLock domainLock = domainLocks.lockDomain(new EdgeDomain(feed.domain()))) {
feedData = fetchFeedData(feed, client);
feedData = fetchFeedData(feed, client, ifModifiedSinceDate, ifNoneMatchTag);
} catch (Exception ex) {
feedData = new FetchResult.TransientError();
}
switch (feedData) {
case FetchResult.Success(String value) -> writer.saveFeed(parseFeed(value, feed));
case FetchResult.Success(String value, String etag) -> {
writer.saveEtag(feed.domain(), etag);
writer.saveFeed(parseFeed(value, feed));
}
case FetchResult.NotModified() -> {
writer.saveEtag(feed.domain(), ifNoneMatchTag);
writer.saveFeed(oldData);
}
case FetchResult.TransientError() -> {
int errorCount = errorCounts.getOrDefault(feed.domain().toLowerCase(), 0);
writer.setErrorCount(feed.domain().toLowerCase(), ++errorCount);
@@ -212,30 +206,73 @@ public class FeedFetcherService {
}
}
private FetchResult fetchFeedData(FeedDefinition feed, HttpClient client) {
@Nullable
static String getIfModifiedSinceDate(FeedDb feedDb) {
// If the db is fresh, we don't send If-Modified-Since
if (!feedDb.hasData())
return null;
Instant cutoffInstant = feedDb.getFetchTime();
// If we're unable to establish fetch time, we don't send If-Modified-Since
if (cutoffInstant == Instant.EPOCH)
return null;
return cutoffInstant.atZone(ZoneId.of("GMT")).format(DateTimeFormatter.RFC_1123_DATE_TIME);
}
private FetchResult fetchFeedData(FeedDefinition feed,
HttpClient client,
@Nullable String ifModifiedSinceDate,
@Nullable String ifNoneMatchTag)
{
try {
URI uri = new URI(feed.feedUrl());
HttpRequest getRequest = HttpRequest.newBuilder()
HttpRequest.Builder requestBuilder = HttpRequest.newBuilder()
.GET()
.uri(uri)
.header("User-Agent", WmsaHome.getUserAgent().uaIdentifier())
.header("Accept-Encoding", "gzip")
.header("Accept", "text/*, */*;q=0.9")
.timeout(Duration.ofSeconds(15))
.build();
;
if (ifModifiedSinceDate != null) {
requestBuilder.header("If-Modified-Since", ifModifiedSinceDate);
}
if (ifNoneMatchTag != null) {
requestBuilder.header("If-None-Match", ifNoneMatchTag);
}
HttpRequest getRequest = requestBuilder.build();
for (int i = 0; i < 3; i++) {
var rs = client.send(getRequest, HttpResponse.BodyHandlers.ofString());
if (429 == rs.statusCode()) {
HttpResponse<byte[]> rs = client.send(getRequest, HttpResponse.BodyHandlers.ofByteArray());
if (rs.statusCode() == 429) { // Too Many Requests
int retryAfter = Integer.parseInt(rs.headers().firstValue("Retry-After").orElse("2"));
Thread.sleep(Duration.ofSeconds(Math.clamp(retryAfter, 1, 5)));
} else if (200 == rs.statusCode()) {
return new FetchResult.Success(rs.body());
} else if (404 == rs.statusCode()) {
return new FetchResult.PermanentError(); // never try again
} else {
return new FetchResult.TransientError(); // we try again in a few days
continue;
}
String newEtagValue = rs.headers().firstValue("ETag").orElse("");
return switch (rs.statusCode()) {
case 200 -> {
byte[] responseData = getResponseData(rs);
String contentType = rs.headers().firstValue("Content-Type").orElse("");
String bodyText = DocumentBodyToString.getStringData(ContentType.parse(contentType), responseData);
yield new FetchResult.Success(bodyText, newEtagValue);
}
case 304 -> new FetchResult.NotModified(); // via If-Modified-Since semantics
case 404 -> new FetchResult.PermanentError(); // never try again
default -> new FetchResult.TransientError(); // we try again later
};
}
}
catch (Exception ex) {
@@ -245,8 +282,22 @@ public class FeedFetcherService {
return new FetchResult.TransientError();
}
private byte[] getResponseData(HttpResponse<byte[]> response) throws IOException {
String encoding = response.headers().firstValue("Content-Encoding").orElse("");
if ("gzip".equals(encoding)) {
try (var stream = new GZIPInputStream(new ByteArrayInputStream(response.body()))) {
return stream.readAllBytes();
}
}
else {
return response.body();
}
}
public sealed interface FetchResult {
record Success(String value) implements FetchResult {}
record Success(String value, String etag) implements FetchResult {}
record NotModified() implements FetchResult {}
record TransientError() implements FetchResult {}
record PermanentError() implements FetchResult {}
}
@@ -351,6 +402,7 @@ public class FeedFetcherService {
"&ndash;", "-",
"&rsquo;", "'",
"&lsquo;", "'",
"&quot;", "\"",
"&nbsp;", ""
);

View File

@@ -96,7 +96,6 @@ class FeedFetcherServiceTest extends AbstractModule {
feedDb.switchDb(writer);
}
feedFetcherService.setDeterministic();
feedFetcherService.updateFeeds(FeedFetcherService.UpdateMode.REFRESH);
var result = feedDb.getFeed(new EdgeDomain("www.marginalia.nu"));
@@ -104,6 +103,26 @@ class FeedFetcherServiceTest extends AbstractModule {
Assertions.assertFalse(result.isEmpty());
}
@Tag("flaky")
@Test
public void testFetchRepeatedly() throws Exception {
try (var writer = feedDb.createWriter()) {
writer.saveFeed(new FeedItems("www.marginalia.nu", "https://www.marginalia.nu/log/index.xml", "", List.of()));
feedDb.switchDb(writer);
}
feedFetcherService.updateFeeds(FeedFetcherService.UpdateMode.REFRESH);
Assertions.assertNotNull(feedDb.getEtag(new EdgeDomain("www.marginalia.nu")));
feedFetcherService.updateFeeds(FeedFetcherService.UpdateMode.REFRESH);
Assertions.assertNotNull(feedDb.getEtag(new EdgeDomain("www.marginalia.nu")));
feedFetcherService.updateFeeds(FeedFetcherService.UpdateMode.REFRESH);
Assertions.assertNotNull(feedDb.getEtag(new EdgeDomain("www.marginalia.nu")));
var result = feedDb.getFeed(new EdgeDomain("www.marginalia.nu"));
System.out.println(result);
Assertions.assertFalse(result.isEmpty());
}
@Tag("flaky")
@Test
public void test404() throws Exception {
@@ -112,7 +131,6 @@ class FeedFetcherServiceTest extends AbstractModule {
feedDb.switchDb(writer);
}
feedFetcherService.setDeterministic();
feedFetcherService.updateFeeds(FeedFetcherService.UpdateMode.REFRESH);
// We forget the feed on a 404 error

View File

@@ -10,7 +10,6 @@ public class TestXmlSanitization {
Assertions.assertEquals("&amp;", FeedFetcherService.sanitizeEntities("&amp;"));
Assertions.assertEquals("&lt;", FeedFetcherService.sanitizeEntities("&lt;"));
Assertions.assertEquals("&gt;", FeedFetcherService.sanitizeEntities("&gt;"));
Assertions.assertEquals("&quot;", FeedFetcherService.sanitizeEntities("&quot;"));
Assertions.assertEquals("&apos;", FeedFetcherService.sanitizeEntities("&apos;"));
}
@@ -23,4 +22,9 @@ public class TestXmlSanitization {
public void testTranslatedHtmlEntity() {
Assertions.assertEquals("Foo -- Bar", FeedFetcherService.sanitizeEntities("Foo &mdash; Bar"));
}
@Test
public void testTranslatedHtmlEntityQuot() {
Assertions.assertEquals("\"Bob\"", FeedFetcherService.sanitizeEntities("&quot;Bob&quot;"));
}
}

View File

@@ -9,10 +9,9 @@ import nu.marginalia.service.client.GrpcChannelPoolFactory;
import nu.marginalia.service.client.GrpcSingleNodeChannelPool;
import nu.marginalia.service.discovery.property.ServiceKey;
import nu.marginalia.service.discovery.property.ServicePartition;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.CheckReturnValue;
import java.time.Duration;
@Singleton
public class QueryClient {
@@ -24,13 +23,14 @@ public class QueryClient {
private final GrpcSingleNodeChannelPool<QueryApiGrpc.QueryApiBlockingStub> queryApiPool;
private final Logger logger = LoggerFactory.getLogger(getClass());
@Inject
public QueryClient(GrpcChannelPoolFactory channelPoolFactory) {
public QueryClient(GrpcChannelPoolFactory channelPoolFactory) throws InterruptedException {
this.queryApiPool = channelPoolFactory.createSingle(
ServiceKey.forGrpcApi(QueryApiGrpc.class, ServicePartition.any()),
QueryApiGrpc::newBlockingStub);
// Hold up initialization until we have a downstream connection
this.queryApiPool.awaitChannel(Duration.ofSeconds(5));
}
@CheckReturnValue

View File

@@ -25,6 +25,7 @@ public class QueryExpansion {
this::joinDashes,
this::splitWordNum,
this::joinTerms,
this::categoryKeywords,
this::ngramAll
);
@@ -98,6 +99,24 @@ public class QueryExpansion {
}
}
// Category keyword substitution, e.g. guitar wiki -> guitar generator:wiki
public void categoryKeywords(QWordGraph graph) {
for (var qw : graph) {
// Ensure we only perform the substitution on the last word in the query
if (!graph.getNextOriginal(qw).getFirst().isEnd()) {
continue;
}
switch (qw.word()) {
case "recipe", "recipes" -> graph.addVariant(qw, "category:food");
case "forum" -> graph.addVariant(qw, "generator:forum");
case "wiki" -> graph.addVariant(qw, "generator:wiki");
}
}
}
// Turn 'lawn chair' into 'lawnchair'
public void joinTerms(QWordGraph graph) {
QWord prev = null;

View File

@@ -155,16 +155,25 @@ public class QueryParser {
// Remove trailing punctuation
int lastChar = str.charAt(str.length() - 1);
if (":.,!?$'".indexOf(lastChar) >= 0)
entity.replace(new QueryToken.LiteralTerm(str.substring(0, str.length() - 1), lt.displayStr()));
if (":.,!?$'".indexOf(lastChar) >= 0) {
str = str.substring(0, str.length() - 1);
entity.replace(new QueryToken.LiteralTerm(str, lt.displayStr()));
}
// Remove term elements that aren't indexed by the search engine
if (str.endsWith("'s"))
entity.replace(new QueryToken.LiteralTerm(str.substring(0, str.length() - 2), lt.displayStr()));
if (str.endsWith("()"))
entity.replace(new QueryToken.LiteralTerm(str.substring(0, str.length() - 2), lt.displayStr()));
if (str.startsWith("$"))
entity.replace(new QueryToken.LiteralTerm(str.substring(1), lt.displayStr()));
if (str.endsWith("'s")) {
str = str.substring(0, str.length() - 2);
entity.replace(new QueryToken.LiteralTerm(str, lt.displayStr()));
}
if (str.endsWith("()")) {
str = str.substring(0, str.length() - 2);
entity.replace(new QueryToken.LiteralTerm(str, lt.displayStr()));
}
while (str.startsWith("$") || str.startsWith("_")) {
str = str.substring(1);
entity.replace(new QueryToken.LiteralTerm(str, lt.displayStr()));
}
if (entity.isBlank()) {
entity.remove();

View File

@@ -1,165 +0,0 @@
package nu.marginalia.util.language;
import com.google.inject.Inject;
import nu.marginalia.term_frequency_dict.TermFrequencyDict;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.util.*;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
public class EnglishDictionary {
private final Set<String> englishWords = new HashSet<>();
private final TermFrequencyDict tfDict;
private final Logger logger = LoggerFactory.getLogger(getClass());
@Inject
public EnglishDictionary(TermFrequencyDict tfDict) {
this.tfDict = tfDict;
try (var resource = Objects.requireNonNull(ClassLoader.getSystemResourceAsStream("dictionary/en-words"),
"Could not load word frequency table");
var br = new BufferedReader(new InputStreamReader(resource))
) {
for (;;) {
String s = br.readLine();
if (s == null) {
break;
}
englishWords.add(s.toLowerCase());
}
}
catch (Exception ex) {
throw new RuntimeException(ex);
}
}
public boolean isWord(String word) {
return englishWords.contains(word);
}
private static final Pattern ingPattern = Pattern.compile(".*(\\w)\\1ing$");
public Collection<String> getWordVariants(String s) {
var variants = findWordVariants(s);
var ret = variants.stream()
.filter(var -> tfDict.getTermFreq(var) > 100)
.collect(Collectors.toList());
if (s.equals("recipe") || s.equals("recipes")) {
ret.add("category:food");
}
return ret;
}
public Collection<String> findWordVariants(String s) {
int sl = s.length();
if (sl < 2) {
return Collections.emptyList();
}
if (s.endsWith("s")) {
String a = s.substring(0, sl-1);
String b = s + "es";
if (isWord(a) && isWord(b)) {
return List.of(a, b);
}
else if (isWord(a)) {
return List.of(a);
}
else if (isWord(b)) {
return List.of(b);
}
}
if (s.endsWith("sm")) {
String a = s.substring(0, sl-1)+"t";
String b = s.substring(0, sl-1)+"ts";
if (isWord(a) && isWord(b)) {
return List.of(a, b);
}
else if (isWord(a)) {
return List.of(a);
}
else if (isWord(b)) {
return List.of(b);
}
}
if (s.endsWith("st")) {
String a = s.substring(0, sl-1)+"m";
String b = s + "s";
if (isWord(a) && isWord(b)) {
return List.of(a, b);
}
else if (isWord(a)) {
return List.of(a);
}
else if (isWord(b)) {
return List.of(b);
}
}
else if (ingPattern.matcher(s).matches() && sl > 4) { // humming, clapping
var a = s.substring(0, sl-4);
var b = s.substring(0, sl-3) + "ed";
if (isWord(a) && isWord(b)) {
return List.of(a, b);
}
else if (isWord(a)) {
return List.of(a);
}
else if (isWord(b)) {
return List.of(b);
}
}
else {
String a = s + "s";
String b = ingForm(s);
String c = s + "ed";
if (isWord(a) && isWord(b) && isWord(c)) {
return List.of(a, b, c);
}
else if (isWord(a) && isWord(b)) {
return List.of(a, b);
}
else if (isWord(b) && isWord(c)) {
return List.of(b, c);
}
else if (isWord(a) && isWord(c)) {
return List.of(a, c);
}
else if (isWord(a)) {
return List.of(a);
}
else if (isWord(b)) {
return List.of(b);
}
else if (isWord(c)) {
return List.of(c);
}
}
return Collections.emptyList();
}
public String ingForm(String s) {
if (s.endsWith("t") && !s.endsWith("tt")) {
return s + "ting";
}
if (s.endsWith("n") && !s.endsWith("nn")) {
return s + "ning";
}
if (s.endsWith("m") && !s.endsWith("mm")) {
return s + "ming";
}
if (s.endsWith("r") && !s.endsWith("rr")) {
return s + "ring";
}
return s + "ing";
}
}

View File

@@ -0,0 +1,32 @@
package nu.marginalia.functions.searchquery.query_parser;
import nu.marginalia.functions.searchquery.query_parser.token.QueryToken;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.util.List;
class QueryParserTest {
@Test
// https://github.com/MarginaliaSearch/MarginaliaSearch/issues/140
void parse__builtin_ffs() {
QueryParser parser = new QueryParser();
var tokens = parser.parse("__builtin_ffs");
Assertions.assertEquals(List.of(new QueryToken.LiteralTerm("builtin_ffs", "__builtin_ffs")), tokens);
}
@Test
void trailingParens() {
QueryParser parser = new QueryParser();
var tokens = parser.parse("strcpy()");
Assertions.assertEquals(List.of(new QueryToken.LiteralTerm("strcpy", "strcpy()")), tokens);
}
@Test
void trailingQuote() {
QueryParser parser = new QueryParser();
var tokens = parser.parse("bob's");
Assertions.assertEquals(List.of(new QueryToken.LiteralTerm("bob", "bob's")), tokens);
}
}

View File

@@ -12,6 +12,7 @@ import nu.marginalia.index.query.limit.SpecificationLimit;
import nu.marginalia.index.query.limit.SpecificationLimitType;
import nu.marginalia.segmentation.NgramLexicon;
import nu.marginalia.term_frequency_dict.TermFrequencyDict;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
@@ -207,6 +208,17 @@ public class QueryFactoryTest {
System.out.println(subquery);
}
@Test
public void testExpansion9() {
var subquery = parseAndGetSpecs("pie recipe");
Assertions.assertTrue(subquery.query.compiledQuery.contains(" category:food "));
subquery = parseAndGetSpecs("recipe pie");
Assertions.assertFalse(subquery.query.compiledQuery.contains(" category:food "));
}
@Test
public void testParsing() {
var subquery = parseAndGetSpecs("strlen()");

View File

@@ -25,12 +25,11 @@ public class ProcessedDocumentDetails {
public List<EdgeUrl> linksInternal;
public List<EdgeUrl> linksExternal;
public List<EdgeUrl> feedLinks;
public DocumentMetadata metadata;
public GeneratorType generator;
public String toString() {
return "ProcessedDocumentDetails(title=" + this.title + ", description=" + this.description + ", pubYear=" + this.pubYear + ", length=" + this.length + ", quality=" + this.quality + ", hashCode=" + this.hashCode + ", features=" + this.features + ", standard=" + this.standard + ", linksInternal=" + this.linksInternal + ", linksExternal=" + this.linksExternal + ", feedLinks=" + this.feedLinks + ", metadata=" + this.metadata + ", generator=" + this.generator + ")";
return "ProcessedDocumentDetails(title=" + this.title + ", description=" + this.description + ", pubYear=" + this.pubYear + ", length=" + this.length + ", quality=" + this.quality + ", hashCode=" + this.hashCode + ", features=" + this.features + ", standard=" + this.standard + ", linksInternal=" + this.linksInternal + ", linksExternal=" + this.linksExternal + ", metadata=" + this.metadata + ", generator=" + this.generator + ")";
}
}

View File

@@ -34,7 +34,6 @@ public class LinkProcessor {
ret.linksExternal = new ArrayList<>();
ret.linksInternal = new ArrayList<>();
ret.feedLinks = new ArrayList<>();
}
public Set<EdgeUrl> getSeenUrls() {
@@ -72,19 +71,6 @@ public class LinkProcessor {
}
}
/** Accepts a link as a feed link */
public void acceptFeed(EdgeUrl link) {
if (!isLinkPermitted(link)) {
return;
}
if (!seenUrls.add(link)) {
return;
}
ret.feedLinks.add(link);
}
private boolean isLinkPermitted(EdgeUrl link) {
if (!permittedSchemas.contains(link.proto.toLowerCase())) {
return false;

View File

@@ -294,11 +294,6 @@ public class HtmlDocumentProcessorPlugin extends AbstractDocumentProcessorPlugin
for (var meta : doc.select("meta[http-equiv=refresh]")) {
linkParser.parseMetaRedirect(baseUrl, meta).ifPresent(lp::accept);
}
for (var link : doc.select("link[rel=alternate]")) {
feedExtractor
.getFeedFromAlternateTag(baseUrl, link)
.ifPresent(lp::acceptFeed);
}
words.addAllSyntheticTerms(FileLinks.createFileLinkKeywords(lp, domain));
words.addAllSyntheticTerms(FileLinks.createFileEndingKeywords(doc));

View File

@@ -125,7 +125,6 @@ public class PlainTextDocumentProcessorPlugin extends AbstractDocumentProcessorP
/* These are assumed to be populated */
ret.linksInternal = new ArrayList<>();
ret.linksExternal = new ArrayList<>();
ret.feedLinks = new ArrayList<>();
return new DetailsWithWords(ret, words);
}

View File

@@ -166,7 +166,6 @@ public class StackexchangeSideloader implements SideloadSource {
ret.details.length = 128;
ret.details.standard = HtmlStandard.HTML5;
ret.details.feedLinks = List.of();
ret.details.linksExternal = List.of();
ret.details.linksInternal = List.of();
ret.state = UrlIndexingState.OK;

View File

@@ -178,7 +178,6 @@ public class ConverterBatchWriter implements AutoCloseable, ConverterBatchWriter
public void writeDomainData(ProcessedDomain domain) throws IOException {
DomainMetadata metadata = DomainMetadata.from(domain);
List<String> feeds = getFeedUrls(domain);
domainWriter.write(
new SlopDomainRecord(
@@ -188,25 +187,11 @@ public class ConverterBatchWriter implements AutoCloseable, ConverterBatchWriter
metadata.visited(),
Optional.ofNullable(domain.state).map(DomainIndexingState::toString).orElse(""),
Optional.ofNullable(domain.redirect).map(EdgeDomain::toString).orElse(""),
domain.ip,
feeds
domain.ip
)
);
}
private List<String> getFeedUrls(ProcessedDomain domain) {
var documents = domain.documents;
if (documents == null)
return List.of();
return documents.stream().map(doc -> doc.details)
.filter(Objects::nonNull)
.flatMap(dets -> dets.feedLinks.stream())
.distinct()
.map(EdgeUrl::toString)
.toList();
}
public void close() throws IOException {
domainWriter.close();
documentWriter.close();

View File

@@ -1,7 +1,6 @@
package nu.marginalia.model.processed;
import nu.marginalia.slop.SlopTable;
import nu.marginalia.slop.column.array.ObjectArrayColumn;
import nu.marginalia.slop.column.primitive.IntColumn;
import nu.marginalia.slop.column.string.EnumColumn;
import nu.marginalia.slop.column.string.TxtStringColumn;
@@ -10,7 +9,6 @@ import nu.marginalia.slop.desc.StorageType;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
import java.util.List;
import java.util.function.Consumer;
public record SlopDomainRecord(
@@ -20,8 +18,7 @@ public record SlopDomainRecord(
int visitedUrls,
String state,
String redirectDomain,
String ip,
List<String> rssFeeds)
String ip)
{
public record DomainWithIpProjection(
@@ -38,9 +35,6 @@ public record SlopDomainRecord(
private static final IntColumn goodUrlsColumn = new IntColumn("goodUrls", StorageType.PLAIN);
private static final IntColumn visitedUrlsColumn = new IntColumn("visitedUrls", StorageType.PLAIN);
private static final ObjectArrayColumn<String> rssFeedsColumn = new TxtStringColumn("rssFeeds", StandardCharsets.UTF_8, StorageType.GZIP).asArray();
public static class DomainNameReader extends SlopTable {
private final TxtStringColumn.Reader domainsReader;
@@ -101,8 +95,6 @@ public record SlopDomainRecord(
private final IntColumn.Reader goodUrlsReader;
private final IntColumn.Reader visitedUrlsReader;
private final ObjectArrayColumn<String>.Reader rssFeedsReader;
public Reader(SlopTable.Ref<SlopDomainRecord> ref) throws IOException {
super(ref);
@@ -114,8 +106,6 @@ public record SlopDomainRecord(
knownUrlsReader = knownUrlsColumn.open(this);
goodUrlsReader = goodUrlsColumn.open(this);
visitedUrlsReader = visitedUrlsColumn.open(this);
rssFeedsReader = rssFeedsColumn.open(this);
}
public Reader(Path baseDir, int page) throws IOException {
@@ -140,8 +130,7 @@ public record SlopDomainRecord(
visitedUrlsReader.get(),
statesReader.get(),
redirectReader.get(),
ipReader.get(),
rssFeedsReader.get()
ipReader.get()
);
}
}
@@ -156,8 +145,6 @@ public record SlopDomainRecord(
private final IntColumn.Writer goodUrlsWriter;
private final IntColumn.Writer visitedUrlsWriter;
private final ObjectArrayColumn<String>.Writer rssFeedsWriter;
public Writer(Path baseDir, int page) throws IOException {
super(baseDir, page);
@@ -169,8 +156,6 @@ public record SlopDomainRecord(
knownUrlsWriter = knownUrlsColumn.create(this);
goodUrlsWriter = goodUrlsColumn.create(this);
visitedUrlsWriter = visitedUrlsColumn.create(this);
rssFeedsWriter = rssFeedsColumn.create(this);
}
public void write(SlopDomainRecord record) throws IOException {
@@ -182,8 +167,6 @@ public record SlopDomainRecord(
knownUrlsWriter.put(record.knownUrls());
goodUrlsWriter.put(record.goodUrls());
visitedUrlsWriter.put(record.visitedUrls());
rssFeedsWriter.put(record.rssFeeds());
}
}
}

View File

@@ -9,7 +9,6 @@ import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -35,8 +34,7 @@ public class SlopDomainRecordTest {
1, 2, 3,
"state",
"redirectDomain",
"192.168.0.1",
List.of("rss1", "rss2")
"192.168.0.1"
);
try (var writer = new SlopDomainRecord.Writer(testDir, 0)) {

View File

@@ -20,34 +20,11 @@ public record ContentTags(String etag, String lastMod) {
public void paint(Request.Builder getBuilder) {
if (etag != null) {
getBuilder.addHeader("If-None-Match", ifNoneMatch());
getBuilder.addHeader("If-None-Match", etag);
}
if (lastMod != null) {
getBuilder.addHeader("If-Modified-Since", ifModifiedSince());
getBuilder.addHeader("If-Modified-Since", lastMod);
}
}
private String ifNoneMatch() {
// Remove the W/ prefix if it exists
//'W/' (case-sensitive) indicates that a weak validator is used. Weak etags are
// easy to generate, but are far less useful for comparisons. Strong validators
// are ideal for comparisons but can be very difficult to generate efficiently.
// Weak ETag values of two representations of the same resources might be semantically
// equivalent, but not byte-for-byte identical. This means weak etags prevent caching
// when byte range requests are used, but strong etags mean range requests can
// still be cached.
// - https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/ETag
if (null != etag && etag.startsWith("W/")) {
return etag.substring(2);
} else {
return etag;
}
}
private String ifModifiedSince() {
return lastMod;
}
}

View File

@@ -34,8 +34,9 @@ import java.util.*;
public class WarcRecorder implements AutoCloseable {
/** Maximum time we'll wait on a single request */
static final int MAX_TIME = 30_000;
/** Maximum (decompressed) size we'll fetch */
static final int MAX_SIZE = 1024 * 1024 * 10;
/** Maximum (decompressed) size we'll save */
static final int MAX_SIZE = Integer.getInteger("crawler.maxFetchSize", 10 * 1024 * 1024);
private final WarcWriter writer;
private final Path warcFile;

View File

@@ -297,16 +297,16 @@ public class CrawlerRetreiver implements AutoCloseable {
}
private final List<String> likelyFeedEndpoints = List.of(
"/rss.xml",
"/atom.xml",
"/feed.xml",
"/index.xml",
"/feed",
"/rss",
"/atom",
"/feeds",
"/blog/feed",
"/blog/rss"
"rss.xml",
"atom.xml",
"feed.xml",
"index.xml",
"feed",
"rss",
"atom",
"feeds",
"blog/feed",
"blog/rss"
);
private Optional<String> guessFeedUrl(CrawlDelayTimer timer) throws InterruptedException {

View File

@@ -1,11 +1,15 @@
package nu.marginalia.io;
import nu.marginalia.model.crawldata.CrawledDocument;
import nu.marginalia.model.crawldata.CrawledDomain;
import nu.marginalia.model.crawldata.SerializableCrawlData;
import org.jetbrains.annotations.Nullable;
import java.io.IOException;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
/** Closable iterator exceptional over serialized crawl data
* The data may appear in any order, and the iterator must be closed.
@@ -26,6 +30,37 @@ public interface SerializableCrawlDataStream extends AutoCloseable {
@Nullable
default Path path() { return null; }
/** For tests */
default List<SerializableCrawlData> asList() throws IOException {
List<SerializableCrawlData> data = new ArrayList<>();
while (hasNext()) {
data.add(next());
}
return data;
}
/** For tests */
default List<CrawledDocument> docsAsList() throws IOException {
List<CrawledDocument> data = new ArrayList<>();
while (hasNext()) {
if (next() instanceof CrawledDocument doc) {
data.add(doc);
}
}
return data;
}
/** For tests */
default List<CrawledDomain> domainsAsList() throws IOException {
List<CrawledDomain> data = new ArrayList<>();
while (hasNext()) {
if (next() instanceof CrawledDomain domain) {
data.add(domain);
}
}
return data;
}
// Dummy iterator over nothing
static SerializableCrawlDataStream empty() {
return new SerializableCrawlDataStream() {

View File

@@ -179,6 +179,9 @@ public class LiveCrawlerMain extends ProcessMainClass {
EdgeDomain domain = new EdgeDomain(entry.getKey());
List<String> urls = entry.getValue();
if (urls.isEmpty())
continue;
fetcher.scheduleRetrieval(domain, urls);
}
}

View File

@@ -3,6 +3,8 @@ package nu.marginalia.livecrawler;
import crawlercommons.robots.SimpleRobotRules;
import crawlercommons.robots.SimpleRobotRulesParser;
import nu.marginalia.WmsaHome;
import nu.marginalia.contenttype.ContentType;
import nu.marginalia.contenttype.DocumentBodyToString;
import nu.marginalia.crawl.fetcher.HttpFetcherImpl;
import nu.marginalia.crawl.logic.DomainLocks;
import nu.marginalia.crawl.retreival.CrawlDelayTimer;
@@ -16,6 +18,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.Nullable;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.net.URISyntaxException;
import java.net.http.HttpClient;
@@ -23,10 +26,12 @@ import java.net.http.HttpHeaders;
import java.net.http.HttpRequest;
import java.net.http.HttpResponse;
import java.time.Duration;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.zip.GZIPInputStream;
/** A simple link scraper that fetches URLs and stores them in a database,
* with no concept of a crawl frontier, WARC output, or other advanced features
@@ -43,6 +48,8 @@ public class SimpleLinkScraper implements AutoCloseable {
private final Duration readTimeout = Duration.ofSeconds(10);
private final DomainLocks domainLocks = new DomainLocks();
private final static int MAX_SIZE = Integer.getInteger("crawler.maxFetchSize", 10 * 1024 * 1024);
public SimpleLinkScraper(LiveCrawlDataSet dataSet,
DbDomainQueries domainQueries,
DomainBlacklist domainBlacklist) {
@@ -61,52 +68,68 @@ public class SimpleLinkScraper implements AutoCloseable {
pool.submitQuietly(() -> retrieveNow(domain, id.getAsInt(), urls));
}
public void retrieveNow(EdgeDomain domain, int domainId, List<String> urls) throws Exception {
public int retrieveNow(EdgeDomain domain, int domainId, List<String> urls) throws Exception {
EdgeUrl rootUrl = domain.toRootUrlHttps();
List<EdgeUrl> relevantUrls = new ArrayList<>();
for (var url : urls) {
Optional<EdgeUrl> optParsedUrl = lp.parseLink(rootUrl, url);
if (optParsedUrl.isEmpty()) {
continue;
}
if (dataSet.hasUrl(optParsedUrl.get())) {
continue;
}
relevantUrls.add(optParsedUrl.get());
}
if (relevantUrls.isEmpty()) {
return 0;
}
int fetched = 0;
try (HttpClient client = HttpClient
.newBuilder()
.connectTimeout(connectTimeout)
.followRedirects(HttpClient.Redirect.NEVER)
.version(HttpClient.Version.HTTP_2)
.build();
DomainLocks.DomainLock lock = domainLocks.lockDomain(domain) // throttle concurrent access per domain; do not remove
// throttle concurrent access per domain; IDE will complain it's not used, but it holds a semaphore -- do not remove:
DomainLocks.DomainLock lock = domainLocks.lockDomain(domain)
) {
EdgeUrl rootUrl = domain.toRootUrlHttps();
SimpleRobotRules rules = fetchRobotsRules(rootUrl, client);
if (rules == null) { // I/O error fetching robots.txt
// If we can't fetch the robots.txt,
for (var url : urls) {
lp.parseLink(rootUrl, url).ifPresent(this::maybeFlagAsBad);
for (var url : relevantUrls) {
maybeFlagAsBad(url);
}
return;
return fetched;
}
CrawlDelayTimer timer = new CrawlDelayTimer(rules.getCrawlDelay());
for (var url : urls) {
Optional<EdgeUrl> optParsedUrl = lp.parseLink(rootUrl, url);
if (optParsedUrl.isEmpty()) {
continue;
}
if (dataSet.hasUrl(optParsedUrl.get())) {
continue;
}
for (var parsedUrl : relevantUrls) {
EdgeUrl parsedUrl = optParsedUrl.get();
if (!rules.isAllowed(url)) {
if (!rules.isAllowed(parsedUrl.toString())) {
maybeFlagAsBad(parsedUrl);
continue;
}
switch (fetchUrl(domainId, parsedUrl, timer, client)) {
case FetchResult.Success(int id, EdgeUrl docUrl, String body, String headers)
-> dataSet.saveDocument(id, docUrl, body, headers, "");
case FetchResult.Success(int id, EdgeUrl docUrl, String body, String headers) -> {
dataSet.saveDocument(id, docUrl, body, headers, "");
fetched++;
}
case FetchResult.Error(EdgeUrl docUrl) -> maybeFlagAsBad(docUrl);
}
}
}
return fetched;
}
private void maybeFlagAsBad(EdgeUrl url) {
@@ -128,6 +151,7 @@ public class SimpleLinkScraper implements AutoCloseable {
var robotsRequest = HttpRequest.newBuilder(rootUrl.withPathAndParam("/robots.txt", null).asURI())
.GET()
.header("User-Agent", WmsaHome.getUserAgent().uaString())
.header("Accept-Encoding","gzip")
.timeout(readTimeout);
// Fetch the robots.txt
@@ -135,9 +159,10 @@ public class SimpleLinkScraper implements AutoCloseable {
try {
SimpleRobotRulesParser parser = new SimpleRobotRulesParser();
HttpResponse<byte[]> robotsTxt = client.send(robotsRequest.build(), HttpResponse.BodyHandlers.ofByteArray());
if (robotsTxt.statusCode() == 200) {
return parser.parseContent(rootUrl.toString(),
robotsTxt.body(),
getResponseData(robotsTxt),
robotsTxt.headers().firstValue("Content-Type").orElse("text/plain"),
WmsaHome.getUserAgent().uaIdentifier());
}
@@ -161,18 +186,19 @@ public class SimpleLinkScraper implements AutoCloseable {
.GET()
.header("User-Agent", WmsaHome.getUserAgent().uaString())
.header("Accept", "text/html")
.header("Accept-Encoding", "gzip")
.timeout(readTimeout)
.build();
try {
HttpResponse<String> response = client.send(request, HttpResponse.BodyHandlers.ofString());
HttpResponse<byte[]> response = client.send(request, HttpResponse.BodyHandlers.ofByteArray());
// Handle rate limiting by waiting and retrying once
if (response.statusCode() == 429) {
timer.waitRetryDelay(new HttpFetcherImpl.RateLimitException(
response.headers().firstValue("Retry-After").orElse("5")
));
response = client.send(request, HttpResponse.BodyHandlers.ofString());
response = client.send(request, HttpResponse.BodyHandlers.ofByteArray());
}
String contentType = response.headers().firstValue("Content-Type").orElse("").toLowerCase();
@@ -182,12 +208,14 @@ public class SimpleLinkScraper implements AutoCloseable {
return new FetchResult.Error(parsedUrl);
}
String body = response.body();
if (body.length() > 1024 * 1024) {
byte[] body = getResponseData(response);
if (body.length > MAX_SIZE) {
return new FetchResult.Error(parsedUrl);
}
return new FetchResult.Success(domainId, parsedUrl, body, headersToString(response.headers()));
String bodyText = DocumentBodyToString.getStringData(ContentType.parse(contentType), body);
return new FetchResult.Success(domainId, parsedUrl, bodyText, headersToString(response.headers()));
}
}
catch (IOException ex) {
@@ -198,6 +226,19 @@ public class SimpleLinkScraper implements AutoCloseable {
return new FetchResult.Error(parsedUrl);
}
private byte[] getResponseData(HttpResponse<byte[]> response) throws IOException {
String encoding = response.headers().firstValue("Content-Encoding").orElse("");
if ("gzip".equals(encoding)) {
try (var stream = new GZIPInputStream(new ByteArrayInputStream(response.body()))) {
return stream.readAllBytes();
}
}
else {
return response.body();
}
}
sealed interface FetchResult {
record Success(int domainId, EdgeUrl url, String body, String headers) implements FetchResult {}
record Error(EdgeUrl url) implements FetchResult {}

View File

@@ -0,0 +1,66 @@
package nu.marginalia.livecrawler;
import nu.marginalia.db.DomainBlacklistImpl;
import nu.marginalia.io.SerializableCrawlDataStream;
import nu.marginalia.model.EdgeDomain;
import nu.marginalia.model.EdgeUrl;
import nu.marginalia.model.crawldata.CrawledDocument;
import org.apache.commons.io.FileUtils;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mockito;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.sql.SQLException;
import java.util.List;
class SimpleLinkScraperTest {
private Path tempDir;
private LiveCrawlDataSet dataSet;
@BeforeEach
public void setUp() throws IOException, SQLException {
tempDir = Files.createTempDirectory(getClass().getSimpleName());
dataSet = new LiveCrawlDataSet(tempDir);
}
@AfterEach
public void tearDown() throws Exception {
dataSet.close();
FileUtils.deleteDirectory(tempDir.toFile());
}
@Test
public void testRetrieveNow() throws Exception {
var scraper = new SimpleLinkScraper(dataSet, null, Mockito.mock(DomainBlacklistImpl.class));
int fetched = scraper.retrieveNow(new EdgeDomain("www.marginalia.nu"), 1, List.of("https://www.marginalia.nu/"));
Assertions.assertEquals(1, fetched);
var streams = dataSet.getDataStreams();
Assertions.assertEquals(1, streams.size());
SerializableCrawlDataStream firstStream = streams.iterator().next();
Assertions.assertTrue(firstStream.hasNext());
List<CrawledDocument> documents = firstStream.docsAsList();
Assertions.assertEquals(1, documents.size());
Assertions.assertTrue(documents.getFirst().documentBody.startsWith("<!doctype"));
}
@Test
public void testRetrieveNow_Redundant() throws Exception {
dataSet.saveDocument(1, new EdgeUrl("https://www.marginalia.nu/"), "<html>", "", "127.0.0.1");
var scraper = new SimpleLinkScraper(dataSet, null, Mockito.mock(DomainBlacklistImpl.class));
// If the requested URL is already in the dataSet, we retrieveNow should shortcircuit and not fetch anything
int fetched = scraper.retrieveNow(new EdgeDomain("www.marginalia.nu"), 1, List.of("https://www.marginalia.nu/"));
Assertions.assertEquals(0, fetched);
}
}

View File

@@ -0,0 +1,14 @@
<section id="frontpage-tips">
<h2>Public Beta Available</h2>
<div class="info">
<p>
A redesigned version of the search engine UI is available for beta testing.
Feel free to give it a spin, feedback is welcome!
The old one will also be keep being available if you hate it,
or have compatibility issues.
</p>
<p>
<a href="https://test.marginalia.nu/">Try it out!</a>
</p>
</div>
</section>

View File

@@ -24,7 +24,7 @@
<section id="frontpage">
{{>search/index/index-news}}
{{>search/index/index-about}}
{{>search/index/index-tips}}
{{>search/index/index-redesign}}
</section>
{{>search/parts/search-footer}}

View File

@@ -1,39 +1,19 @@
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;
public SiteStatisticsExperiment() {
}
@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;
}

34
tools/deployment/deployment.py Normal file → Executable file
View File

@@ -1,3 +1,5 @@
#!/usr/bin/env python3
from dataclasses import dataclass
import subprocess, os
from typing import List, Set, Dict, Optional
@@ -220,6 +222,31 @@ def run_gradle_build(targets: str) -> None:
if return_code != 0:
raise BuildError(service, return_code)
def find_free_tag() -> str:
cmd = ['git', 'tag']
result = subprocess.run(cmd, capture_output=True, text=True)
if result.returncode != 0:
raise RuntimeError(f"Git command failed: {result.stderr}")
existing_tags = set(result.stdout.splitlines())
for i in range(1, 100000):
tag = f'deploy-{i:04d}'
if not tag in existing_tags:
return tag
raise RuntimeError(f"Failed to find a free deployment tag")
def add_tags(tags: str) -> None:
new_tag = find_free_tag()
cmd = ['git', 'tag', new_tag, '-am', tags]
result = subprocess.run(cmd)
if result.returncode != 0:
raise RuntimeError(f"Git command failed: {result.stderr}")
# Example usage:
if __name__ == '__main__':
# Define service configuration
@@ -293,7 +320,9 @@ if __name__ == '__main__':
parser = argparse.ArgumentParser(
prog='deployment.py',
description='Continuous Deployment helper')
parser.add_argument('-v', '--verify', help='Verify the tags are valid, if present', action='store_true')
parser.add_argument('-a', '--add', help='Add the tags provided as a new deployment tag, usually combined with -t', action='store_true')
parser.add_argument('-t', '--tag', help='Use the specified tag value instead of the head git tag starting with deploy-')
args = parser.parse_args()
@@ -314,7 +343,10 @@ if __name__ == '__main__':
print("Services to build:", plan.services_to_build)
print("Instances to deploy:", [container.name for container in plan.instances_to_deploy])
if not args.verify:
if args.verify:
if args.add:
add_tags(args.tag)
else:
print("\nExecution Plan:")
build_and_deploy(plan, SERVICE_CONFIG)