summaryrefslogtreecommitdiffstats
path: root/container-search/src/main/java/com
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2019-03-07 18:00:38 +0100
committerHenning Baldersheim <balder@yahoo-inc.com>2019-03-07 18:04:32 +0100
commite43b67aaa104fe87b2077f19ae8a0e744e177603 (patch)
tree7ede786a3a34b02f1694946f0eb84a8bfdc7ce1a /container-search/src/main/java/com
parentbb9e49ed647fdb06b7daa46ab1d981040c5295ce (diff)
Remove the packet cache. It will very soon loose its value as we are changing the protocol.
And also as it does not work for grouping I think it is better to just let it go. We might want a cache sometime later, but then we should redesign it.
Diffstat (limited to 'container-search/src/main/java/com')
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java22
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheControl.java119
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheKey.java81
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheParams.java26
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4FillInvoker.java71
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4SearchInvoker.java23
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/FastHit.java5
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/FastSearcher.java15
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/PacketCache.java187
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/PacketWrapper.java319
-rw-r--r--container-search/src/main/java/com/yahoo/prelude/fastsearch/VespaBackEndSearcher.java124
-rw-r--r--container-search/src/main/java/com/yahoo/search/dispatch/InterleavedSearchInvoker.java5
-rw-r--r--container-search/src/main/java/com/yahoo/search/dispatch/RpcResourcePool.java3
-rw-r--r--container-search/src/main/java/com/yahoo/search/dispatch/SearchErrorInvoker.java3
-rw-r--r--container-search/src/main/java/com/yahoo/search/dispatch/SearchInvoker.java7
-rw-r--r--container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java3
16 files changed, 28 insertions, 985 deletions
diff --git a/container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java b/container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java
index 92b6eef906e..c9a156faede 100644
--- a/container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/cluster/ClusterSearcher.java
@@ -17,8 +17,6 @@ import com.yahoo.prelude.fastsearch.FS4ResourcePool;
import com.yahoo.prelude.IndexFacts;
import com.yahoo.prelude.Ping;
import com.yahoo.prelude.Pong;
-import com.yahoo.prelude.fastsearch.CacheControl;
-import com.yahoo.prelude.fastsearch.CacheParams;
import com.yahoo.prelude.fastsearch.ClusterParams;
import com.yahoo.prelude.fastsearch.DocumentdbInfoConfig;
import com.yahoo.prelude.fastsearch.FastSearcher;
@@ -127,7 +125,6 @@ public class ClusterSearcher extends Searcher {
maxQueryCacheTimeout = ParameterParser.asMilliSeconds(clusterConfig.maxQueryCacheTimeout(),
DEFAULT_MAX_QUERY_CACHE_TIMEOUT);
- CacheParams cacheParams = new CacheParams(createCache(clusterConfig, clusterModelName));
SummaryParameters docSumParams = new SummaryParameters(qrsConfig
.com().yahoo().prelude().fastsearch().FastSearcher().docsum()
.defaultclass());
@@ -144,7 +141,7 @@ public class ClusterSearcher extends Searcher {
if (searchClusterConfig.indexingmode() == STREAMING) {
VdsStreamingSearcher searcher = vdsCluster(fs4ResourcePool.getServerId(),
searchClusterIndex,
- searchClusterConfig, cacheParams, docSumParams,
+ searchClusterConfig, docSumParams,
documentDbConfig);
addBackendSearcher(searcher);
} else {
@@ -153,8 +150,7 @@ public class ClusterSearcher extends Searcher {
try {
if ( ! isRemote(searchClusterConfig.dispatcher(dispatcherIndex).host())) {
Backend dispatchBackend = createBackend(searchClusterConfig.dispatcher(dispatcherIndex));
- FastSearcher searcher = searchDispatch(searchClusterIndex, fs4ResourcePool,
- cacheParams, docSumParams,
+ FastSearcher searcher = searchDispatch(searchClusterIndex, fs4ResourcePool, docSumParams,
documentDbConfig, dispatchBackend, dispatcher, dispatcherIndex);
addBackendSearcher(searcher);
}
@@ -196,7 +192,6 @@ public class ClusterSearcher extends Searcher {
private static FastSearcher searchDispatch(int searchclusterIndex,
FS4ResourcePool fs4ResourcePool,
- CacheParams cacheParams,
SummaryParameters docSumParams,
DocumentdbInfoConfig documentdbInfoConfig,
Backend backend,
@@ -204,14 +199,13 @@ public class ClusterSearcher extends Searcher {
int dispatcherIndex) {
ClusterParams clusterParams = makeClusterParams(searchclusterIndex,
dispatcherIndex);
- return new FastSearcher(backend, fs4ResourcePool, dispatcher, docSumParams, clusterParams, cacheParams,
+ return new FastSearcher(backend, fs4ResourcePool, dispatcher, docSumParams, clusterParams,
documentdbInfoConfig);
}
private static VdsStreamingSearcher vdsCluster(String serverId,
int searchclusterIndex,
QrSearchersConfig.Searchcluster searchClusterConfig,
- CacheParams cacheParams,
SummaryParameters docSumParams,
DocumentdbInfoConfig documentdbInfoConfig)
{
@@ -224,7 +218,7 @@ public class ClusterSearcher extends Searcher {
searcher.setSearchClusterConfigId(searchClusterConfig.rankprofiles().configid());
searcher.setDocumentType(searchClusterConfig.searchdef(0));
searcher.setStorageClusterRouteSpec(searchClusterConfig.storagecluster().routespec());
- searcher.init(serverId, docSumParams, clusterParams, cacheParams, documentdbInfoConfig);
+ searcher.init(serverId, docSumParams, clusterParams, documentdbInfoConfig);
return searcher;
}
@@ -244,14 +238,6 @@ public class ClusterSearcher extends Searcher {
return fs4ResourcePool.getBackend(disp.host(), disp.port());
}
- private static CacheControl createCache(ClusterConfig config, String clusterModelName) {
- log.log(Level.INFO, "Enabling cache for search cluster "
- + clusterModelName + " (size=" + config.cacheSize()
- + ", timeout=" + config.cacheTimeout() + ")");
-
- return new CacheControl(config.cacheSize(), config.cacheTimeout());
- }
-
ClusterMonitor getMonitor() {
return monitor;
}
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheControl.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheControl.java
deleted file mode 100644
index dd5b3caf0c5..00000000000
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheControl.java
+++ /dev/null
@@ -1,119 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.prelude.fastsearch;
-
-
-import com.yahoo.fs4.Packet;
-import com.yahoo.fs4.QueryPacket;
-import com.yahoo.fs4.QueryResultPacket;
-import com.yahoo.search.Query;
-import com.yahoo.processing.request.CompoundName;
-
-import java.util.Optional;
-
-
-/**
- * The cache control logic for FastSearcher
- *
- * @author Steinar Knutsen
- */
-public class CacheControl {
-
- public static final CompoundName nocachewrite=new CompoundName("nocachewrite");
-
- /** Whether this CacheControl actually should cache hits at all. */
- private final boolean activeCache;
-
- /** Direct unsychronized cache access */
- private final PacketCache packetCache;
-
- public CacheControl(int sizeMegaBytes, double cacheTimeOutSeconds) {
- activeCache = sizeMegaBytes > 0 && cacheTimeOutSeconds > 0.0d;
- if (activeCache) {
- packetCache = new PacketCache(sizeMegaBytes, 0, cacheTimeOutSeconds);
- } else {
- packetCache = null;
- }
- }
-
- /** Returns the capacity of the packet cache in megabytes */
- public final int capacity() {
- return packetCache.getCapacity();
- }
-
- public final boolean useCache(Query query) {
- return (activeCache && !query.getNoCache());
- }
-
- public final PacketWrapper lookup(CacheKey key, Query query) {
- if ((key != null) && useCache(query)) {
- long now = System.currentTimeMillis();
- synchronized (packetCache) {
- return packetCache.get(key, now);
- }
- }
- return null;
- }
-
- // updates first phase in multi phase search
- void updateCacheEntry(CacheKey key, Query query, QueryResultPacket resultPacket) {
- long oldTimestamp;
- if (!activeCache) return;
-
- PacketWrapper wrapper = lookup(key, query);
- if (wrapper == null) return;
-
- // The timestamp is owned by the QueryResultPacket, this is why this
- // update method puts entries into the cache differently from elsewhere
- oldTimestamp = wrapper.getTimestamp();
- wrapper = (PacketWrapper) wrapper.clone();
- wrapper.addResultPacket(resultPacket);
- synchronized (packetCache) {
- packetCache.put(key, wrapper, oldTimestamp);
- }
- }
-
- // updates phases after first phase phase in multi phase search
- void updateCacheEntry(CacheKey key, Query query, DocsumPacketKey[] packetKeys, Packet[] packets) {
- if (!activeCache) return;
-
- PacketWrapper wrapper = lookup(key, query);
- if (wrapper== null) return;
-
- wrapper = (PacketWrapper) wrapper.clone();
- wrapper.addDocsums(packetKeys, packets);
- synchronized (packetCache) {
- packetCache.put(key, wrapper, wrapper.getTimestamp());
- }
- }
-
- void cache(CacheKey key, Query query, DocsumPacketKey[] packetKeys, Packet[] packets, Optional<Integer> distributionKey) {
- if ( ! activeCache) return;
-
- if (query.getNoCache()) return;
- if (query.properties().getBoolean(nocachewrite)) return;
-
- PacketWrapper wrapper = lookup(key, query);
- if (wrapper == null) {
- wrapper = new PacketWrapper(key, packetKeys, packets, distributionKey);
- long now = System.currentTimeMillis();
- synchronized (packetCache) {
- packetCache.put(key, wrapper, now);
- }
- } else {
- wrapper = (PacketWrapper) wrapper.clone();
- wrapper.addResultPacket((QueryResultPacket) packets[0]);
- wrapper.addDocsums(packetKeys, packets, 1);
- synchronized (packetCache) {
- packetCache.put(key, wrapper, wrapper.getTimestamp());
- }
- }
- }
-
- /** Test method. */
- public void clear() {
- if (packetCache != null) {
- packetCache.clear();
- }
- }
-
-}
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheKey.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheKey.java
deleted file mode 100644
index 6af0d181695..00000000000
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheKey.java
+++ /dev/null
@@ -1,81 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.prelude.fastsearch;
-
-
-import java.util.Arrays;
-
-import com.yahoo.collections.BobHash;
-import com.yahoo.fs4.QueryPacket;
-
-
-/**
- * The key used in the packet cache.
- *
- * @author <a href="mailto:steinar@yahoo-inc.com">Steinar Knutsen</a>
- */
-public class CacheKey {
- final private int hashCode;
- final private byte[] serialized;
-
- /**
- * Create a cache key from the query packet.
- */
- public CacheKey(QueryPacket queryPacket) {
- if (!queryPacket.isEncoded()) {
- queryPacket.allocateAndEncode(0);
- }
- this.serialized = queryPacket.getOpaqueCacheKey();
- hashCode = calculateHashCode();
- }
-
- private int calculateHashCode() {
- return BobHash.hash(serialized, 0);
- }
-
- public boolean equals(Object o) {
- if (o == null) {
- return false;
- }
- if (!(o instanceof CacheKey)) {
- return false;
- }
-
- CacheKey k = (CacheKey) o;
- return Arrays.equals(serialized, k.serialized);
- // // The following is used for detailed debugging
- // boolean state = true;
- // if (serialized.length != k.serialized.length) {
- // System.out.println("this " + serialized.length + " other " + k.serialized.length);
- // return false;
- // }
- // System.out.println("start of arrays");
- // for (int i = 0; i < serialized.length; ++i) {
- // System.out.print("serialized " + serialized[i] + " " + k.serialized[i]);
- // if (serialized[i] != k.serialized[i]) {
- // System.out.println(" diff at index " + i);
- // state = false; // want to see all the data
- // } else {
- // System.out.println("");
- // }
- // }
- // return state;
- }
-
- public int hashCode() {
- return hashCode;
- }
-
- public byte[] getCopyOfFullKey() {
- return Arrays.copyOf(serialized, serialized.length);
- }
-
- /**
- * Return an estimate of the memory used by this object. Ie the sum of
- * the internal data fields.
- */
- public int byteSize() {
- // 4 = sizeOf(hashCode)
- return serialized.length + 4;
- }
-
-}
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheParams.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheParams.java
deleted file mode 100644
index cad055d3512..00000000000
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/CacheParams.java
+++ /dev/null
@@ -1,26 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.prelude.fastsearch;
-
-
-/**
- * Helper class for carrying around cache-related
- * config parameters to the FastSearcher class.
- *
- * @author arnej27959
- */
-public class CacheParams {
-
- public int cacheMegaBytes = 0;
- public double cacheTimeOutSeconds = 0;
- public CacheControl cacheControl = null;
-
- public CacheParams(int megabytes, double timeoutseconds) {
- this.cacheMegaBytes = megabytes;
- this.cacheTimeOutSeconds = timeoutseconds;
- }
-
- public CacheParams(CacheControl cacheControl) {
- this.cacheControl = cacheControl;
- }
-
-}
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4FillInvoker.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4FillInvoker.java
index 7453af55ec0..190a7905476 100644
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4FillInvoker.java
+++ b/container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4FillInvoker.java
@@ -3,10 +3,8 @@ package com.yahoo.prelude.fastsearch;
import com.yahoo.fs4.BasicPacket;
import com.yahoo.fs4.ChannelTimeoutException;
-import com.yahoo.fs4.DocsumPacket;
import com.yahoo.fs4.GetDocSumsPacket;
import com.yahoo.fs4.Packet;
-import com.yahoo.fs4.QueryPacket;
import com.yahoo.fs4.mplex.Backend;
import com.yahoo.fs4.mplex.FS4Channel;
import com.yahoo.fs4.mplex.InvalidChannelException;
@@ -16,7 +14,6 @@ import com.yahoo.search.Result;
import com.yahoo.search.dispatch.FillInvoker;
import com.yahoo.search.result.ErrorMessage;
import com.yahoo.search.result.Hit;
-import com.yahoo.search.result.HitGroup;
import java.io.IOException;
import java.util.Iterator;
@@ -30,16 +27,12 @@ import static com.yahoo.prelude.fastsearch.VespaBackEndSearcher.hitIterator;
*/
public class FS4FillInvoker extends FillInvoker {
- private final String serverId;
private final VespaBackEndSearcher searcher;
private FS4Channel channel;
private int expectedFillResults = 0;
- private CacheKey summaryCacheKey = null;
- private DocsumPacketKey[] summaryPacketKeys = null;
public FS4FillInvoker(VespaBackEndSearcher searcher, Query query, FS4ResourcePool fs4ResourcePool, String hostname, int port) {
- this.serverId = fs4ResourcePool.getServerId();
this.searcher = searcher;
Backend backend = fs4ResourcePool.getBackend(hostname, port);
this.channel = backend.openChannel();
@@ -47,8 +40,7 @@ public class FS4FillInvoker extends FillInvoker {
}
// fdispatch code path
- public FS4FillInvoker(String serverId, VespaBackEndSearcher searcher, Query query, Backend backend) {
- this.serverId = serverId;
+ public FS4FillInvoker(VespaBackEndSearcher searcher, Query query, Backend backend) {
this.searcher = searcher;
this.channel = backend.openChannel();
channel.setQuery(query);
@@ -56,21 +48,9 @@ public class FS4FillInvoker extends FillInvoker {
@Override
protected void sendFillRequest(Result result, String summaryClass) {
- summaryCacheKey = null;
- if (searcher.getCacheControl().useCache(channel.getQuery())) {
- summaryCacheKey = fetchCacheKeyFromHits(result.hits(), summaryClass);
- if (summaryCacheKey == null) {
- QueryPacket queryPacket = QueryPacket.create(serverId, channel.getQuery());
- summaryCacheKey = new CacheKey(queryPacket);
- }
- boolean cacheFound = cacheLookupTwoPhase(summaryCacheKey, result, summaryClass);
- if (!cacheFound) {
- summaryCacheKey = null;
- }
- }
if (countFastHits(result) > 0) {
- summaryPacketKeys = getPacketKeys(result, summaryClass);
+ DocsumPacketKey[] summaryPacketKeys = getPacketKeys(result, summaryClass);
if (summaryPacketKeys.length == 0) {
expectedFillResults = 0;
} else {
@@ -130,9 +110,6 @@ public class FS4FillInvoker extends FillInvoker {
"Error filling hits with summary fields, source: " + getName() + " Exception thrown: " + e.getMessage()));
return;
}
- if (skippedHits == 0 && summaryCacheKey != null) {
- searcher.getCacheControl().updateCacheEntry(summaryCacheKey, channel.getQuery(), summaryPacketKeys, receivedPackets);
- }
if (skippedHits > 0)
result.hits().addError(
@@ -163,50 +140,6 @@ public class FS4FillInvoker extends FillInvoker {
}
}
- private boolean cacheLookupTwoPhase(CacheKey cacheKey, Result result, String summaryClass) {
- Query query = result.getQuery();
- PacketWrapper packetWrapper = searcher.getCacheControl().lookup(cacheKey, query);
-
- if (packetWrapper == null) {
- return false;
- }
- if (packetWrapper.getNumPackets() != 0) {
- for (Iterator<Hit> i = hitIterator(result); i.hasNext();) {
- Hit hit = i.next();
-
- if (hit instanceof FastHit) {
- FastHit fastHit = (FastHit) hit;
- DocsumPacketKey key = new DocsumPacketKey(fastHit.getGlobalId(), fastHit.getPartId(), summaryClass);
-
- if (searcher.fillHit(fastHit, (DocsumPacket) packetWrapper.getPacket(key), summaryClass).ok) {
- fastHit.setCached(true);
- }
-
- }
- }
- result.hits().setSorted(false);
- result.analyzeHits();
- }
-
- return true;
- }
-
- private CacheKey fetchCacheKeyFromHits(HitGroup hits, String summaryClass) {
- for (Iterator<Hit> i = hits.unorderedDeepIterator(); i.hasNext();) {
- Hit h = i.next();
- if (h instanceof FastHit) {
- FastHit hit = (FastHit) h;
- if (hit.isFilled(summaryClass)) {
- continue;
- }
- if (hit.getCacheKey() != null) {
- return hit.getCacheKey();
- }
- }
- }
- return null;
- }
-
private int countFastHits(Result result) {
int count = 0;
for (Iterator<Hit> i = hitIterator(result); i.hasNext();) {
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4SearchInvoker.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4SearchInvoker.java
index 856de8d33ef..d2910ba3fbc 100644
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4SearchInvoker.java
+++ b/container-search/src/main/java/com/yahoo/prelude/fastsearch/FS4SearchInvoker.java
@@ -3,7 +3,6 @@ package com.yahoo.prelude.fastsearch;
import com.yahoo.fs4.BasicPacket;
import com.yahoo.fs4.ChannelTimeoutException;
-import com.yahoo.fs4.Packet;
import com.yahoo.fs4.QueryPacket;
import com.yahoo.fs4.QueryResultPacket;
import com.yahoo.fs4.mplex.FS4Channel;
@@ -76,7 +75,7 @@ public class FS4SearchInvoker extends SearchInvoker implements ResponseMonitor<F
}
@Override
- protected Result getSearchResult(CacheKey cacheKey, Execution execution) throws IOException {
+ protected Result getSearchResult(Execution execution) throws IOException {
if (pendingSearchError != null) {
return errorResult(pendingSearchError);
}
@@ -110,24 +109,8 @@ public class FS4SearchInvoker extends SearchInvoker implements ResponseMonitor<F
searcher.addMetaInfo(query, queryPacket.getQueryPacketData(), resultPacket, result);
- searcher.addUnfilledHits(result, resultPacket.getDocuments(), false, queryPacket.getQueryPacketData(), cacheKey, distributionKey());
- Packet[] packets;
- CacheControl cacheControl = searcher.getCacheControl();
- PacketWrapper packetWrapper = cacheControl.lookup(cacheKey, query);
-
- if (packetWrapper != null) {
- cacheControl.updateCacheEntry(cacheKey, query, resultPacket);
- } else {
- if (resultPacket.getCoverageFeature() && !resultPacket.getCoverageFull()) {
- // Don't add error here, it was done in first phase
- // No check if packetWrapper already exists, since incomplete
- // first phase data won't be cached anyway.
- } else {
- packets = new Packet[1];
- packets[0] = resultPacket;
- cacheControl.cache(cacheKey, query, new DocsumPacketKey[0], packets, distributionKey());
- }
- }
+ searcher.addUnfilledHits(result, resultPacket.getDocuments(), queryPacket.getQueryPacketData(), distributionKey());
+
return result;
}
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/FastHit.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/FastHit.java
index 5433d37c66e..3d6c094c784 100644
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/FastHit.java
+++ b/container-search/src/main/java/com/yahoo/prelude/fastsearch/FastHit.java
@@ -44,7 +44,6 @@ public class FastHit extends Hit {
private URI indexUri = null;
private transient QueryPacketData queryPacketData = null;
- private transient CacheKey cacheKey = null;
/**
* Summaries added to this hit which are not yet decoded into fields.
@@ -149,10 +148,6 @@ public class FastHit extends Hit {
/** Returns a serial encoding of the query which produced this hit, ot null if not available. */
public QueryPacketData getQueryPacketData() { return queryPacketData; }
- CacheKey getCacheKey() { return cacheKey; }
-
- void setCacheKey(CacheKey cacheKey) { this.cacheKey = cacheKey; }
-
/** For internal use */
public void addSummary(DocsumDefinition docsumDef, Inspector value) {
if (removedFields != null)
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/FastSearcher.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/FastSearcher.java
index 27b479decd5..ac02b5f8e5e 100644
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/FastSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/fastsearch/FastSearcher.java
@@ -69,13 +69,12 @@ public class FastSearcher extends VespaBackEndSearcher {
* backend.
* @param docSumParams document summary parameters
* @param clusterParams the cluster number, and other cluster backend parameters
- * @param cacheParams the size, lifetime, and controller of our cache
* @param documentdbInfoConfig document database parameters
*/
- public FastSearcher(Backend dispatchBackend, FS4ResourcePool fs4ResourcePool,
- Dispatcher dispatcher, SummaryParameters docSumParams, ClusterParams clusterParams,
- CacheParams cacheParams, DocumentdbInfoConfig documentdbInfoConfig) {
- init(fs4ResourcePool.getServerId(), docSumParams, clusterParams, cacheParams, documentdbInfoConfig);
+ public FastSearcher(Backend dispatchBackend, FS4ResourcePool fs4ResourcePool, Dispatcher dispatcher,
+ SummaryParameters docSumParams, ClusterParams clusterParams,
+ DocumentdbInfoConfig documentdbInfoConfig) {
+ init(fs4ResourcePool.getServerId(), docSumParams, clusterParams, documentdbInfoConfig);
this.dispatchBackend = dispatchBackend;
this.dispatcher = dispatcher;
this.fs4InvokerFactory = new FS4InvokerFactory(fs4ResourcePool, dispatcher.searchCluster(), this);
@@ -146,11 +145,11 @@ public class FastSearcher extends VespaBackEndSearcher {
}
@Override
- public Result doSearch2(Query query, QueryPacket queryPacket, CacheKey cacheKey, Execution execution) {
+ public Result doSearch2(Query query, QueryPacket queryPacket, Execution execution) {
if (dispatcher.searchCluster().groupSize() == 1)
forceSinglePassGrouping(query);
try(SearchInvoker invoker = getSearchInvoker(query)) {
- Result result = invoker.search(query, queryPacket, cacheKey, execution);
+ Result result = invoker.search(query, queryPacket, execution);
if (query.properties().getBoolean(Ranking.RANKFEATURES, false)) {
// There is currently no correct choice for which
@@ -238,7 +237,7 @@ public class FastSearcher extends VespaBackEndSearcher {
if (direct.isPresent()) {
return fs4InvokerFactory.getFillInvoker(query, direct.get());
}
- return new FS4FillInvoker(getServerId(), this, query, dispatchBackend);
+ return new FS4FillInvoker(this, query, dispatchBackend);
}
/**
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/PacketCache.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/PacketCache.java
deleted file mode 100644
index ce3ccb1d1b8..00000000000
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/PacketCache.java
+++ /dev/null
@@ -1,187 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.prelude.fastsearch;
-
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.logging.Logger;
-
-import com.yahoo.log.LogLevel;
-
-
-/**
- * An LRU cache using number of hits cached inside the results as
- * size limiting factor. Directly modelled after com.yahoo.collections.Cache.
- *
- * @author Steinar Knutsen
- * @author bratseth
- */
-// TODO: Remove packet cache as it timed out a long time ago.
-// 1 - It does not work with grouping, 2 the packet protocol is eroding away.
-public class PacketCache extends LinkedHashMap<CacheKey, PacketWrapper> {
-
- private static final long serialVersionUID = -7403077211906108356L;
-
- /** The <i>current</i> number of bytes of packets in this cache */
- private int totalSize;
-
- /** The maximum number of bytes of packets in this cache */
- private final int capacity;
-
- /** The max size of a cached item compared to the total size */
- private int maxCacheItemPercentage = 1;
-
- /** The max age for a valid cache entry, 0 mean infinite */
- private final long maxAge;
-
- private static final Logger log = Logger.getLogger(PacketCache.class.getName());
-
- public void clear() {
- super.clear();
- totalSize = 0;
- }
-
- /**
- * Sets the max size of a cached item compared to the total size
- * Cache requests for larger objects will be ignored
- */
- public void setMaxCacheItemPercentage(int maxCapacityPercentage) {
- maxCacheItemPercentage = maxCapacityPercentage;
- }
-
- /**
- * Creates a cache with a size given by
- * cachesizemegabytes*2^20+cachesizebytes
- *
- * @param capacityMegaBytes the cache size, measured in megabytes
- * @param capacityBytes additional number of bytes to add to the cache size
- * @param maxAge seconds a cache entry is valid, 0 or less are illegal arguments
- */
- public PacketCache(int capacityMegaBytes,int capacityBytes,double maxAge) {
- // hardcoded inital entry capacity, won't matter much anyway
- // after a while
- super(12500, 1.0f, true);
- if (maxAge <= 0.0d) {
- throw new IllegalArgumentException("maxAge <= 0 not legal on 5.1, use some very large number for no timeout.");
- }
- if (capacityMegaBytes > (Integer.MAX_VALUE >> 20)) {
- log.log(LogLevel.INFO, "Packet cache of more than 2 GB requested. Reverting to 2 GB packet cache.");
- this.capacity = Integer.MAX_VALUE;
- } else {
- this.capacity = (capacityMegaBytes << 20) + capacityBytes;
- }
- if (this.capacity <= 0) {
- throw new IllegalArgumentException("Total cache size set to 0 or less bytes. If no caching is desired, avoid creating this object instead.");
- }
- this.maxAge = (long) (maxAge * 1000.0d);
- }
-
- /**
- * Overrides LinkedHashMap.removeEldestEntry as suggested to implement LRU cache.
- */
- protected boolean removeEldestEntry(Map.Entry<CacheKey, PacketWrapper> eldest)
- {
- if (totalSize > capacity) {
- totalSize -= eldest.getValue().getPacketsSize();
- return true;
- }
- return false;
- }
-
- private void removeOverflow() {
- if (totalSize < capacity) return;
-
- for (Iterator<PacketWrapper> i = values().iterator(); i.hasNext();) {
- PacketWrapper eldestEntry = i.next();
- totalSize -= eldestEntry.getPacketsSize();
-
- i.remove();
- if (totalSize < capacity) {
- break;
- }
- }
- }
-
- public int getCapacity() {
- return capacity >> 20;
- }
-
- public int getByteCapacity() {
- return capacity;
- }
-
- /**
- * Adds a PacketWrapper object to this cache,
- * unless the size is more than maxCacheItemPercentage of the total size
- */
- public PacketWrapper put(CacheKey key, PacketWrapper value) {
- return put(key, value, System.currentTimeMillis());
- }
-
- /**
- * Adds a BasicPacket array to this cache,
- * unless the size is more than maxCacheItemPercentage of the total size
- *
- * @param timestamp the timestamp for the first packet in the array,
- * unit milliseconds
- */
- public PacketWrapper put(CacheKey key, PacketWrapper result, long timestamp) {
- int size = result.getPacketsSize();
-
- if (size > 0) {
- result.setTimestamp(timestamp);
- }
-
- // don't insert if it is too big
- if (size * 100 > capacity * maxCacheItemPercentage) {
- // removeField the old one since that is now stale.
- return remove(key);
- }
-
- totalSize += size;
- PacketWrapper previous = super.put(key, result);
- if (previous != null) {
- totalSize -= previous.getPacketsSize();
- }
- if (totalSize > (capacity * 1.1)) {
- removeOverflow();
- }
-
- return previous;
- }
-
- public PacketWrapper get(CacheKey key) {
- return get(key, System.currentTimeMillis());
- }
-
- public PacketWrapper get(CacheKey key, long now) {
- PacketWrapper result = super.get(key);
-
- if (result == null) {
- return result;
- }
-
- long timestamp = result.getTimestamp();
-
- if ((now - timestamp) > maxAge) {
- remove(key);
- return null;
- } else {
- return result;
- }
- }
-
- public PacketWrapper remove(CacheKey key) {
- PacketWrapper removed = super.remove(key);
-
- if (removed != null) {
- totalSize -= removed.getPacketsSize();
- }
- return removed;
- }
-
- public int totalPacketSize() {
- return totalSize;
- }
-
-}
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/PacketWrapper.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/PacketWrapper.java
deleted file mode 100644
index 8e22bae430a..00000000000
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/PacketWrapper.java
+++ /dev/null
@@ -1,319 +0,0 @@
-// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-package com.yahoo.prelude.fastsearch;
-
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
-import java.util.logging.Logger;
-
-import com.yahoo.fs4.BasicPacket;
-import com.yahoo.fs4.DocsumPacket;
-import com.yahoo.fs4.DocumentInfo;
-import com.yahoo.fs4.Packet;
-import com.yahoo.fs4.QueryResultPacket;
-import com.yahoo.document.GlobalId;
-import com.yahoo.document.DocumentId;
-
-
-/**
- * A wrapper for cache entries to make it possible to check whether the
- * hits are truly correct.
- *
- * @author Steinar Knutsen
- * @author Mathias Lidal
- */
-// TODO: Remove packet cache as it timed out a long time ago.
-// 1 - It does not work with grouping, 2 the packet protocol is eroding away.
-public class PacketWrapper implements Cloneable {
-
- private static Logger log = Logger.getLogger(PacketWrapper.class.getName());
-
- private final int keySize;
- // associated result packets, sorted in regard to offset
- private ArrayList<BasicPacket> resultPackets = new ArrayList<>(3); // length = "some small number"
-
- private LinkedHashMap<DocsumPacketKey, BasicPacket> packets;
-
- private final Optional<Integer> distributionKey;
-
- private static class ResultPacketComparator<T extends BasicPacket> implements Comparator<T> {
- @Override
- public int compare(T o1, T o2) {
- QueryResultPacket r1 = (QueryResultPacket) o1;
- QueryResultPacket r2 = (QueryResultPacket) o2;
- return r1.getOffset() - r2.getOffset();
- }
- }
-
- private static ResultPacketComparator<BasicPacket> resultPacketComparator = new ResultPacketComparator<>();
-
- public PacketWrapper(CacheKey key, DocsumPacketKey[] packetKeys, BasicPacket[] bpackets, Optional<Integer> distributionKey) {
- // Should not support key == null
- this.keySize = key.byteSize();
- resultPackets.add(bpackets[0]);
- this.packets = new LinkedHashMap<>();
- this.distributionKey = distributionKey;
- Packet[] ppackets = new Packet[packetKeys.length];
-
- for (int i = 0; i < packetKeys.length; i++) {
- ppackets[i] = (Packet) bpackets[i + 1];
- }
- addDocsums(packetKeys, ppackets);
- }
-
- /**
- * Only used by PacketCacheTestCase, should not be used otherwise
- */
- public PacketWrapper(CacheKey key, BasicPacket[] packets) {
- // Should support key == null as this is for testing
- if (key == null) {
- keySize = 0;
- } else {
- this.keySize = key.byteSize();
- }
- resultPackets.add(packets[0]);
- this.packets = new LinkedHashMap<>();
- this.distributionKey = Optional.empty();
- for (int i = 0; i < packets.length - 1; i++) {
- this.packets.put(new DocsumPacketKey(new GlobalId(new DocumentId("doc:test:" + i).getGlobalId()), i, null), packets[i + 1]);
- }
-
- }
-
- public QueryResultPacket getFirstResultPacket() {
- if (resultPackets.size() > 0) {
- return (QueryResultPacket) resultPackets.get(0);
- } else {
- return null;
- }
- }
-
- /**
- * Returns the distribution key of the content node producing these hits,
- * or empty if the hits were returned through dispatch
- */
- public Optional<Integer> distributionKey() { return distributionKey; }
-
- /**
- * Returns the list of documents, null if not all are available
- */
- public List<DocumentInfo> getDocuments(int offset, int hits) {
- // speculatively allocate list for the hits
- List<DocumentInfo> docs = new ArrayList<>(hits);
- int currentOffset = 0;
- QueryResultPacket r = getFirstResultPacket();
- if (offset >= r.getTotalDocumentCount()) {
- // shortcut especially for results with 0 hits
- // >= both necessary for end of result sets and
- // offset == 0 && totalDocumentCount == 0
- return docs;
- }
- for (Iterator<BasicPacket> i = resultPackets.iterator(); i.hasNext();) {
- QueryResultPacket result = (QueryResultPacket) i.next();
- if (result.getOffset() > offset + currentOffset) {
- // we haven't got all the requested document info objects
- return null;
- }
- if (result.getOffset() + result.getDocumentCount()
- <= currentOffset + offset) {
- // no new hits available
- continue;
- }
- List<DocumentInfo> documents = result.getDocuments();
- int packetOffset = (offset + currentOffset) - result.getOffset();
- int afterLastDoc = Math.min(documents.size(), packetOffset + hits);
- for (Iterator<DocumentInfo> j = documents.subList(packetOffset, afterLastDoc).iterator();
- docs.size() < hits && j.hasNext();
- ++currentOffset) {
- docs.add(j.next());
- }
- if (hits == docs.size()
- || offset + docs.size() >= result.getTotalDocumentCount()) {
- // We have the hits we need, or there are no more hits available
- return docs;
- }
- }
- return null;
- }
-
- public void addResultPacket(QueryResultPacket resultPacket) {
- // This function only keeps the internal list sorted according
- // to offset
- int insertionPoint;
- QueryResultPacket r;
-
- if (resultPacket.getDocumentCount() == 0) {
- return; // do not add a packet which does not contain new info
- }
-
- insertionPoint = Collections.binarySearch(resultPackets, resultPacket, resultPacketComparator);
- if (insertionPoint < 0) {
- // new offset
- insertionPoint = ~insertionPoint; // (insertionPoint + 1) * -1;
- resultPackets.add(insertionPoint, resultPacket);
- cleanResultPackets();
- } else {
- // there exists a packet with same offset
- r = (QueryResultPacket) resultPackets.get(insertionPoint);
- if (resultPacket.getDocumentCount() > r.getDocumentCount()) {
- resultPackets.set(insertionPoint, resultPacket);
- cleanResultPackets();
- }
- }
- }
-
- private void cleanResultPackets() {
- int marker;
- QueryResultPacket previous;
- if (resultPackets.size() == 1) {
- return;
- }
-
- // we know the list is sorted with regard to offset
- // First ensure the list grows in regards to lastOffset as well.
- // Could have done this addResultPacket, but this makes the code
- // simpler.
- previous = (QueryResultPacket) resultPackets.get(0);
- for (int i = 1; i < resultPackets.size(); ++i) {
- QueryResultPacket r = (QueryResultPacket) resultPackets.get(i);
- if (r.getOffset() + r.getDocumentCount()
- <= previous.getOffset() + previous.getDocumentCount()) {
- resultPackets.remove(i--);
- } else {
- previous = r;
- }
- }
-
- marker = 0;
- while (marker < (resultPackets.size() - 2)) {
- QueryResultPacket r0 = (QueryResultPacket) resultPackets.get(marker);
- QueryResultPacket r1 = (QueryResultPacket) resultPackets.get(marker + 1);
- QueryResultPacket r2 = (QueryResultPacket) resultPackets.get(marker + 2);
- int nextOffset = r0.getOffset() + r0.getDocumentCount();
-
- if (r1.getOffset() < nextOffset
- && r2.getOffset() <= nextOffset) {
- resultPackets.remove(marker + 1);
- }
- ++marker;
- }
- }
-
- /** Only for testing. */
- public List<BasicPacket> getResultPackets() {
- return resultPackets;
- }
-
- public void addDocsums(DocsumPacketKey[] packetKeys, BasicPacket[] bpackets,
- int offset) {
- Packet[] ppackets = new Packet[packetKeys.length];
-
- for (int i = 0; i < packetKeys.length; i++) {
- ppackets[i] = (Packet) bpackets[i + offset];
- }
- addDocsums(packetKeys, ppackets);
- }
-
- public void addDocsums(DocsumPacketKey[] packetKeys, Packet[] packets) {
- if (packetKeys == null || packets == null) {
- log.warning(
- "addDocsums called with "
- + (packetKeys == null ? "packetKeys == null " : "")
- + (packets == null ? "packets == null" : ""));
- return;
- }
- for (int i = 0; i < packetKeys.length && i < packets.length; i++) {
- if (packetKeys[i] == null) {
- log.warning(
- "addDocsums called, but packetsKeys[" + i + "] is null");
- } else if (packets[i] instanceof DocsumPacket) {
- DocsumPacket dp = (DocsumPacket) packets[i];
-
- if (packetKeys[i].getGlobalId().equals(dp.getGlobalId())
- && dp.getData().length > 0)
- {
- this.packets.put(packetKeys[i], packets[i]);
- log.fine("addDocsums " + i + " globalId: " + dp.getGlobalId());
- } else {
- log.warning("not caching bad Docsum for globalId " + packetKeys[i].getGlobalId() + ": " + dp);
- }
- } else {
- log.warning(
- "addDocsums called, but packets[" + i
- + "] is not a DocsumPacket instance");
- }
- }
- }
-
- public int getNumPackets() {
- return packets.size();
- }
-
- BasicPacket getPacket(GlobalId globalId, int partid, String summaryClass) {
- return getPacket(
- new DocsumPacketKey(globalId, partid, summaryClass));
- }
-
- BasicPacket getPacket(DocsumPacketKey packetKey) {
- return packets.get(packetKey);
- }
-
- long getTimestamp() {
- return getFirstResultPacket().getTimestamp();
- }
-
- public void setTimestamp(long timestamp) {
- getFirstResultPacket().setTimestamp(timestamp);
- }
-
- public int getPacketsSize() {
- int size = 0;
-
- for (Iterator<BasicPacket> i = resultPackets.iterator(); i.hasNext();) {
- QueryResultPacket r = (QueryResultPacket) i.next();
- int l = r.getLength();
-
- if (l < 0) {
- log.warning("resultpacket length " + l);
- l = 10240;
- }
- size += l;
- }
- for (Iterator<BasicPacket> i = packets.values().iterator(); i.hasNext();) {
- BasicPacket packet = i.next();
- int l = packet.getLength();
-
- if (l < 0) {
- log.warning("BasicPacket length " + l);
- l = 10240;
- }
- size += l;
- }
- size += keySize;
- return size;
- }
-
- /**
- * Straightforward shallow copy.
- */
- @SuppressWarnings("unchecked")
- @Override
- public Object clone() {
- try {
- PacketWrapper other = (PacketWrapper) super.clone();
- other.resultPackets = (ArrayList<BasicPacket>) resultPackets.clone();
- if (packets != null) {
- other.packets = (LinkedHashMap<DocsumPacketKey, BasicPacket>) packets.clone();
- }
- return other;
- } catch (CloneNotSupportedException e) {
- throw new RuntimeException("A non-cloneable superclass has been inserted.",
- e);
- }
- }
-
-}
diff --git a/container-search/src/main/java/com/yahoo/prelude/fastsearch/VespaBackEndSearcher.java b/container-search/src/main/java/com/yahoo/prelude/fastsearch/VespaBackEndSearcher.java
index 024dc4a6b06..bee1fab5686 100644
--- a/container-search/src/main/java/com/yahoo/prelude/fastsearch/VespaBackEndSearcher.java
+++ b/container-search/src/main/java/com/yahoo/prelude/fastsearch/VespaBackEndSearcher.java
@@ -12,7 +12,6 @@ import com.yahoo.fs4.QueryPacket;
import com.yahoo.fs4.QueryPacketData;
import com.yahoo.fs4.QueryResultPacket;
import com.yahoo.io.GrowableByteBuffer;
-import com.yahoo.io.HexDump;
import com.yahoo.log.LogLevel;
import com.yahoo.prelude.ConfigurationException;
import com.yahoo.prelude.query.Item;
@@ -33,9 +32,6 @@ import com.yahoo.search.result.Relevance;
import com.yahoo.search.searchchain.Execution;
import com.yahoo.searchlib.aggregation.Grouping;
import com.yahoo.slime.BinaryFormat;
-import com.yahoo.slime.JsonFormat;
-import com.yahoo.slime.Slime;
-import com.yahoo.text.Utf8;
import com.yahoo.vespa.objects.BufferSerializer;
import java.io.IOException;
@@ -80,18 +76,12 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
/** The name of this source */
private String name;
- /** Cache wrapper */
- private CacheControl cacheControl = null;
-
public final String getName() { return name; }
protected final String getDefaultDocsumClass() { return defaultDocsumClass; }
/** Sets default document summary class. Default is null */
private void setDefaultDocsumClass(String docsumClass) { defaultDocsumClass = docsumClass; }
- /** Returns the packet cache controller of this */
- public final CacheControl getCacheControl() { return cacheControl; }
-
public final Logger getLogger() { return super.getLogger(); }
/**
@@ -100,10 +90,9 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
*
* @param query the query to search
* @param queryPacket the serialized query representation to pass to the search cluster
- * @param cacheKey the cache key created from the query packet, or null if caching is not used
* @param execution the query execution context
*/
- protected abstract Result doSearch2(Query query, QueryPacket queryPacket, CacheKey cacheKey, Execution execution);
+ protected abstract Result doSearch2(Query query, QueryPacket queryPacket, Execution execution);
protected abstract void doPartialFill(Result result, String summaryClass);
@@ -131,28 +120,6 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
return false;
}
- private Result cacheLookupFirstPhase(CacheKey key, QueryPacketData queryPacketData, Query query, int offset, int hits, String summaryClass) {
- PacketWrapper packetWrapper = cacheControl.lookup(key, query);
-
- if (packetWrapper == null) return null;
-
- // Check if the cache entry contains the requested hits
- List<DocumentInfo> documents = packetWrapper.getDocuments(offset, hits);
- if (documents == null) return null;
-
- if (query.getPresentation().getSummary() == null)
- query.getPresentation().setSummary(getDefaultDocsumClass());
- Result result = new Result(query);
- QueryResultPacket resultPacket = packetWrapper.getFirstResultPacket();
-
- addMetaInfo(query, queryPacketData, resultPacket, result);
- if (packetWrapper.getNumPackets() == 0)
- addUnfilledHits(result, documents, true, queryPacketData, key, packetWrapper.distributionKey());
- else
- addCachedHits(result, packetWrapper, summaryClass, documents);
- return result;
- }
-
public String getServerId() { return serverId; }
protected DocumentDatabase getDocumentDatabase(Query query) {
@@ -174,7 +141,7 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
}
public final void init(String serverId, SummaryParameters docSumParams, ClusterParams clusterParams,
- CacheParams cacheParams, DocumentdbInfoConfig documentdbInfoConfig) {
+ DocumentdbInfoConfig documentdbInfoConfig) {
this.serverId = serverId;
this.name = clusterParams.searcherName;
@@ -191,12 +158,6 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
documentDbs.put(docDb.name(), db);
}
}
-
- if (cacheParams.cacheControl == null) {
- this.cacheControl = new CacheControl(cacheParams.cacheMegaBytes, cacheParams.cacheTimeOutSeconds);
- } else {
- this.cacheControl = cacheParams.cacheControl;
- }
}
protected void transformQuery(Query query) { }
@@ -230,14 +191,9 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
getLogger().fine("made QueryPacket: " + queryPacket);
Result result = null;
- CacheKey cacheKey = null;
- if (cacheControl.useCache(query)) {
- cacheKey = new CacheKey(queryPacket);
- result = getCached(cacheKey, queryPacket.getQueryPacketData(), query);
- }
if (result == null) {
- result = doSearch2(query, queryPacket, cacheKey, execution);
+ result = doSearch2(query, queryPacket, execution);
if (isLoggingFine())
getLogger().fine("Result NOT retrieved from cache");
@@ -257,34 +213,6 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
return queryPacket;
}
- /**
- * Returns a cached result, or null if no result was cached for this key
- *
- * @param cacheKey the cache key created from the query packet
- * @param queryPacketData a serialization of the query, to avoid having to recompute this, or null if not available
- * @param query the query, used for tracing, lookup of result window and result creation
- */
- private Result getCached(CacheKey cacheKey, QueryPacketData queryPacketData, Query query) {
- if (query.getTraceLevel() >= 6) {
- query.trace("Cache key hash: " + cacheKey.hashCode(), 6);
- if (query.getTraceLevel() >= 8) {
- query.trace("Cache key: " + HexDump.toHexString(cacheKey.getCopyOfFullKey()), 8);
- }
- }
-
- Result result = cacheLookupFirstPhase(cacheKey, queryPacketData, query, query.getOffset(), query.getHits(), query.getPresentation().getSummary());
- if (result == null) return null;
-
- if (isLoggingFine()) {
- getLogger().fine("Result retrieved from cache: " + result);
- }
- if (query.getTraceLevel() >= 1) {
- query.trace(getName() + " cached response: " + result, false, 1);
- }
- result.trace(getName());
- return result;
- }
-
private List<Result> partitionHits(Result result, String summaryClass) {
List<Result> parts = new ArrayList<>();
TinyIdentitySet<Query> queryMap = new TinyIdentitySet<>(4);
@@ -525,46 +453,6 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
return new FillHitsResult(skippedHits, lastError);
}
- private boolean addCachedHits(Result result,
- PacketWrapper packetWrapper,
- String summaryClass,
- List<DocumentInfo> documents) {
- boolean filledAllOfEm = true;
- Query myQuery = result.getQuery();
-
- for (DocumentInfo document : documents) {
- FastHit hit = new FastHit();
- hit.setQuery(myQuery);
-
- hit.setFillable();
- hit.setCached(true);
-
- extractDocumentInfo(hit, document);
-
- DocsumPacket docsum = (DocsumPacket) packetWrapper.getPacket(document.getGlobalId(), document.getPartId(), summaryClass);
-
- if (docsum != null) {
- byte[] docsumdata = docsum.getData();
-
- if (docsumdata.length > 0) {
- String error = decodeSummary(summaryClass, hit, docsumdata);
- if (error != null) {
- filledAllOfEm = false;
- }
- } else {
- filledAllOfEm = false;
- }
- } else {
- filledAllOfEm = false;
- }
-
- result.hits().add(hit);
-
- }
-
- return filledAllOfEm;
- }
-
private void extractDocumentInfo(FastHit hit, DocumentInfo document) {
hit.setSource(getName());
@@ -602,16 +490,13 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
* created from a cache in the current call path.
*
* @param queryPacketData binary data from first phase of search, or null
- * @param cacheKey the key this hit should match in the packet cache, or null
* @param channelDistributionKey distribution key of the node producing these hits.
* Only set if produced directly by a search node, not dispatch
* (in which case it is not set in the received packets.)
*/
void addUnfilledHits(Result result,
List<DocumentInfo> documents,
- boolean fromCache,
QueryPacketData queryPacketData,
- CacheKey cacheKey,
Optional<Integer> channelDistributionKey) {
Query myQuery = result.getQuery();
@@ -622,10 +507,9 @@ public abstract class VespaBackEndSearcher extends PingableSearcher {
hit.setQuery(myQuery);
if (queryPacketData != null)
hit.setQueryPacketData(queryPacketData);
- hit.setCacheKey(cacheKey);
hit.setFillable();
- hit.setCached(fromCache);
+ hit.setCached(false);
extractDocumentInfo(hit, document);
channelDistributionKey.ifPresent(hit::setDistributionKey);
diff --git a/container-search/src/main/java/com/yahoo/search/dispatch/InterleavedSearchInvoker.java b/container-search/src/main/java/com/yahoo/search/dispatch/InterleavedSearchInvoker.java
index 67be6a5486c..4c71290e5f9 100644
--- a/container-search/src/main/java/com/yahoo/search/dispatch/InterleavedSearchInvoker.java
+++ b/container-search/src/main/java/com/yahoo/search/dispatch/InterleavedSearchInvoker.java
@@ -2,7 +2,6 @@
package com.yahoo.search.dispatch;
import com.yahoo.fs4.QueryPacket;
-import com.yahoo.prelude.fastsearch.CacheKey;
import com.yahoo.prelude.fastsearch.VespaBackEndSearcher;
import com.yahoo.search.Query;
import com.yahoo.search.Result;
@@ -96,7 +95,7 @@ public class InterleavedSearchInvoker extends SearchInvoker implements ResponseM
}
@Override
- protected Result getSearchResult(CacheKey cacheKey, Execution execution) throws IOException {
+ protected Result getSearchResult(Execution execution) throws IOException {
long nextTimeout = query.getTimeLeft();
try {
while (!invokers.isEmpty() && nextTimeout >= 0) {
@@ -105,7 +104,7 @@ public class InterleavedSearchInvoker extends SearchInvoker implements ResponseM
log.fine(() -> "Search timed out with " + askedNodes + " requests made, " + answeredNodes + " responses received");
break;
} else {
- mergeResult(invoker.getSearchResult(cacheKey, execution));
+ mergeResult(invoker.getSearchResult(execution));
ejectInvoker(invoker);
}
nextTimeout = nextTimeout();
diff --git a/container-search/src/main/java/com/yahoo/search/dispatch/RpcResourcePool.java b/container-search/src/main/java/com/yahoo/search/dispatch/RpcResourcePool.java
index bf626312b13..29641080ba6 100644
--- a/container-search/src/main/java/com/yahoo/search/dispatch/RpcResourcePool.java
+++ b/container-search/src/main/java/com/yahoo/search/dispatch/RpcResourcePool.java
@@ -50,8 +50,7 @@ public class RpcResourcePool {
public Optional<FillInvoker> getFillInvoker(Query query, VespaBackEndSearcher searcher, DocumentDatabase documentDb) {
if (query.properties().getBoolean(dispatchSummaries, true)
&& ! searcher.summaryNeedsQuery(query)
- && query.getRanking().getLocation() == null
- && ! searcher.getCacheControl().useCache(query))
+ && query.getRanking().getLocation() == null)
{
return Optional.of(new RpcFillInvoker(this, documentDb));
} else {
diff --git a/container-search/src/main/java/com/yahoo/search/dispatch/SearchErrorInvoker.java b/container-search/src/main/java/com/yahoo/search/dispatch/SearchErrorInvoker.java
index 7f24954298a..4dfdd65714d 100644
--- a/container-search/src/main/java/com/yahoo/search/dispatch/SearchErrorInvoker.java
+++ b/container-search/src/main/java/com/yahoo/search/dispatch/SearchErrorInvoker.java
@@ -2,7 +2,6 @@
package com.yahoo.search.dispatch;
import com.yahoo.fs4.QueryPacket;
-import com.yahoo.prelude.fastsearch.CacheKey;
import com.yahoo.search.Query;
import com.yahoo.search.Result;
import com.yahoo.search.result.Coverage;
@@ -44,7 +43,7 @@ public class SearchErrorInvoker extends SearchInvoker {
}
@Override
- protected Result getSearchResult(CacheKey cacheKey, Execution execution) throws IOException {
+ protected Result getSearchResult(Execution execution) throws IOException {
Result res = new Result(query, message);
if (coverage != null) {
res.setCoverage(coverage);
diff --git a/container-search/src/main/java/com/yahoo/search/dispatch/SearchInvoker.java b/container-search/src/main/java/com/yahoo/search/dispatch/SearchInvoker.java
index aec59df9a38..0d7ef53bb50 100644
--- a/container-search/src/main/java/com/yahoo/search/dispatch/SearchInvoker.java
+++ b/container-search/src/main/java/com/yahoo/search/dispatch/SearchInvoker.java
@@ -2,7 +2,6 @@
package com.yahoo.search.dispatch;
import com.yahoo.fs4.QueryPacket;
-import com.yahoo.prelude.fastsearch.CacheKey;
import com.yahoo.search.Query;
import com.yahoo.search.Result;
import com.yahoo.search.dispatch.searchcluster.Node;
@@ -30,16 +29,16 @@ public abstract class SearchInvoker extends CloseableInvoker {
* nodes, the provided {@link Execution} may be used to retrieve document summaries required
* for correct result windowing.
*/
- public Result search(Query query, QueryPacket queryPacket, CacheKey cacheKey, Execution execution) throws IOException {
+ public Result search(Query query, QueryPacket queryPacket, Execution execution) throws IOException {
sendSearchRequest(query, queryPacket);
- Result result = getSearchResult(cacheKey, execution);
+ Result result = getSearchResult(execution);
setFinalStatus(result.hits().getError() == null);
return result;
}
protected abstract void sendSearchRequest(Query query, QueryPacket queryPacket) throws IOException;
- protected abstract Result getSearchResult(CacheKey cacheKey, Execution execution) throws IOException;
+ protected abstract Result getSearchResult(Execution execution) throws IOException;
protected void setMonitor(ResponseMonitor<SearchInvoker> monitor) {
this.monitor = monitor;
diff --git a/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java b/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java
index a6e4ddaafbc..e41a582ea13 100644
--- a/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java
+++ b/container-search/src/main/java/com/yahoo/vespa/streamingvisitors/VdsStreamingSearcher.java
@@ -18,7 +18,6 @@ import com.yahoo.log.LogLevel;
import com.yahoo.messagebus.routing.Route;
import com.yahoo.prelude.Ping;
import com.yahoo.prelude.Pong;
-import com.yahoo.prelude.fastsearch.CacheKey;
import com.yahoo.prelude.fastsearch.FastHit;
import com.yahoo.prelude.fastsearch.GroupingListHit;
import com.yahoo.prelude.fastsearch.TimeoutException;
@@ -93,7 +92,7 @@ public class VdsStreamingSearcher extends VespaBackEndSearcher {
}
@Override
- public Result doSearch2(Query query, QueryPacket queryPacket, CacheKey cacheKey, Execution execution) {
+ public Result doSearch2(Query query, QueryPacket queryPacket, Execution execution) {
// TODO refactor this method into smaller methods, it's hard to see the actual code
lazyTrace(query, 7, "Routing to storage cluster ", getStorageClusterRouteSpec());