Fixed termination of Host browser and link structure Solr query threads

On some conditions (especially when reaching timeout), concurrent Solr
query tasks used by the /HostBrowser.html and /api/linkstructure.json
never terminated, thus leaking resources, as reported by @Vort in issue
#246
pull/258/head
luccioman 6 years ago
parent 260ac11c65
commit e85f231bdf

@ -32,7 +32,9 @@ import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.regex.Pattern;
import org.apache.solr.common.SolrDocument;
@ -417,7 +419,11 @@ public class HostBrowser {
q.append(" AND ").append(CollectionSchema.url_paths_sxt.getSolrFieldName()).append(AbstractSolrConnector.CATCHALL_DTERM);
}
}
BlockingQueue<SolrDocument> docs = fulltext.getDefaultConnector().concurrentDocumentsByQuery(q.toString(), CollectionSchema.url_chars_i.getSolrFieldName() + " asc", 0, 100000, TIMEOUT, 100, 1, false,
final int pageSize = 100;
final BlockingQueue<SolrDocument> docs = new ArrayBlockingQueue<>(pageSize);
final List<String> queries = new ArrayList<>();
queries.add(q.toString());
final Thread solrQueryTask = new Thread(fulltext.getDefaultConnector().newDocumentsByQueriesTask(docs, queries, CollectionSchema.url_chars_i.getSolrFieldName() + " asc", 0, 100000, TIMEOUT, pageSize, 1,
CollectionSchema.id.getSolrFieldName(),
CollectionSchema.sku.getSolrFieldName(),
CollectionSchema.failreason_s.getSolrFieldName(),
@ -433,8 +439,8 @@ public class HostBrowser {
CollectionSchema.references_exthosts_i.getSolrFieldName(),
CollectionSchema.cr_host_chance_d.getSolrFieldName(),
CollectionSchema.cr_host_norm_i.getSolrFieldName()
);
SolrDocument doc;
));
solrQueryTask.start();
Set<String> storedDocs = new HashSet<String>();
Map<String, FailType> errorDocs = new HashMap<String, FailType>();
Set<String> inboundLinks = new HashSet<String>();
@ -445,60 +451,72 @@ public class HostBrowser {
final Collection<String> reloadURLs = new ArrayList<String>();
final Set<String> reloadURLCollection = new HashSet<String>();
long timeoutList = System.currentTimeMillis() + TIMEOUT;
long remainingTime = TIMEOUT;
long timeoutReferences = System.currentTimeMillis() + 6000;
ReferenceReportCache rrCache = sb.index.getReferenceReportCache();
while ((doc = docs.take()) != AbstractSolrConnector.POISON_DOCUMENT) {
String u = (String) doc.getFieldValue(CollectionSchema.sku.getSolrFieldName());
String errortype = (String) doc.getFieldValue(CollectionSchema.failtype_s.getSolrFieldName());
FailType error = errortype == null ? null : FailType.valueOf(errortype);
String ids = (String) doc.getFieldValue(CollectionSchema.id.getSolrFieldName());
infoCache.put(ids, new InfoCacheEntry(sb.index.fulltext(), rrCache, doc, ids, System.currentTimeMillis() < timeoutReferences));
if (u.startsWith(path)) {
if (delete) {
deleteIDs.add(ids);
} else {
if (error == null) storedDocs.add(u); else {
if (reload404 && error == FailType.fail) {
ArrayList<String> collections = (ArrayList<String>) doc.getFieldValue(CollectionSchema.collection_sxt.getSolrFieldName());
if (collections != null) reloadURLCollection.addAll(collections);
reloadURLs.add(u);
}
if (authorized) errorDocs.put(u, error);
}
}
} else if (complete) {
if (error == null) storedDocs.add(u); else {
if (authorized) errorDocs.put(u, error);
}
}
if ((complete || u.startsWith(path)) && !storedDocs.contains(u)) inboundLinks.add(u); // add the current link
if (error == null) {
hostsize++;
// collect inboundlinks to browse the host
Iterator<String> links = URIMetadataNode.getLinks(doc, true);
while (links.hasNext()) {
u = links.next();
if ((complete || u.startsWith(path)) && !storedDocs.contains(u)) inboundLinks.add(u);
}
try {
SolrDocument doc = docs.poll(remainingTime, TimeUnit.MILLISECONDS);
while (doc != AbstractSolrConnector.POISON_DOCUMENT && doc != null) {
String u = (String) doc.getFieldValue(CollectionSchema.sku.getSolrFieldName());
String errortype = (String) doc.getFieldValue(CollectionSchema.failtype_s.getSolrFieldName());
FailType error = errortype == null ? null : FailType.valueOf(errortype);
String ids = (String) doc.getFieldValue(CollectionSchema.id.getSolrFieldName());
infoCache.put(ids, new InfoCacheEntry(sb.index.fulltext(), rrCache, doc, ids, System.currentTimeMillis() < timeoutReferences));
if (u.startsWith(path)) {
if (delete) {
deleteIDs.add(ids);
} else {
if (error == null) storedDocs.add(u); else {
if (reload404 && error == FailType.fail) {
ArrayList<String> collections = (ArrayList<String>) doc.getFieldValue(CollectionSchema.collection_sxt.getSolrFieldName());
if (collections != null) reloadURLCollection.addAll(collections);
reloadURLs.add(u);
}
if (authorized) errorDocs.put(u, error);
}
}
} else if (complete) {
if (error == null) storedDocs.add(u); else {
if (authorized) errorDocs.put(u, error);
}
}
if ((complete || u.startsWith(path)) && !storedDocs.contains(u)) inboundLinks.add(u); // add the current link
if (error == null) {
hostsize++;
// collect inboundlinks to browse the host
Iterator<String> links = URIMetadataNode.getLinks(doc, true);
while (links.hasNext()) {
u = links.next();
if ((complete || u.startsWith(path)) && !storedDocs.contains(u)) inboundLinks.add(u);
}
// collect referrer links
links = URIMetadataNode.getLinks(doc, false);
while (links.hasNext()) {
u = links.next();
try {
MultiProtocolURL mu = new MultiProtocolURL(u);
if (mu.getHost() != null) {
ReversibleScoreMap<String> lks = outboundHosts.get(mu.getHost());
if (lks == null) {
lks = new ClusteredScoreMap<String>(UTF8.insensitiveUTF8Comparator);
outboundHosts.put(mu.getHost(), lks);
}
lks.set(u, u.length());
}
} catch (final MalformedURLException e) {}
}
}
if (System.currentTimeMillis() > timeoutList) break;
// collect referrer links
links = URIMetadataNode.getLinks(doc, false);
while (links.hasNext()) {
u = links.next();
try {
MultiProtocolURL mu = new MultiProtocolURL(u);
if (mu.getHost() != null) {
ReversibleScoreMap<String> lks = outboundHosts.get(mu.getHost());
if (lks == null) {
lks = new ClusteredScoreMap<String>(UTF8.insensitiveUTF8Comparator);
outboundHosts.put(mu.getHost(), lks);
}
lks.set(u, u.length());
}
} catch (final MalformedURLException e) {}
}
}
remainingTime = timeoutList - System.currentTimeMillis();
if (remainingTime <= 0) {
break;
}
doc = docs.poll(remainingTime, TimeUnit.MILLISECONDS);
}
} finally {
/* Ensure termination and proper resources release of the query thread */
solrQueryTask.interrupt();
}
if (deleteIDs.size() > 0) sb.remove(deleteIDs);
if (reloadURLs.size() > 0) {

@ -21,6 +21,7 @@
package net.yacy.cora.federate.solr.connector;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Date;
@ -31,24 +32,17 @@ import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import net.yacy.cora.document.encoding.UTF8;
import net.yacy.cora.sorting.ClusteredScoreMap;
import net.yacy.cora.sorting.ReversibleScoreMap;
import net.yacy.cora.util.ConcurrentLog;
import net.yacy.cora.util.LookAheadIterator;
import net.yacy.kelondro.data.word.Word;
import net.yacy.search.schema.CollectionSchema;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.response.FacetField;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.client.solrj.response.FacetField.Count;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrDocument;
import org.apache.solr.common.SolrDocumentList;
import org.apache.solr.common.SolrException;
@ -58,6 +52,14 @@ import org.apache.solr.common.params.CommonParams;
import org.apache.solr.common.params.DisMaxParams;
import org.apache.solr.common.params.FacetParams;
import net.yacy.cora.document.encoding.UTF8;
import net.yacy.cora.sorting.ClusteredScoreMap;
import net.yacy.cora.sorting.ReversibleScoreMap;
import net.yacy.cora.util.ConcurrentLog;
import net.yacy.cora.util.LookAheadIterator;
import net.yacy.kelondro.data.word.Word;
import net.yacy.search.schema.CollectionSchema;
public abstract class AbstractSolrConnector implements SolrConnector {
protected static Set<String> SOLR_ID_FIELDS = new HashSet<String>();
@ -170,19 +172,7 @@ public abstract class AbstractSolrConnector implements SolrConnector {
}
/**
* 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
* @param querystrings the list of solr query strings
* @param sort the solr sort string, may be null to be not used
* @param offset first result offset
* @param maxcount the maximum number of results
* @param maxtime the maximum time in milliseconds
* @param buffersize the size of an ArrayBlockingQueue; if <= 0 then a LinkedBlockingQueue is used
* @param concurrency is the number of AbstractSolrConnector.POISON_DOCUMENT entries to add at the end of the feed
* @param prefetchIDs if true, then first all IDs are fetched and then all documents are queries by the ID. If false then documents are retrieved directly
* @param fields list of fields
* @return a blocking queue which is terminated with AbstractSolrConnector.POISON_DOCUMENT as last element
* {@inheritDoc}
*/
@Override
public BlockingQueue<SolrDocument> concurrentDocumentsByQueries(
@ -195,12 +185,11 @@ public abstract class AbstractSolrConnector implements SolrConnector {
final int concurrency,
final boolean prefetchIDs,
final String ... fields) {
assert buffersize > 0;
if (!prefetchIDs) return concurrentDocumentsByQueriesNoPrefetch(querystrings, sort, offset, maxcount, maxtime, buffersize, concurrency, fields);
final BlockingQueue<SolrDocument> queue = buffersize <= 0 ? new LinkedBlockingQueue<SolrDocument>() : new ArrayBlockingQueue<SolrDocument>(Math.max(buffersize, concurrency));
if (querystrings.size() == 0) {
for (int i = 0; i < Math.max(1, concurrency); i++) try {queue.put(AbstractSolrConnector.POISON_DOCUMENT);} catch (final InterruptedException e1) {}
return queue;
if (!prefetchIDs) {
final Thread t = new Thread(newDocumentsByQueriesTask(queue, querystrings, sort, offset, maxcount, maxtime, buffersize, concurrency, fields));
t.start();
return queue;
}
final BlockingQueue<String> idQueue = concurrentIDsByQueries(querystrings, sort, offset, maxcount, maxtime, Math.min(maxcount, 10000000), concurrency);
final long endtime = maxtime < 0 || maxtime == Long.MAX_VALUE ? Long.MAX_VALUE : System.currentTimeMillis() + maxtime; // we know infinity!
@ -235,7 +224,9 @@ public abstract class AbstractSolrConnector implements SolrConnector {
return queue;
}
private BlockingQueue<SolrDocument> concurrentDocumentsByQueriesNoPrefetch(
@Override
public Runnable newDocumentsByQueriesTask(
final BlockingQueue<SolrDocument> queue,
final List<String> querystrings,
final String sort,
final int offset,
@ -244,59 +235,85 @@ public abstract class AbstractSolrConnector implements SolrConnector {
final int buffersize,
final int concurrency,
final String ... fields) {
assert buffersize > 0;
final BlockingQueue<SolrDocument> queue = buffersize <= 0 ? new LinkedBlockingQueue<SolrDocument>() : new ArrayBlockingQueue<SolrDocument>(buffersize);
if (querystrings.size() == 0) {
for (int i = 0; i < Math.max(1, concurrency); i++) try {queue.put(AbstractSolrConnector.POISON_DOCUMENT);} catch (final InterruptedException e1) {}
return queue;
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++) {
try {
queue.put(AbstractSolrConnector.POISON_DOCUMENT);
} catch (final InterruptedException e1) {
Thread.currentThread().interrupt(); // preserve interrupted thread state
}
}
};
}
final long endtime = maxtime < 0 || maxtime == Long.MAX_VALUE ? Long.MAX_VALUE : System.currentTimeMillis() + maxtime; // we know infinity!
final int ps = buffersize < 0 ? pagesize_docs : Math.min(pagesize_docs, buffersize);
final int maxretries = 6;
final Thread t = new Thread() {
@Override
public void run() {
try {
for (String querystring: querystrings) {
this.setName("AbstractSolrConnector:concurrentDocumentsByQueryNoPrefetch(" + querystring + ")");
int o = offset;
int count = 0;
int retry = 0;
loop: while (System.currentTimeMillis() < endtime && count < maxcount) {
try {
SolrDocumentList sdl = getDocumentListByQuery(querystring, sort, o, Math.min(maxcount, ps), fields);
for (SolrDocument d: sdl) {
try {queue.put(d);} catch (final InterruptedException e) {break;}
count++;
}
if (sdl.size() < ps) {
//System.out.println("sdl.size() = " + sdl.size() + ", pagesize = " + pagesize);
break loop; // finished
}
o += sdl.size();
retry = 0;
} catch (final SolrException | IOException e) {
ConcurrentLog.logException(e);
if (retry++ < maxretries) {
// remote Solr may be temporary down, so we wait a bit
try {Thread.sleep(100);} catch (InterruptedException e1) {}
continue loop;
}
// fail
ConcurrentLog.severe("AbstractSolrConnector", "aborted concurrentDocumentsByQueryNoPrefetch after " + maxretries + " retries: " + e.getMessage());
break;
}
return () -> {
long remainingTime = endtime - System.currentTimeMillis();
try {
for (final String querystring: querystrings) {
Thread.currentThread().setName("AbstractSolrConnector:concurrentDocumentsByQueryNoPrefetch(" + querystring + ")");
int o = offset;
int count = 0;
int retry = 0;
loop: while (remainingTime > 0 && count < maxcount) {
try {
final SolrDocumentList sdl = getDocumentListByQuery(querystring, sort, o, Math.min(maxcount, ps), fields);
for (final SolrDocument d: sdl) {
if (endtime != Long.MAX_VALUE) {
/*
* A timeout is defined : we must not use here queue.put() otherwise this
* thread could indefinitely wait here when the queue is full and the
* consumer thread has stopped taking in the queue.
*/
if (!queue.offer(d, remainingTime, TimeUnit.MILLISECONDS)) {
break;
}
} else {
queue.put(d);
}
count++;
}
if (sdl.size() < ps) {
break loop; // finished
}
o += sdl.size();
retry = 0;
} catch(final InterruptedIOException e) {
throw new InterruptedException(); // rethrow to finish the process
} catch (final SolrException | IOException e) {
ConcurrentLog.logException(e);
if (retry++ < maxretries) {
// remote Solr may be temporary down, so we wait a bit
Thread.sleep(100);
continue loop;
}
// fail
ConcurrentLog.severe("AbstractSolrConnector", "aborted concurrentDocumentsByQueryNoPrefetch after " + maxretries + " retries: " + e.getMessage());
break;
}
}
} catch (Throwable e) {} finally {
for (int i = 0; i < Math.max(1, concurrency); i++) {
try {queue.put(AbstractSolrConnector.POISON_DOCUMENT);} catch (final InterruptedException e1) {}
remainingTime = endtime - System.currentTimeMillis();
}
}
} catch(final InterruptedException e) {
Thread.currentThread().interrupt(); // preserve interrupted thread state
} catch (final RuntimeException e) {
ConcurrentLog.logException(e);
} finally {
/* Add poison elements only when the thread has not been interrupted */
for (int i = 0; i < Math.max(1, concurrency); i++) {
try {
queue.put(AbstractSolrConnector.POISON_DOCUMENT);
} catch (final InterruptedException e1) {
Thread.currentThread().interrupt(); // preserve interrupted thread state
break; // thread is interrupted : in that case we no more try to add poison elements to the queue
}
}
}
};
t.start();
return queue;
}
/**

@ -224,9 +224,11 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
public LinkedHashMap<String, ReversibleScoreMap<String>> getFacets(String query, int maxresults, final String ... fields) throws IOException;
/**
* Get results from a solr query as a stream of documents.
* <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
* 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,
* 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
* @param offset first result offset
@ -249,6 +251,27 @@ public interface SolrConnector extends Iterable<String> /* Iterable of document
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
* @param offset the results offset position for each query
* @param maxcount the maximum number of documents per query to retrieve
* @param maxtime the total maximum time to spend. Unlimited when the value
* is negative or equals to Long.MAX_VALUE
* @param buffersize this is the maximum size of a page of results to retrieve
* in one step when running a query
* @param concurrency the number of consuming threads
* @param fields the indexed fields to retrieve
* @return a ready to run task
*/
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.

@ -21,11 +21,14 @@
package net.yacy.search.schema;
import java.net.MalformedURLException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import net.yacy.cora.document.id.DigestURL;
@ -62,7 +65,11 @@ public class HyperlinkGraph implements Iterable<HyperlinkEdge> {
if (hostname.startsWith("www.")) hostname = hostname.substring(4);
StringBuilder q = new StringBuilder();
q.append(CollectionSchema.host_s.getSolrFieldName()).append(':').append(hostname).append(" OR ").append(CollectionSchema.host_s.getSolrFieldName()).append(':').append("www.").append(hostname);
BlockingQueue<SolrDocument> docs = solrConnector.concurrentDocumentsByQuery(q.toString(), CollectionSchema.url_chars_i.getSolrFieldName() + " asc", 0, maxnodes, maxtime, 100, 1, true,
final int pageSize = 100;
final BlockingQueue<SolrDocument> docs = new ArrayBlockingQueue<>(pageSize);
final List<String> queries = new ArrayList<>();
queries.add(q.toString());
final Thread solrQueryTask = new Thread(solrConnector.newDocumentsByQueriesTask(docs, queries, CollectionSchema.url_chars_i.getSolrFieldName() + " asc", 0, maxnodes, maxtime, pageSize, 1,
CollectionSchema.id.getSolrFieldName(),
CollectionSchema.sku.getSolrFieldName(),
CollectionSchema.failreason_s.getSolrFieldName(),
@ -71,7 +78,8 @@ public class HyperlinkGraph implements Iterable<HyperlinkEdge> {
CollectionSchema.inboundlinks_urlstub_sxt.getSolrFieldName(),
CollectionSchema.outboundlinks_protocol_sxt.getSolrFieldName(),
CollectionSchema.outboundlinks_urlstub_sxt.getSolrFieldName()
);
));
solrQueryTask.start();
SolrDocument doc;
Map<String, FailType> errorDocs = new HashMap<String, FailType>();
HyperlinkEdges inboundEdges = new HyperlinkEdges();
@ -80,7 +88,12 @@ public class HyperlinkGraph implements Iterable<HyperlinkEdge> {
try {
retrieval: while ((doc = docs.take()) != AbstractSolrConnector.POISON_DOCUMENT) {
String u = (String) doc.getFieldValue(CollectionSchema.sku.getSolrFieldName());
MultiProtocolURL from = new MultiProtocolURL(u);
MultiProtocolURL from;
try {
from = new MultiProtocolURL(u);
} catch (final MalformedURLException e1) {
continue;
}
String errortype = (String) doc.getFieldValue(CollectionSchema.failtype_s.getSolrFieldName());
FailType error = errortype == null ? null : FailType.valueOf(errortype);
if (error != null) {
@ -94,7 +107,9 @@ public class HyperlinkGraph implements Iterable<HyperlinkEdge> {
HyperlinkEdge.Target linkurl = new HyperlinkEdge.Target(link, HyperlinkType.Inbound);
inboundEdges.addEdge(from, linkurl);
if (stopURL != null && linkurl.equals(stopURL)) break retrieval;
} catch (MalformedURLException e) {}
} catch (final MalformedURLException e) {
/* Continue on the next link */
}
}
links = URIMetadataNode.getLinks(doc, false); // outbound
while (links.hasNext()) {
@ -103,42 +118,49 @@ public class HyperlinkGraph implements Iterable<HyperlinkEdge> {
HyperlinkEdge.Target linkurl = new HyperlinkEdge.Target(link, HyperlinkType.Outbound);
outboundEdges.addEdge(from, linkurl);
if (stopURL != null && linkurl.equals(stopURL)) break retrieval;
} catch (MalformedURLException e) {}
} catch (final MalformedURLException e) {
/* Continue on the next link */
}
}
}
if (inboundEdges.size() + outboundEdges.size() > maxnodes) {
break retrieval;
}
}
} catch (InterruptedException e) {
} catch (MalformedURLException e) {
} catch (final InterruptedException e) {
Thread.currentThread().interrupt(); // preserve interrupted thread state
} finally {
/* Ensure termination and proper resources release of the query thread */
solrQueryTask.interrupt();
}
// we use the errorDocs to mark all edges with endpoint to error documents
Iterator<HyperlinkEdge> i = inboundEdges.iterator();
HyperlinkEdge edge;
while (i.hasNext()) {
edge = i.next();
if (errorDocs.containsKey(edge.target.toNormalform(true))) {
i.remove();
edge.target.type = HyperlinkType.Dead;
errorEdges.add(edge);
}
}
i = outboundEdges.iterator();
while (i.hasNext()) {
edge = i.next();
if (errorDocs.containsKey(edge.target.toNormalform(true))) {
i.remove();
edge.target.type = HyperlinkType.Dead;
errorEdges.add(edge);
}
if(!Thread.currentThread().isInterrupted()) {
// we use the errorDocs to mark all edges with endpoint to error documents
Iterator<HyperlinkEdge> i = inboundEdges.iterator();
HyperlinkEdge edge;
while (i.hasNext()) {
edge = i.next();
if (errorDocs.containsKey(edge.target.toNormalform(true))) {
i.remove();
edge.target.type = HyperlinkType.Dead;
errorEdges.add(edge);
}
}
i = outboundEdges.iterator();
while (i.hasNext()) {
edge = i.next();
if (errorDocs.containsKey(edge.target.toNormalform(true))) {
i.remove();
edge.target.type = HyperlinkType.Dead;
errorEdges.add(edge);
}
}
// we put all edges together in a specific order which is used to create nodes in a svg display:
// notes that appear first are possible painted over by nodes coming later.
// less important nodes shall appear therefore first
this.edges.addAll(outboundEdges);
this.edges.addAll(inboundEdges);
this.edges.addAll(errorEdges);
}
// we put all edges together in a specific order which is used to create nodes in a svg display:
// notes that appear first are possible painted over by nodes coming later.
// less important nodes shall appear therefore first
this.edges.addAll(outboundEdges);
this.edges.addAll(inboundEdges);
this.edges.addAll(errorEdges);
}
public void path(final Segment segment, DigestURL from, DigestURL to, final int maxtime, final int maxnodes) {

Loading…
Cancel
Save