replaced check for load time method

instead of loading the solr document, an index only for the last loading
time was created. This prevents that solr has to fetch from its index
while the index is created. Excessive re-loading of documents while
indexing has shown to produce deadlocks, so this should now be
prevented.
pull/436/head
Michael Peter Christen 3 years ago
parent 1ead7b85b5
commit 163ba26d90

@ -58,14 +58,14 @@ public class IndexControlURLs_p {
final Switchboard sb = (Switchboard) env;
final serverObjects prop = new serverObjects();
/* Acquire a transaction token for the next possible POST form submissions */
final String nextTransactionToken = TransactionManager.getTransactionToken(header);
prop.put(TransactionManager.TRANSACTION_TOKEN_PARAM, nextTransactionToken);
Segment segment = sb.index;
long ucount = segment.fulltext().collectionSize();
// set default values
prop.put("urlstring", "");
prop.put("urlhash", "");
@ -82,16 +82,16 @@ public class IndexControlURLs_p {
List<File> dumpFiles = segment.fulltext().dumpFiles();
prop.put("dumprestore_dumpfile", dumpFiles.size() == 0 ? "" : dumpFiles.get(dumpFiles.size() - 1).getAbsolutePath());
prop.put("dumprestore_optimizemax", 10);
prop.put("dumprestore_rebootSolrEnabled",
sb.getConfigBool(SwitchboardConstants.CORE_SERVICE_FULLTEXT,
SwitchboardConstants.CORE_SERVICE_FULLTEXT_DEFAULT)
&& !sb.getConfigBool(SwitchboardConstants.FEDERATED_SERVICE_SOLR_INDEXING_ENABLED,
SwitchboardConstants.FEDERATED_SERVICE_SOLR_INDEXING_ENABLED_DEFAULT));
prop.put("dumprestore_rebootSolrEnabled",
sb.getConfigBool(SwitchboardConstants.CORE_SERVICE_FULLTEXT,
SwitchboardConstants.CORE_SERVICE_FULLTEXT_DEFAULT)
&& !sb.getConfigBool(SwitchboardConstants.FEDERATED_SERVICE_SOLR_INDEXING_ENABLED,
SwitchboardConstants.FEDERATED_SERVICE_SOLR_INDEXING_ENABLED_DEFAULT));
prop.put("cleanup", ucount == 0 ? 0 : 1);
prop.put("cleanupsolr", segment.fulltext().connectedRemoteSolr() ? 1 : 0);
prop.put("cleanuprwi", segment.termIndex() != null && !segment.termIndex().isEmpty() ? 1 : 0);
prop.put("cleanupcitation", segment.connectedCitation() && !segment.urlCitation().isEmpty() ? 1 : 0);
if (post == null || env == null) {
prop.putNum("ucount", ucount);
return prop; // nothing to do
@ -119,14 +119,20 @@ public class IndexControlURLs_p {
// delete everything
if ( post.containsKey("deletecomplete") ) {
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
if ( post.get("deleteIndex", "").equals("on") ) {
try {segment.fulltext().clearLocalSolr();} catch (final IOException e) {}
try {
segment.fulltext().clearLocalSolr();
segment.loadTimeIndex().clear();
} catch (final IOException e) {}
}
if ( post.get("deleteRemoteSolr", "").equals("on")) {
try {segment.fulltext().clearRemoteSolr();} catch (final IOException e) {}
try {
segment.fulltext().clearRemoteSolr();
segment.loadTimeIndex().clear();
} catch (final IOException e) {}
}
if ( post.get("deleteRWI", "").equals("on")) {
if (segment.termIndex() != null) try {segment.termIndex().clear();} catch (final IOException e) {}
@ -135,7 +141,10 @@ public class IndexControlURLs_p {
if (segment.connectedCitation()) try {segment.urlCitation().clear();} catch (final IOException e) {}
}
if ( post.get("deleteFirstSeen", "").equals("on")) {
try {segment.firstSeen().clear();} catch (final IOException e) {}
try {
segment.firstSeenIndex().clear();
segment.loadTimeIndex().clear();
} catch (final IOException e) {}
}
if ( post.get("deleteCrawlQueues", "").equals("on") ) {
sb.crawlQueues.clear();
@ -152,18 +161,18 @@ public class IndexControlURLs_p {
}
if (post.containsKey("urlhashdeleteall")) {
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
ClientIdentification.Agent agent = ClientIdentification.getAgent(post.get("agentName", ClientIdentification.yacyInternetCrawlerAgentName));
int i = segment.removeAllUrlReferences(urlhash.getBytes(), sb.loader, agent, CacheStrategy.IFEXIST);
prop.put("result", "Deleted URL and " + i + " references from " + i + " word indexes.");
}
if (post.containsKey("urlhashdelete")) {
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
DigestURL url;
try {
url = segment.fulltext().getURL(urlhash);
@ -181,9 +190,9 @@ public class IndexControlURLs_p {
}
if (post.containsKey("urldelete")) {
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
try {
urlhash = ASCII.String((new DigestURL(urlstring)).hash());
} catch (final MalformedURLException e) {
@ -227,34 +236,34 @@ public class IndexControlURLs_p {
prop.put("statistics", 0);
}
}
if (post.containsKey("optimizesolr")) {
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
final int size = post.getInt("optimizemax", 10);
segment.fulltext().optimize(size);
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
final int size = post.getInt("optimizemax", 10);
segment.fulltext().optimize(size);
sb.tables.recordAPICall(post, "IndexControlURLs_p.html", WorkTables.TABLE_API_TYPE_STEERING, "solr optimize " + size);
}
if (post.containsKey("rebootsolr")) {
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
if (sb.getConfigBool(SwitchboardConstants.CORE_SERVICE_FULLTEXT,
SwitchboardConstants.CORE_SERVICE_FULLTEXT_DEFAULT)
&& !sb.getConfigBool(SwitchboardConstants.FEDERATED_SERVICE_SOLR_INDEXING_ENABLED,
SwitchboardConstants.FEDERATED_SERVICE_SOLR_INDEXING_ENABLED_DEFAULT)) {
/* This operation is designed only for an embdded local Solr with no mirroring to an external remote Solr server */
segment.fulltext().rebootEmbeddedLocalSolr();
sb.tables.recordAPICall(post, "IndexControlURLs_p.html", WorkTables.TABLE_API_TYPE_STEERING, "solr reboot");
}
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
if (sb.getConfigBool(SwitchboardConstants.CORE_SERVICE_FULLTEXT,
SwitchboardConstants.CORE_SERVICE_FULLTEXT_DEFAULT)
&& !sb.getConfigBool(SwitchboardConstants.FEDERATED_SERVICE_SOLR_INDEXING_ENABLED,
SwitchboardConstants.FEDERATED_SERVICE_SOLR_INDEXING_ENABLED_DEFAULT)) {
/* This operation is designed only for an embdded local Solr with no mirroring to an external remote Solr server */
segment.fulltext().rebootEmbeddedLocalSolr();
sb.tables.recordAPICall(post, "IndexControlURLs_p.html", WorkTables.TABLE_API_TYPE_STEERING, "solr reboot");
}
}
if (post.containsKey("deletedomain")) {
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
/* Check the transaction is valid */
TransactionManager.checkPostTransaction(header, post);
final String domain = post.get("domain");
Set<String> hostnames = new HashSet<String>();
hostnames.add(domain);

@ -69,7 +69,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
SOLR_ID_and_LOAD_DATE_FIELDS.add(CollectionSchema.id.getSolrFieldName());
SOLR_ID_and_LOAD_DATE_FIELDS.add(CollectionSchema.load_date_dt.getSolrFieldName());
}
public final static SolrDocument POISON_DOCUMENT = new SolrDocument();
public final static String POISON_ID = "POISON_ID";
public final static String CATCHALL_TERM = "[* TO *]";
@ -92,10 +92,11 @@ public abstract class AbstractSolrConnector implements SolrConnector {
catchSuccessQuery.setRows(0);
catchSuccessQuery.setStart(0);
}
protected final static int pagesize_docs = 100;
protected final static int pagesize_ids = 1000;
@Deprecated
protected static LoadTimeURL getLoadTimeURL(final Object doc) {
if (doc == null) return null;
Object d = null;
@ -139,7 +140,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
}
return fields;
}
/**
* Get results from a solr query as a stream of documents.
* The result queue is considered as terminated if AbstractSolrConnector.POISON_DOCUMENT is returned.
@ -170,7 +171,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
querystrings.add(querystring);
return concurrentDocumentsByQueries(querystrings, sort, offset, maxcount, maxtime, buffersize, concurrency, prefetchIDs, fields);
}
/**
* {@inheritDoc}
*/
@ -223,7 +224,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
}
return queue;
}
@Override
public Runnable newDocumentsByQueriesTask(
final BlockingQueue<SolrDocument> queue,
@ -236,7 +237,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
final int concurrency,
final String ... fields) {
Objects.requireNonNull(queue, "The queue parameter must not be null.");
if (querystrings == null || querystrings.isEmpty()) {
return () -> {
for (int i = 0; i < Math.max(1, concurrency); i++) {
@ -315,7 +316,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
}
};
}
/**
* get a document id result stream from a solr query.
* The result queue is considered as terminated if AbstractSolrConnector.POISON_ID is returned.
@ -341,7 +342,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
querystrings.add(querystring);
return concurrentIDsByQueries(querystrings, sort, offset, maxcount, maxtime, buffersize, concurrency);
}
/**
* get a document id result stream from a set of solr queries.
* The result queue is considered as terminated if AbstractSolrConnector.POISON_ID is returned.
@ -420,7 +421,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
};
}
/**
* get a query result from solr
* to get all results set the query String to "*:*"
@ -436,7 +437,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
final String ... fields) throws IOException {
// construct query
final SolrQuery params = getSolrQuery(querystring, sort, offset, count, fields);
// query the server
final SolrDocumentList docs = getDocumentListByParams(params);
return docs;
@ -471,13 +472,14 @@ public abstract class AbstractSolrConnector implements SolrConnector {
}
return params;
}
/**
* check if a given document, identified by url hash as document id exists
* @param id the url hash and document id
* @return metadata if any entry in solr exists, null otherwise
* @throws IOException
*/
@Deprecated
@Override
public LoadTimeURL getLoadTimeURL(String id) throws IOException {
// construct raw query
@ -499,7 +501,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
LoadTimeURL md = getLoadTimeURL(doc);
return md;
}
/**
* check if a given document, identified by url hash as document id exists
* @param id the url hash and document id
@ -515,7 +517,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
return false;
}
}
/**
* get the number of results when this query is done.
* This should only be called if the actual result is never used, and only the count is interesting
@ -564,7 +566,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
params.clearSorts();
params.setIncludeScore(false);
for (String field: fields) params.addFacetField(field);
// query the server
QueryResponse rsp = getResponseByParams(params);
LinkedHashMap<String, ReversibleScoreMap<String>> facets = new LinkedHashMap<String, ReversibleScoreMap<String>>(fields.length);
@ -578,7 +580,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
}
return facets;
}
@Override
public SolrDocument getDocumentById(final String id, final String ... fields) throws IOException {
assert id.length() == Word.commonHashLength : "wrong id: " + id;
@ -606,7 +608,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
throw new IOException(e.getMessage(), e);
}
}
/**
* Update a solr document.
* This will write only a partial update for all fields given in the SolrInputDocument
@ -634,7 +636,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
for (SolrInputDocument doc: solrdoc) docs.add(partialUpdatePatch(doc));
this.add(docs);
}
private SolrInputDocument partialUpdatePatch(final SolrInputDocument docIn) {
SolrInputDocument docOut = new SolrInputDocument();
docOut.setField(CollectionSchema.id.name(), docIn.getFieldValue(CollectionSchema.id.name()));
@ -649,6 +651,6 @@ public abstract class AbstractSolrConnector implements SolrConnector {
}
return docOut;
}
}

@ -425,10 +425,12 @@ public class EmbeddedSolrConnector extends SolrServerConnector implements SolrCo
/**
* check if a given document, identified by url hash as document id exists
* @Deprecated use Segment.getLastSeenTime instead
* @param id the url hash and document id
* @return the load date if any entry in solr exists, null otherwise
* @throws IOException
*/
@Deprecated
@Override
public LoadTimeURL getLoadTimeURL(String id) throws IOException {
int responseCount = 0;

@ -27,9 +27,6 @@ import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.atomic.AtomicLong;
import net.yacy.cora.sorting.ReversibleScoreMap;
import net.yacy.kelondro.data.word.Word;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
@ -38,6 +35,9 @@ import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import net.yacy.cora.sorting.ReversibleScoreMap;
import net.yacy.kelondro.data.word.Word;
public class MirrorSolrConnector extends AbstractSolrConnector implements SolrConnector {
// the twin solrs
@ -47,7 +47,7 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
this.solr0 = null;
this.solr1 = null;
}
public MirrorSolrConnector(SolrConnector solr0, SolrConnector solr1) {
this.solr0 = solr0;
this.solr1 = solr1;
@ -64,8 +64,8 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
((this.solr0 == null && ((MirrorSolrConnector) o).solr0 == null) || (((this.solr0 != null && ((MirrorSolrConnector) o).solr0 != null)) && this.solr0.equals(((MirrorSolrConnector) o).solr0))) &&
((this.solr1 == null && ((MirrorSolrConnector) o).solr1 == null) || (((this.solr1 != null && ((MirrorSolrConnector) o).solr1 != null)) && this.solr1.equals(((MirrorSolrConnector) o).solr1)));
}
@Override
public int bufferSize() {
int b = 0;
@ -79,7 +79,7 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
if (this.solr0 != null) this.solr0.clearCaches();
if (this.solr1 != null) this.solr1.clearCaches();
}
public boolean isConnected0() {
return this.solr0 != null;
}
@ -139,7 +139,7 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
public boolean isClosed() {
return (this.solr0 == null || this.solr0.isClosed()) && (this.solr1 == null || this.solr1.isClosed());
}
@Override
public synchronized void close() {
if (this.solr0 != null) {this.solr0.close(); this.solr0 = null;}
@ -183,12 +183,12 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
if (this.solr0 != null) this.solr0.deleteByQuery(querystring);
if (this.solr1 != null) this.solr1.deleteByQuery(querystring);
}
@Override
public SolrDocument getDocumentById(final String key, final String ... fields) throws IOException {
assert key.length() == Word.commonHashLength : "wrong id: " + key;
SolrDocument doc;
if ((solr0 != null && ((doc = solr0.getDocumentById(key, fields)) != null)) || (solr1 != null && ((doc = solr1.getDocumentById(key, fields)) != null))) {
if ((this.solr0 != null && ((doc = this.solr0.getDocumentById(key, fields)) != null)) || (this.solr1 != null && ((doc = this.solr1.getDocumentById(key, fields)) != null))) {
return doc;
}
return null;
@ -204,7 +204,7 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
if (this.solr0 != null) this.solr0.add(solrdoc);
if (this.solr1 != null) this.solr1.add(solrdoc);
}
@Override
public void add(final Collection<SolrInputDocument> solrdocs) throws IOException, SolrException {
if (this.solr0 != null) this.solr0.add(solrdocs);
@ -346,7 +346,7 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
// TODO: combine both
return l1;
}
@Override
public long getCountByQuery(final String querystring) throws IOException {
if (this.solr0 == null && this.solr1 == null) return 0;
@ -409,7 +409,7 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
if (this.solr1 != null) s += this.solr1.getSize();
return s;
}
@Override
public boolean exists(final String id) {
boolean result = false;
@ -418,6 +418,7 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
return result;
}
@Deprecated
@Override
public LoadTimeURL getLoadTimeURL(String id) throws IOException {
if (this.solr0 != null && this.solr1 == null) return this.solr0.getLoadTimeURL(id);
@ -438,5 +439,5 @@ public class MirrorSolrConnector extends AbstractSolrConnector implements SolrCo
if (this.solr0 == null && this.solr1 != null) return this.solr1.concurrentIDsByQuery(querystring, sort, offset, maxcount, maxtime, buffersize, concurrency);
return super.concurrentIDsByQuery(querystring, sort, offset, maxcount, maxtime, buffersize, concurrency);
}
}

@ -26,8 +26,6 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.concurrent.BlockingQueue;
import net.yacy.cora.sorting.ReversibleScoreMap;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
@ -35,6 +33,8 @@ import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.ModifiableSolrParams;
import net.yacy.cora.sorting.ReversibleScoreMap;
public interface SolrConnector extends Iterable<String> /* Iterable of document IDs */ {
public static class LoadTimeURL {
@ -45,23 +45,23 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
this.date = date;
}
}
/**
* clear all caches: inside solr and ouside solr within the implementations of this interface
*/
public void clearCaches();
/**
* get the size of a write buffer (if any) of pending write requests
*/
public int bufferSize();
/**
* get the size of the index
* @return number of results if solr is queries with a catch-all pattern
*/
public long getSize();
/**
* force a commit
*/
@ -72,19 +72,19 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
* @param maxSegments the maximum number of segments. Set to 1 for maximum optimization
*/
public void optimize(int maxSegments);
/**
* ask the solr subsystem about it's segment number
* @return the segment count, which corresponds to the number of files for an index
*/
public int getSegmentCount();
/**
* test if the connector is already closed
* @return true if the connector is closed
*/
public boolean isClosed();
/**
* close the server connection
*/
@ -123,8 +123,9 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
* @return the load time metadata (url and load data) if any entry in solr exists, null otherwise
* @throws IOException
*/
@Deprecated
public LoadTimeURL getLoadTimeURL(final String id) throws IOException;
/**
* check if a given document, identified by url hash as document id exists
* @param id the url hash and document id
@ -139,7 +140,7 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
* @throws SolrException
*/
public void add(final SolrInputDocument solrdoc) throws IOException, SolrException;
/**
* Update a solr document.
* This will write only a partial update for all fields given in the SolrInputDocument
@ -167,7 +168,7 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
* @throws SolrException
*/
public void update(final Collection<SolrInputDocument> solrdoc) throws IOException, SolrException;
/**
* get a document from solr by given key for the id-field
* @param key
@ -194,7 +195,7 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
* @throws SolrException
*/
public SolrDocumentList getDocumentListByParams(ModifiableSolrParams params) throws IOException;
/**
* get a query result from solr
* to get all results set the query String to "*:*"
@ -211,7 +212,7 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
final int offset,
final int count,
final String ... fields) throws IOException;
/**
* get the number of results when this query is done.
* This should only be called if the actual result is never used, and only the count is interesting
@ -229,12 +230,12 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
* @throws IOException
*/
public LinkedHashMap<String, ReversibleScoreMap<String>> getFacets(String query, int maxresults, final String ... fields) throws IOException;
/**
* <p>Get results from solr queries as a stream of documents.
* The result queue is considered as terminated if AbstractSolrConnector.POISON_DOCUMENT is returned.
* The method returns immediately and feeds the search results into the queue.</p>
* <p><strong>Important</strong> : be careful if the consumer thread(s) terminate before taking the poison document(s) from the queue,
* <p><strong>Important</strong> : be careful if the consumer thread(s) terminate before taking the poison document(s) from the queue,
* as the producer thread(s) may indefinitely block on their last step (adding poison element) because the queue would be full.</p>
* @param querystring the solr query string
* @param sort the solr sort string, may be null to be not used
@ -257,11 +258,11 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
final int concurrency,
final boolean prefetchIDs,
final String ... fields);
/**
* Creates a new runnable task to run a given list of Solr queries and fill a
* results queue by packets of a limited number of results.
*
*
* @param queue the results queue. Must not be null.
* @param querystrings a list of Solr queries
* @param sort an eventual Solr sort criteria
@ -278,7 +279,7 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
public Runnable newDocumentsByQueriesTask(final BlockingQueue<SolrDocument> queue, final List<String> querystrings,
final String sort, final int offset, final int maxcount, final long maxtime, final int buffersize,
final int concurrency, final String... fields);
/**
* Get results from solr queries as a stream of documents.
* The result queue is considered as terminated if AbstractSolrConnector.POISON_DOCUMENT is returned.
@ -304,7 +305,7 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
final int concurrency,
final boolean prefetchIDs,
final String ... fields);
/**
* get a document id result stream from a solr query.
* The result queue is considered as terminated if AbstractSolrConnector.POISON_ID is returned.

@ -425,23 +425,15 @@ public final class CrawlStacker implements WorkflowTask<Request>{
if (dbocc != null) {
return CRAWL_REJECT_REASON_DOUBLE_IN_PREFIX + ": " + dbocc.name();
}
final String urlhash = ASCII.String(url.hash());
LoadTimeURL oldEntry = null;
try {
oldEntry = this.indexSegment.fulltext().getDefaultConnector().getLoadTimeURL(urlhash);
} catch (final IOException e) {
// if an exception here occurs then there is the danger that urls which had been in the crawler are overwritten a second time
// to prevent that, we reject urls in these events
ConcurrentLog.logException(e);
return "exception during double-test: " + e.getMessage();
}
String urls = url.toNormalform(false);
LoadTimeURL oldEntry = this.indexSegment.getLoadTimeURL(urls, url.hash());
// deny urls that exceed allowed number of occurrences
final int maxAllowedPagesPerDomain = profile.domMaxPages();
if (maxAllowedPagesPerDomain < Integer.MAX_VALUE && maxAllowedPagesPerDomain > 0) {
final AtomicInteger dp = profile.getCount(url.getHost());
if (dp != null && dp.get() >= maxAllowedPagesPerDomain) {
if (CrawlStacker.log.isFine()) CrawlStacker.log.fine("URL '" + url.toNormalform(false) + "' appeared too often in crawl stack, a maximum of " + maxAllowedPagesPerDomain + " is allowed.");
if (CrawlStacker.log.isFine()) CrawlStacker.log.fine("URL '" + urls + "' appeared too often in crawl stack, a maximum of " + maxAllowedPagesPerDomain + " is allowed.");
return "crawl stack domain counter exceeded (test by profile)";
}

File diff suppressed because it is too large Load Diff

@ -217,7 +217,7 @@ public final class Fulltext {
}
return null;
} finally {
this.solrInstancesLock.unlock();
this.solrInstancesLock.unlock();
}
}
@ -226,7 +226,7 @@ public final class Fulltext {
try {
return this.solrInstances.getDefaultMirrorConnector();
} finally {
this.solrInstancesLock.unlock();
this.solrInstancesLock.unlock();
}
}
@ -319,8 +319,8 @@ public final class Fulltext {
private long lastCommit = 0;
public void commit(boolean softCommit) {
long t = System.currentTimeMillis();
if (lastCommit + 10000 > t) return;
lastCommit = t;
if (this.lastCommit + 10000 > t) return;
this.lastCommit = t;
getDefaultConnector().commit(softCommit);
if (this.writeWebgraph) getWebgraphConnector().commit(softCommit);
}
@ -332,13 +332,13 @@ public final class Fulltext {
* are accessible) of the returned document.
* If the no document with url.hash = solrdocument.id is found in the embedded
* Solr index null is return.
*
*
* @param element rwi wordreference
* @return URIMetadataNode (solrdocument) with all fields stored in embedded solr index
*/
public URIMetadataNode getMetadata(final WeakPriorityBlockingQueue.Element<WordReferenceVars> element) {
if (element == null) return null;
WordReferenceVars wre = element.getElement();
WordReferenceVars wre = element.getElement();
if (wre == null) return null; // all time was already wasted in takeRWI to get another element
long score = element.getWeight();
URIMetadataNode node = getMetadata(wre.urlhash(), wre, score);
@ -572,7 +572,7 @@ public final class Fulltext {
if (md == null) return null;
return new DigestURL(md.url, ASCII.getBytes(urlHash));
}
/**
* check if a given document, identified by url hash as document id exists
* @param id the url hash and document id
@ -587,6 +587,7 @@ public final class Fulltext {
* @param urlHash
* @return the time in milliseconds since epoch for the load time or -1 if the document does not exist
*/
@Deprecated
private long getLoadTime(final String urlHash) throws IOException {
if (urlHash == null) return -1l;
SolrConnector.LoadTimeURL md = this.getDefaultConnector().getLoadTimeURL(urlHash);
@ -727,7 +728,7 @@ public final class Fulltext {
String nowstr = new Date(now).toInstant().toString();
String fromstr = new Date(from).toInstant().toString();
String dateq = CollectionSchema.load_date_dt.getSolrFieldName() + ":[" + fromstr + " TO " + nowstr + "]";
query = query == null || AbstractSolrConnector.CATCHALL_QUERY.equals(query) ? dateq : query + " AND " + dateq;
query = query == null || AbstractSolrConnector.CATCHALL_QUERY.equals(query) ? dateq : query + " AND " + dateq;
} else {
query = query == null? AbstractSolrConnector.CATCHALL_QUERY : query;
}
@ -744,7 +745,7 @@ public final class Fulltext {
final Date firstdate, lastdate;
if (firstdoclist.size() == 0 || lastdoclist.size() == 0) {
/* Now check again the number of documents without sorting, for compatibility with old fields indexed without DocValues fields (prior to YaCy 1.90)
* When the local Solr index contains such old documents, requests with sort query return nothing and trace in logs
* When the local Solr index contains such old documents, requests with sort query return nothing and trace in logs
* "java.lang.IllegalStateException: unexpected docvalues type NONE for field..." */
doccount = this.getDefaultConnector().getCountByQuery(query);
if(doccount == 0) {
@ -764,8 +765,8 @@ public final class Fulltext {
firstdateobject = firstdoc.getFieldValue(CollectionSchema.load_date_dt.getSolrFieldName());
lastdateobject = lastdoc.getFieldValue(CollectionSchema.load_date_dt.getSolrFieldName());
/* When firstdate or lastdate is null, we use a default one just to generate a proper dump file path
* This should not happen because load_date_dt field is mandatory in the main Solr schema,
/* When firstdate or lastdate is null, we use a default one just to generate a proper dump file path
* This should not happen because load_date_dt field is mandatory in the main Solr schema,
* but for some reason some documents might end up here with an empty load_date_dt field value */
if(firstdateobject instanceof Date) {
firstdate = (Date) firstdateobject;
@ -854,7 +855,7 @@ public final class Fulltext {
try (/* Resources automatically closed by this try-with-resources statement */
final OutputStream os = new FileOutputStream(this.format == ExportFormat.solr ? new File(this.f.getAbsolutePath() + ".gz") : this.f);
final OutputStream wrappedStream = ((this.format == ExportFormat.solr)) ? new GZIPOutputStream(os, 65536){{def.setLevel(Deflater.BEST_COMPRESSION);}} : os;
final OutputStream wrappedStream = ((this.format == ExportFormat.solr)) ? new GZIPOutputStream(os, 65536){{this.def.setLevel(Deflater.BEST_COMPRESSION);}} : os;
final PrintWriter pw = new PrintWriter(new BufferedOutputStream(wrappedStream));
) {
if (this.format == ExportFormat.html) {
@ -907,7 +908,7 @@ public final class Fulltext {
this.count++;
}
} else {
BlockingQueue<SolrDocument> docs = Fulltext.this.getDefaultConnector().concurrentDocumentsByQuery(this.query + " AND " + CollectionSchema.httpstatus_i.getSolrFieldName() + ":200", null, 0, 100000000, Long.MAX_VALUE, 100, 1, true,
BlockingQueue<SolrDocument> docs = Fulltext.this.getDefaultConnector().concurrentDocumentsByQuery(this.query + " AND " + CollectionSchema.httpstatus_i.getSolrFieldName() + ":200", null, 0, 100000000, Long.MAX_VALUE, 100, 1, true,
CollectionSchema.id.getSolrFieldName(), CollectionSchema.sku.getSolrFieldName(), CollectionSchema.title.getSolrFieldName(),
CollectionSchema.author.getSolrFieldName(), CollectionSchema.description_txt.getSolrFieldName(), CollectionSchema.size_i.getSolrFieldName(), CollectionSchema.last_modified.getSolrFieldName());
SolrDocument doc;

@ -51,6 +51,7 @@ import net.yacy.cora.document.id.DigestURL;
import net.yacy.cora.document.id.MultiProtocolURL;
import net.yacy.cora.federate.solr.connector.AbstractSolrConnector;
import net.yacy.cora.federate.solr.connector.SolrConnector;
import net.yacy.cora.federate.solr.connector.SolrConnector.LoadTimeURL;
import net.yacy.cora.federate.yacy.CacheStrategy;
import net.yacy.cora.order.Base64Order;
import net.yacy.cora.order.ByteOrder;
@ -110,8 +111,9 @@ public class Segment {
public static final long targetFileSize = 64 * 1024 * 1024; // 256 MB
public static final int writeBufferSize = 4 * 1024 * 1024;
public static final String termIndexName = "text.index";
public static final String citationIndexName = "citation.index";
public static final String citationIndexName = "citation.index";
public static final String firstseenIndexName = "firstseen.index";
public static final String loadtimeIndexName = "loadtime.index";
// the reference factory
public static final ReferenceFactory<WordReference> wordReferenceFactory = new WordReferenceFactory();
@ -122,9 +124,10 @@ public class Segment {
private final File segmentPath;
protected final Fulltext fulltext;
protected IndexCell<WordReference> termIndex;
protected IndexCell<CitationReference> urlCitationIndex;
protected IndexTable firstSeenIndex;
protected IODispatcher merger = null; // shared iodispatcher for kelondro indexes
private IndexCell<CitationReference> urlCitationIndex;
private IndexTable firstSeenIndex;
private IndexTable loadTimeIndex;
private IODispatcher merger = null; // shared iodispatcher for kelondro indexes
/**
* create a new Segment
@ -138,20 +141,21 @@ public class Segment {
this.log = log;
this.segmentPath = segmentPath;
archivePath.mkdirs();
this.fulltext = new Fulltext(segmentPath, archivePath, collectionConfiguration, webgraphConfiguration);
this.termIndex = null;
this.urlCitationIndex = null;
this.firstSeenIndex = new IndexTable(new File(segmentPath, firstseenIndexName), 12, 8, false, false);
this.loadTimeIndex = new IndexTable(new File(segmentPath, loadtimeIndexName), 12, 8, false, false);
}
public boolean connectedRWI() {
return this.termIndex != null;
}
public void connectRWI(final int entityCacheMaxSize, final long maxFileSize) throws IOException {
if (this.termIndex != null) return;
if (this.merger == null) { // init shared iodispatcher if none running
this.merger = new IODispatcher(2, 2, writeBufferSize);
this.merger.start();
@ -166,7 +170,7 @@ public class Segment {
targetFileSize,
maxFileSize,
writeBufferSize,
merger);
this.merger);
}
public void disconnectRWI() {
@ -196,7 +200,7 @@ public class Segment {
targetFileSize,
maxFileSize,
writeBufferSize,
merger);
this.merger);
}
public void disconnectCitation() {
@ -208,7 +212,7 @@ public class Segment {
public int citationCount() {
return this.urlCitationIndex == null ? 0 : this.urlCitationIndex.sizesMax();
}
public long citationSegmentCount() {
return this.urlCitationIndex == null ? 0 : this.urlCitationIndex.getSegmentCount();
}
@ -224,27 +228,31 @@ public class Segment {
public IndexCell<CitationReference> urlCitation() {
return this.urlCitationIndex;
}
public IndexTable firstSeen() {
public IndexTable firstSeenIndex() {
return this.firstSeenIndex;
}
public IndexTable loadTimeIndex() {
return this.loadTimeIndex;
}
public ReferenceReportCache getReferenceReportCache() {
return new ReferenceReportCache();
}
public class ReferenceReportCache {
private final Map<String, ReferenceReport> cache;
public ReferenceReportCache() {
this.cache = new ConcurrentHashMap<String, ReferenceReport>();
}
public ReferenceReport getReferenceReport(final String id, final boolean acceptSelfReference) throws IOException {
ReferenceReport rr = cache.get(id);
if (MemoryControl.shortStatus()) cache.clear();
ReferenceReport rr = this.cache.get(id);
if (MemoryControl.shortStatus()) this.cache.clear();
if (rr != null) return rr;
try {
rr = new ReferenceReport(ASCII.getBytes(id), acceptSelfReference);
cache.put(id, rr);
this.cache.put(id, rr);
return rr;
} catch (final SpaceExceededException e) {
ConcurrentLog.logException(e);
@ -252,7 +260,7 @@ public class Segment {
}
}
}
/**
* A ReferenceReport object is a container for all references to a specific url.
* The class stores the number of links from domain-internal and domain-external backlinks,
@ -278,19 +286,19 @@ public class Segment {
CitationReference ref = ri.next();
byte[] hh = ref.hosthash(); // host hash
if (ByteBuffer.equals(hh, 0, id, 6, 6)) {
internalIDs.put(ref.urlhash());
internal++;
this.internalIDs.put(ref.urlhash());
this.internal++;
} else {
externalHosts.put(hh);
externalIDs.put(ref.urlhash());
external++;
this.externalHosts.put(hh);
this.externalIDs.put(ref.urlhash());
this.external++;
}
}
} catch (SpaceExceededException e) {
// the Citation Index got too large, we ignore the problem and hope that a second solr index is attached which will take over now
if (Segment.this.fulltext.useWebgraph()) internalIDs.clear();
if (Segment.this.fulltext.useWebgraph()) this.internalIDs.clear();
}
if ((internalIDs.size() == 0 || !connectedCitation()) && Segment.this.fulltext.useWebgraph()) {
if ((this.internalIDs.size() == 0 || !connectedCitation()) && Segment.this.fulltext.useWebgraph()) {
// reqd the references from the webgraph
SolrConnector webgraph = Segment.this.fulltext.getWebgraphConnector();
BlockingQueue<SolrDocument> docs = webgraph.concurrentDocumentsByQuery("{!cache=false raw f=" + WebgraphSchema.target_id_s.getSolrFieldName() + "}" + ASCII.String(id), WebgraphSchema.source_chars_i.getSolrFieldName() + " asc", 0, 10000000, Long.MAX_VALUE, 100, 1, false, WebgraphSchema.source_id_s.getSolrFieldName());
@ -305,13 +313,13 @@ public class Segment {
System.arraycopy(refidh, 6, hh, 0, 6);
if (ByteBuffer.equals(hh, 0, id, 6, 6)) {
if (acceptSelfReference || !Arrays.equals(refidh, id)) {
internalIDs.put(refidh);
internal++;
this.internalIDs.put(refidh);
this.internal++;
}
} else {
externalHosts.put(hh);
externalIDs.put(refidh);
external++;
this.externalHosts.put(hh);
this.externalIDs.put(refidh);
this.external++;
}
}
} catch (final InterruptedException e) {
@ -338,12 +346,12 @@ public class Segment {
return this.internalIDs;
}
}
public long RWICount() {
if (this.termIndex == null) return 0;
return this.termIndex.sizesMax();
}
public long RWISegmentCount() {
if (this.termIndex == null) return 0;
return this.termIndex.getSegmentCount();
@ -377,7 +385,7 @@ public class Segment {
return 0;
}
}
public void setFirstSeenTime(final byte[] urlhash, long time) {
if (urlhash == null || time <= 0) return;
try {
@ -387,7 +395,7 @@ public class Segment {
ConcurrentLog.logException(e);
}
}
public long getFirstSeenTime(final byte[] urlhash) {
if (urlhash == null) return -1;
try {
@ -397,7 +405,36 @@ public class Segment {
return -1;
}
}
public void setLoadTime(final byte[] urlhash, long time) {
if (urlhash == null || time <= 0) return;
try {
this.loadTimeIndex.put(urlhash, time); // ALWAYS overwrite!
} catch (IOException e) {
ConcurrentLog.logException(e);
}
}
public long getLoadTime(final byte[] urlhash) {
if (urlhash == null) return -1;
try {
return this.loadTimeIndex.get(urlhash);
} catch (IOException e) {
ConcurrentLog.logException(e);
return -1;
}
}
public LoadTimeURL getLoadTimeURL(String url, byte[] urlhash) {
long t = getLoadTime(urlhash);
if (t < 0) return null;
return new LoadTimeURL(url, t);
}
public LoadTimeURL getLoadTimeURL(String url, String id) {
return getLoadTimeURL(url, id.getBytes());
}
/**
* check if a given document, identified by url hash as document id exists
* @param id the url hash and document id
@ -467,7 +504,7 @@ public class Segment {
ConcurrentLog.logException(e);
}
}
public void clearCaches() {
if (this.urlCitationIndex != null) this.urlCitationIndex.clearCache();
if (this.termIndex != null) this.termIndex.clearCache();
@ -483,6 +520,7 @@ public class Segment {
if (this.fulltext != null) this.fulltext.close();
if (this.urlCitationIndex != null) this.urlCitationIndex.close();
if (this.firstSeenIndex != null) this.firstSeenIndex.close();
if (this.loadTimeIndex != null) this.loadTimeIndex.close();
if (this.merger != null) {
this.merger.terminate();
this.merger = null;
@ -575,11 +613,11 @@ public class Segment {
) {
final CollectionConfiguration collectionConfig = this.fulltext.getDefaultConfiguration();
final String language = votedLanguage(url, url.toNormalform(true), document, condenser); // identification of the language
final CollectionConfiguration.SolrVector vector = collectionConfig.yacy2solr(this, collections, responseHeader,
document, condenser, referrerURL, language, crawlProfile.isPushCrawlProfile(),
this.fulltext().useWebgraph() ? this.fulltext.getWebgraphConfiguration() : null, sourceName);
return storeDocument(url, crawlProfile, responseHeader, document, vector, language, condenser,
searchEvent, sourceName, storeToRWI, proxy, acceptLanguage);
}
@ -599,10 +637,10 @@ public class Segment {
final String acceptLanguage
) {
final long startTime = System.currentTimeMillis();
final CollectionConfiguration collectionConfig = this.fulltext.getDefaultConfiguration();
final String urlNormalform = url.toNormalform(true);
// CREATE INDEX
// load some document metadata
final Date loadDate = new Date();
@ -619,7 +657,7 @@ public class Segment {
// ENRICH DOCUMENT WITH RANKING INFORMATION
this.fulltext.getDefaultConfiguration().postprocessing_references(this.getReferenceReportCache(), vector, url, null);
// CREATE SNAPSHOT
if ((url.getProtocol().equals("http") || url.getProtocol().equals("https")) &&
crawlProfile != null && document.getDepth() <= crawlProfile.snapshotMaxdepth() &&
@ -634,13 +672,13 @@ public class Segment {
// STORE IMAGE AND METADATA
Transactions.store(vector, true, crawlProfile.snapshotLoadImage(), crawlProfile.snapshotReplaceold(), proxy, acceptLanguage);
}
// STORE TO SOLR
this.putDocument(vector);
List<SolrInputDocument> webgraph = vector.getWebgraphDocuments();
String error = null;
if (webgraph != null && webgraph.size() > 0) {
// write the edges to the webgraph solr index
if (this.fulltext.useWebgraph()) {
tryloop: for (int i = 0; i < 20; i++) {
@ -657,16 +695,18 @@ public class Segment {
}
}
}
}
// REMEMBER FIRST SEEN
setFirstSeenTime(url.hash(), Math.min(document.getLastModified().getTime(), System.currentTimeMillis())); // should exist already in the index at this time, but just to make sure
long now = System.currentTimeMillis();
setFirstSeenTime(url.hash(), Math.min(document.getLastModified().getTime(), now)); // should exist already in the index at this time, but just to make sure
setLoadTime(url.hash(), now); // always overwrites index entry
// write the edges to the citation reference index
if (this.connectedCitation()) try {
// we use the subgraph to write the citation index, that shall cause that the webgraph and the citation index is identical
if (collectionConfig.contains(CollectionSchema.inboundlinks_protocol_sxt) || collectionConfig.contains(CollectionSchema.inboundlinks_urlstub_sxt)) {
Collection<Object> inboundlinks_urlstub = vector.getFieldValues(CollectionSchema.inboundlinks_urlstub_sxt.getSolrFieldName());
List<String> inboundlinks_protocol = inboundlinks_urlstub == null ? null : CollectionConfiguration.indexedList2protocolList(vector.getFieldValues(CollectionSchema.inboundlinks_protocol_sxt.getSolrFieldName()), inboundlinks_urlstub.size());
@ -676,7 +716,7 @@ public class Segment {
String referrerhash = id;
String anchorhash = ASCII.String(new DigestURL(targetURL).hash());
if (referrerhash != null && anchorhash != null) {
urlCitationIndex.add(ASCII.getBytes(anchorhash), new CitationReference(ASCII.getBytes(referrerhash), loadDate.getTime()));
this.urlCitationIndex.add(ASCII.getBytes(anchorhash), new CitationReference(ASCII.getBytes(referrerhash), loadDate.getTime()));
}
} catch (Throwable e) {
ConcurrentLog.logException(e);
@ -692,7 +732,7 @@ public class Segment {
String referrerhash = id;
String anchorhash = ASCII.String(new DigestURL(targetURL).hash());
if (referrerhash != null && anchorhash != null) {
urlCitationIndex.add(ASCII.getBytes(anchorhash), new CitationReference(ASCII.getBytes(referrerhash), loadDate.getTime()));
this.urlCitationIndex.add(ASCII.getBytes(anchorhash), new CitationReference(ASCII.getBytes(referrerhash), loadDate.getTime()));
}
} catch (Throwable e) {
ConcurrentLog.logException(e);
@ -702,7 +742,7 @@ public class Segment {
} catch (Throwable e) {
ConcurrentLog.logException(e);
}
if (error != null) {
ConcurrentLog.severe("SOLR", error + ", PLEASE REPORT TO https://github.com/yacy/yacy_search_server/issues");
//Switchboard.getSwitchboard().pauseCrawlJob(SwitchboardConstants.CRAWLJOB_LOCAL_CRAWL, error);
@ -728,7 +768,7 @@ public class Segment {
UTF8.getBytes(language),
docType,
outlinksSame, outlinksOther);
// iterate over all words of content text
Word wprop = null;
byte[] wordhash;
@ -744,7 +784,7 @@ public class Segment {
} catch (final Exception e) {
ConcurrentLog.logException(e);
}
// during a search event it is possible that a heuristic is used which aquires index
// data during search-time. To transfer indexed data directly to the search process
// the following lines push the index data additionally to the search process
@ -762,7 +802,7 @@ public class Segment {
}
}
if (searchEvent != null) searchEvent.addFinalize();
// assign the catchall word
ientry.setWord(wprop == null ? catchallWord : wprop); // we use one of the word properties as template to get the document characteristics
if (this.termIndex != null) try {this.termIndex.add(catchallHash, ientry);} catch (final Throwable e) {ConcurrentLog.logException(e);}

Loading…
Cancel
Save