summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java21
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDirectory.java2
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java83
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java5
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/PeriodicApplicationMaintainer.java7
-rw-r--r--searchcore/src/vespa/searchcore/proton/documentmetastore/CMakeLists.txt1
-rw-r--r--searchcore/src/vespa/searchcore/proton/documentmetastore/lid_allocator.cpp26
-rw-r--r--searchcore/src/vespa/searchcore/proton/documentmetastore/white_list_provider.cpp3
-rw-r--r--searchcore/src/vespa/searchcore/proton/documentmetastore/white_list_provider.h18
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/query.cpp30
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/query.h5
-rw-r--r--searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/memoryindex/field_index.cpp8
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/leaf_blueprints.h8
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.cpp23
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.h1
-rw-r--r--storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.cpp9
-rw-r--r--storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.h7
19 files changed, 159 insertions, 104 deletions
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java b/configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java
index 22de36e98aa..a4dfec708d6 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/application/TenantApplications.java
@@ -18,6 +18,7 @@ import com.yahoo.vespa.config.server.NotFoundException;
import com.yahoo.vespa.config.server.ReloadHandler;
import com.yahoo.vespa.config.server.ReloadListener;
import com.yahoo.vespa.config.server.RequestHandler;
+import com.yahoo.vespa.config.server.deploy.TenantFileSystemDirs;
import com.yahoo.vespa.config.server.host.HostRegistry;
import com.yahoo.vespa.config.server.host.HostValidator;
import com.yahoo.vespa.config.server.monitoring.MetricUpdater;
@@ -31,6 +32,8 @@ import com.yahoo.vespa.curator.transaction.CuratorTransaction;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
+import java.nio.file.Files;
+import java.nio.file.Paths;
import java.time.Clock;
import java.time.Duration;
import java.util.Collection;
@@ -73,10 +76,12 @@ public class TenantApplications implements RequestHandler, ReloadHandler, HostVa
private final ApplicationMapper applicationMapper = new ApplicationMapper();
private final MetricUpdater tenantMetricUpdater;
private final Clock clock = Clock.systemUTC();
+ private final TenantFileSystemDirs tenantFileSystemDirs;
public TenantApplications(TenantName tenant, Curator curator, StripedExecutor<TenantName> zkWatcherExecutor,
ExecutorService zkCacheExecutor, Metrics metrics, ReloadListener reloadListener,
- ConfigserverConfig configserverConfig, HostRegistry<ApplicationId> hostRegistry) {
+ ConfigserverConfig configserverConfig, HostRegistry<ApplicationId> hostRegistry,
+ TenantFileSystemDirs tenantFileSystemDirs) {
this.curator = curator;
this.applicationsPath = TenantRepository.getApplicationsPath(tenant);
this.locksPath = TenantRepository.getLocksPath(tenant);
@@ -90,6 +95,7 @@ public class TenantApplications implements RequestHandler, ReloadHandler, HostVa
this.responseFactory = ConfigResponseFactory.create(configserverConfig);
this.tenantMetricUpdater = metrics.getOrCreateMetricUpdater(Metrics.createDimensions(tenant));
this.hostRegistry = hostRegistry;
+ this.tenantFileSystemDirs = tenantFileSystemDirs;
}
// For testing only
@@ -101,7 +107,8 @@ public class TenantApplications implements RequestHandler, ReloadHandler, HostVa
componentRegistry.getMetrics(),
componentRegistry.getReloadListener(),
componentRegistry.getConfigserverConfig(),
- componentRegistry.getHostRegistries().createApplicationHostRegistry(tenantName));
+ componentRegistry.getHostRegistries().createApplicationHostRegistry(tenantName),
+ new TenantFileSystemDirs(componentRegistry.getConfigServerDB(), tenantName));
}
/**
@@ -128,6 +135,10 @@ public class TenantApplications implements RequestHandler, ReloadHandler, HostVa
return data.isEmpty() ? Optional.empty() : Optional.of(Long.parseLong(data));
}
+ public boolean hasLocalSession(long sessionId) {
+ return Files.exists(Paths.get(tenantFileSystemDirs.sessionsPath().getAbsolutePath(), String.valueOf(sessionId)));
+ }
+
/**
* Returns a transaction which writes the given session id as the currently active for the given application.
*
@@ -237,12 +248,6 @@ public class TenantApplications implements RequestHandler, ReloadHandler, HostVa
return application.resolveConfig(req, responseFactory);
}
- // For testing only
- long getApplicationGeneration(ApplicationId appId, Optional<Version> vespaVersion) {
- Application application = getApplication(appId, vespaVersion);
- return application.getApplicationGeneration();
- }
-
private void notifyReloadListeners(ApplicationSet applicationSet) {
reloadListener.hostsUpdated(tenant, hostRegistry.getAllHosts());
reloadListener.configActivated(applicationSet);
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDirectory.java b/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDirectory.java
index ab7509a49aa..0865b72dbbf 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDirectory.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/filedistribution/FileDirectory.java
@@ -55,7 +55,7 @@ public class FileDirectory {
return root.getAbsolutePath() + "/" + ref.value();
}
- File getFile(FileReference reference) {
+ public File getFile(FileReference reference) {
ensureRootExist();
File dir = new File(getPath(reference));
if (!dir.exists()) {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java
index fc4071916ed..8aba9fa465d 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/session/SessionFactory.java
@@ -1,6 +1,7 @@
// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.config.server.session;
+import com.yahoo.config.FileReference;
import com.yahoo.config.application.api.ApplicationPackage;
import com.yahoo.config.application.api.DeployLogger;
import com.yahoo.config.model.application.provider.DeployData;
@@ -14,15 +15,18 @@ import com.yahoo.vespa.config.server.GlobalComponentRegistry;
import com.yahoo.vespa.config.server.TimeoutBudget;
import com.yahoo.vespa.config.server.application.TenantApplications;
import com.yahoo.vespa.config.server.deploy.TenantFileSystemDirs;
+import com.yahoo.vespa.config.server.filedistribution.FileDirectory;
import com.yahoo.vespa.config.server.host.HostValidator;
import com.yahoo.vespa.config.server.tenant.TenantRepository;
import com.yahoo.vespa.config.server.zookeeper.ConfigCurator;
import com.yahoo.vespa.config.server.zookeeper.SessionCounter;
import com.yahoo.vespa.curator.Curator;
+import com.yahoo.vespa.defaults.Defaults;
import com.yahoo.vespa.flags.BooleanFlag;
import com.yahoo.vespa.flags.Flags;
import java.io.File;
+import java.io.IOException;
import java.time.Clock;
import java.util.List;
import java.util.Optional;
@@ -31,7 +35,7 @@ import java.util.logging.Logger;
/**
* Serves as the factory of sessions. Takes care of copying files to the correct folder and initializing the
- * session state.
+ * session state. There is one SessionFactory per tenant.
*
* @author Ulf Lilleengen
*/
@@ -86,8 +90,7 @@ public class SessionFactory {
}
public RemoteSession createRemoteSession(long sessionId) {
- Path sessionPath = sessionsPath.append(String.valueOf(sessionId));
- SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(sessionPath);
+ SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
return new RemoteSession(tenant, sessionId, componentRegistry, sessionZKClient);
}
@@ -115,10 +118,10 @@ public class SessionFactory {
private LocalSession createSessionFromApplication(ApplicationPackage applicationPackage,
long sessionId,
- SessionZooKeeperClient sessionZKClient,
TimeoutBudget timeoutBudget,
Clock clock) {
log.log(Level.FINE, TenantRepository.logPre(tenant) + "Creating session " + sessionId + " in ZooKeeper");
+ SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
sessionZKClient.createNewSession(clock.instant());
Curator.CompletionWaiter waiter = sessionZKClient.getUploadWaiter();
LocalSession session = new LocalSession(tenant, sessionId, sessionPreparer, applicationPackage, sessionZKClient,
@@ -162,22 +165,46 @@ public class SessionFactory {
long sessionId = getNextSessionId();
try {
ensureSessionPathDoesNotExist(sessionId);
+ ApplicationPackage app = createApplicationPackage(applicationFile, applicationId,
+ sessionId, currentlyActiveSessionId, internalRedeploy);
+ return createSessionFromApplication(app, sessionId, timeoutBudget, clock);
+ } catch (Exception e) {
+ throw new RuntimeException("Error creating session " + sessionId, e);
+ }
+ }
+
+ /**
+ * This method is used when creating a session based on a remote session and the distributed application package
+ * It does not wait for session being created on other servers
+ */
+ private LocalSession createLocalSession(File applicationFile, ApplicationId applicationId,
+ long sessionId, long currentlyActiveSessionId) {
+ try {
+ ApplicationPackage applicationPackage = createApplicationPackage(applicationFile, applicationId,
+ sessionId, currentlyActiveSessionId, false);
SessionZooKeeperClient sessionZooKeeperClient = createSessionZooKeeperClient(getSessionPath(sessionId));
- File userApplicationDir = getSessionAppDir(sessionId);
- IOUtils.copyDirectory(applicationFile, userApplicationDir);
- ApplicationPackage applicationPackage = createApplication(applicationFile,
- userApplicationDir,
- applicationId,
- sessionId,
- currentlyActiveSessionId,
- internalRedeploy);
- applicationPackage.writeMetaData();
- return createSessionFromApplication(applicationPackage, sessionId, sessionZooKeeperClient, timeoutBudget, clock);
+ return new LocalSession(tenant, sessionId, sessionPreparer, applicationPackage, sessionZooKeeperClient,
+ getSessionAppDir(sessionId), applicationRepo, hostRegistry);
} catch (Exception e) {
throw new RuntimeException("Error creating session " + sessionId, e);
}
}
+ private ApplicationPackage createApplicationPackage(File applicationFile, ApplicationId applicationId,
+ long sessionId, long currentlyActiveSessionId,
+ boolean internalRedeploy) throws IOException {
+ File userApplicationDir = getSessionAppDir(sessionId);
+ IOUtils.copyDirectory(applicationFile, userApplicationDir);
+ ApplicationPackage applicationPackage = createApplication(applicationFile,
+ userApplicationDir,
+ applicationId,
+ sessionId,
+ currentlyActiveSessionId,
+ internalRedeploy);
+ applicationPackage.writeMetaData();
+ return applicationPackage;
+ }
+
/**
* Returns a new session instance for the given session id.
*/
@@ -190,6 +217,34 @@ public class SessionFactory {
getSessionAppDir(sessionId), applicationRepo, hostRegistry);
}
+ /**
+ * Returns a new session instance for the given session id.
+ */
+ LocalSession createLocalSessionUsingDistributedApplicationPackage(long sessionId) {
+ if (applicationRepo.hasLocalSession(sessionId)) {
+ log.log(Level.FINE, "Local session for session id " + sessionId + " already exists");
+ return createSessionFromId(sessionId);
+ }
+
+ log.log(Level.INFO, "Creating local session for session id " + sessionId);
+ SessionZooKeeperClient sessionZKClient = createSessionZooKeeperClient(getSessionPath(sessionId));
+ FileReference fileReference = sessionZKClient.readApplicationPackageReference();
+ log.log(Level.FINE, "File reference for session id " + sessionId + ": " + fileReference);
+ if (fileReference != null) {
+ File rootDir = new File(Defaults.getDefaults().underVespaHome(componentRegistry.getConfigserverConfig().fileReferencesDir()));
+ File sessionDir = new FileDirectory(rootDir).getFile(fileReference);
+ if (!sessionDir.exists())
+ throw new RuntimeException("File reference for session " + sessionId + " not found (" + sessionDir.getAbsolutePath() + ")");
+ ApplicationId applicationId = sessionZKClient.readApplicationId();
+ return createLocalSession(sessionDir,
+ applicationId,
+ sessionId,
+ applicationRepo.activeSessionOf(applicationId).orElse(nonExistingActiveSession));
+ }
+ return null;
+ }
+
+ // Return Optional instead of faking it with nonExistingActiveSession
private long getActiveSessionId(ApplicationId applicationId) {
List<ApplicationId> applicationIds = applicationRepo.activeApplications();
if (applicationIds.contains(applicationId)) {
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java b/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java
index 6f758bb4d27..841407817c6 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/tenant/TenantRepository.java
@@ -12,6 +12,7 @@ import com.yahoo.vespa.config.server.GlobalComponentRegistry;
import com.yahoo.vespa.config.server.ReloadHandler;
import com.yahoo.vespa.config.server.RequestHandler;
import com.yahoo.vespa.config.server.application.TenantApplications;
+import com.yahoo.vespa.config.server.deploy.TenantFileSystemDirs;
import com.yahoo.vespa.config.server.monitoring.MetricUpdater;
import com.yahoo.vespa.config.server.session.SessionRepository;
import com.yahoo.vespa.config.server.session.SessionFactory;
@@ -215,7 +216,8 @@ public class TenantRepository {
componentRegistry.getMetrics(),
componentRegistry.getReloadListener(),
componentRegistry.getConfigserverConfig(),
- componentRegistry.getHostRegistries().createApplicationHostRegistry(tenantName));
+ componentRegistry.getHostRegistries().createApplicationHostRegistry(tenantName),
+ new TenantFileSystemDirs(componentRegistry.getConfigServerDB(), tenantName));
if (requestHandler == null)
requestHandler = applicationRepo;
if (reloadHandler == null)
@@ -239,7 +241,6 @@ public class TenantRepository {
return tenants.get(DEFAULT_TENANT);
}
-
private void removeUnusedApplications() {
getAllTenants().forEach(tenant -> tenant.getApplicationRepo().removeUnusedApplications());
}
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/PeriodicApplicationMaintainer.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/PeriodicApplicationMaintainer.java
index d06d24872e1..06ecb1f4a01 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/PeriodicApplicationMaintainer.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/maintenance/PeriodicApplicationMaintainer.java
@@ -64,9 +64,16 @@ public class PeriodicApplicationMaintainer extends ApplicationMaintainer {
return deploymentTimes.entrySet().stream()
.sorted(Map.Entry.comparingByValue())
.map(Map.Entry::getKey)
+ .filter(id -> shouldMaintain(id))
.collect(Collectors.toCollection(LinkedHashSet::new));
}
+ private boolean shouldMaintain(ApplicationId id) {
+ if (id.tenant().value().equals("stream") && id.application().value().equals("stream-ranking")) return false;
+ if (id.tenant().value().equals("stream") && id.application().value().equals("stream-ranking-canary")) return false;
+ return true;
+ }
+
// TODO: Do not start deploying until some time has gone (ideally only until bootstrap of config server is finished)
private boolean waitInitially() {
return clock.instant().isBefore(start.plus(minTimeBetweenRedeployments));
diff --git a/searchcore/src/vespa/searchcore/proton/documentmetastore/CMakeLists.txt b/searchcore/src/vespa/searchcore/proton/documentmetastore/CMakeLists.txt
index 63ff53bb011..d257f29c9df 100644
--- a/searchcore/src/vespa/searchcore/proton/documentmetastore/CMakeLists.txt
+++ b/searchcore/src/vespa/searchcore/proton/documentmetastore/CMakeLists.txt
@@ -16,7 +16,6 @@ vespa_add_library(searchcore_documentmetastore STATIC
lidreusedelayer.cpp
lidstatevector.cpp
lid_hold_list.cpp
- white_list_provider.cpp
DEPENDS
searchcore_attribute
searchcore_bucketdb
diff --git a/searchcore/src/vespa/searchcore/proton/documentmetastore/lid_allocator.cpp b/searchcore/src/vespa/searchcore/proton/documentmetastore/lid_allocator.cpp
index 31361e40c68..61662f621f1 100644
--- a/searchcore/src/vespa/searchcore/proton/documentmetastore/lid_allocator.cpp
+++ b/searchcore/src/vespa/searchcore/proton/documentmetastore/lid_allocator.cpp
@@ -1,7 +1,6 @@
// Copyright 2017 Yahoo Holdings. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "lid_allocator.h"
-#include "white_list_provider.h"
#include <vespa/searchlib/common/bitvectoriterator.h>
#include <vespa/searchlib/fef/termfieldmatchdataarray.h>
#include <vespa/searchlib/fef/matchdata.h>
@@ -192,17 +191,13 @@ LidAllocator::constructFreeList(DocId lidLimit)
namespace {
-class WhiteListBlueprint : public SimpleLeafBlueprint, public WhiteListProvider
+class WhiteListBlueprint : public SimpleLeafBlueprint
{
private:
const search::GrowableBitVector &_activeLids;
mutable std::mutex _lock;
mutable std::vector<search::fef::TermFieldMatchData *> _matchDataVector;
- search::BitVector::UP get_white_list_filter() const override {
- return search::BitVector::create(_activeLids, 0, get_docid_limit());
- }
-
SearchIterator::UP
createLeafSearch(const TermFieldMatchDataArray &tfmda, bool strict) const override
{
@@ -210,16 +205,6 @@ private:
(void) tfmda;
return createFilterSearch(strict, FilterConstraint::UPPER_BOUND);
}
-
- SearchIterator::UP createFilterSearch(bool strict, FilterConstraint) const override {
- auto tfmd = new search::fef::TermFieldMatchData;
- {
- std::lock_guard<std::mutex> lock(_lock);
- _matchDataVector.push_back(tfmd);
- }
- return search::BitVectorIterator::create(&_activeLids, get_docid_limit(), *tfmd, strict);
- }
-
public:
WhiteListBlueprint(const search::GrowableBitVector &activeLids)
: SimpleLeafBlueprint(FieldSpecBaseList()),
@@ -231,6 +216,15 @@ public:
bool isWhiteList() const override { return true; }
+ SearchIterator::UP createFilterSearch(bool strict, FilterConstraint) const override {
+ auto tfmd = new search::fef::TermFieldMatchData;
+ {
+ std::lock_guard<std::mutex> lock(_lock);
+ _matchDataVector.push_back(tfmd);
+ }
+ return search::BitVectorIterator::create(&_activeLids, get_docid_limit(), *tfmd, strict);
+ }
+
~WhiteListBlueprint() {
for (auto matchData : _matchDataVector) {
delete matchData;
diff --git a/searchcore/src/vespa/searchcore/proton/documentmetastore/white_list_provider.cpp b/searchcore/src/vespa/searchcore/proton/documentmetastore/white_list_provider.cpp
deleted file mode 100644
index 49ea1960005..00000000000
--- a/searchcore/src/vespa/searchcore/proton/documentmetastore/white_list_provider.cpp
+++ /dev/null
@@ -1,3 +0,0 @@
-// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-#include "white_list_provider.h"
diff --git a/searchcore/src/vespa/searchcore/proton/documentmetastore/white_list_provider.h b/searchcore/src/vespa/searchcore/proton/documentmetastore/white_list_provider.h
deleted file mode 100644
index 99e0bf8d103..00000000000
--- a/searchcore/src/vespa/searchcore/proton/documentmetastore/white_list_provider.h
+++ /dev/null
@@ -1,18 +0,0 @@
-// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-
-#include <memory>
-
-#pragma once
-
-namespace search { class BitVector; }
-
-namespace proton::documentmetastore {
-
-/** Interface for fetching a copy of the white list bitvector */
-struct WhiteListProvider {
- virtual std::unique_ptr<search::BitVector> get_white_list_filter() const = 0;
-protected:
- ~WhiteListProvider() = default;
-};
-
-} // namespace
diff --git a/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp b/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
index f7fce994bd1..f19b416b92f 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/match_tools.cpp
@@ -185,6 +185,8 @@ MatchToolsFactory(QueryLimiter & queryLimiter,
_query.optimize();
trace.addEvent(4, "MTF: Fetch Postings");
_query.fetchPostings();
+ trace.addEvent(5, "MTF: Handle Global Filters");
+ _query.handle_global_filters(searchContext.getDocIdLimit());
_query.freeze();
trace.addEvent(5, "MTF: prepareSharedState");
_rankSetup.prepareSharedState(_queryEnv, _queryEnv.getObjectStore());
diff --git a/searchcore/src/vespa/searchcore/proton/matching/query.cpp b/searchcore/src/vespa/searchcore/proton/matching/query.cpp
index d7c0ac04ce5..8fd686e235d 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/query.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/query.cpp
@@ -13,7 +13,6 @@
#include <vespa/searchlib/query/tree/point.h>
#include <vespa/searchlib/query/tree/rectangle.h>
#include <vespa/searchlib/queryeval/intermediate_blueprints.h>
-#include <vespa/searchcore/proton/documentmetastore/white_list_provider.h>
#include <vespa/log/log.h>
LOG_SETUP(".proton.matching.query");
@@ -157,7 +156,6 @@ void
Query::setWhiteListBlueprint(Blueprint::UP whiteListBlueprint)
{
_whiteListBlueprint = std::move(whiteListBlueprint);
- _white_list_provider = dynamic_cast<WhiteListProvider *>(_whiteListBlueprint.get());
}
void
@@ -190,17 +188,7 @@ Query::reserveHandles(const IRequestContext & requestContext, ISearchContext &co
void
Query::optimize()
{
- using search::queryeval::GlobalFilter;
_blueprint = Blueprint::optimize(std::move(_blueprint));
- if (_blueprint->getState().want_global_filter()) {
- auto white_list = (_white_list_provider ?
- _white_list_provider->get_white_list_filter() :
- search::BitVector::UP());
- auto global_filter = GlobalFilter::create(std::move(white_list));
- _blueprint->set_global_filter(*global_filter);
- // optimized order may change after accounting for global filter:
- _blueprint = Blueprint::optimize(std::move(_blueprint));
- }
LOG(debug, "optimized blueprint:\n%s\n", _blueprint->asString().c_str());
}
@@ -211,6 +199,24 @@ Query::fetchPostings()
}
void
+Query::handle_global_filters(uint32_t docid_limit)
+{
+ using search::queryeval::GlobalFilter;
+ if (_blueprint->getState().want_global_filter()) {
+ auto constraint = Blueprint::FilterConstraint::UPPER_BOUND;
+ bool strict = true;
+ auto filter_iterator = _blueprint->createFilterSearch(strict, constraint);
+ filter_iterator->initRange(1, docid_limit);
+ auto white_list = filter_iterator->get_hits(1);
+ auto global_filter = GlobalFilter::create(std::move(white_list));
+ _blueprint->set_global_filter(*global_filter);
+ // optimized order may change after accounting for global filter:
+ _blueprint = Blueprint::optimize(std::move(_blueprint));
+ LOG(debug, "blueprint after handle_global_filters:\n%s\n", _blueprint->asString().c_str());
+ }
+}
+
+void
Query::freeze()
{
_blueprint->freeze();
diff --git a/searchcore/src/vespa/searchcore/proton/matching/query.h b/searchcore/src/vespa/searchcore/proton/matching/query.h
index 4ca66fb7a86..3ed6229830d 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/query.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/query.h
@@ -10,8 +10,6 @@
#include <vespa/searchlib/queryeval/blueprint.h>
#include <vespa/searchlib/queryeval/irequestcontext.h>
-namespace proton::documentmetastore { struct WhiteListProvider; }
-
namespace proton::matching {
class ViewResolver;
@@ -21,12 +19,10 @@ class Query
{
private:
using Blueprint=search::queryeval::Blueprint;
- using WhiteListProvider=proton::documentmetastore::WhiteListProvider;
search::query::Node::UP _query_tree;
Blueprint::UP _blueprint;
search::fef::Location _location;
Blueprint::UP _whiteListBlueprint;
- WhiteListProvider *_white_list_provider;
public:
Query();
@@ -93,6 +89,7 @@ public:
**/
void optimize();
void fetchPostings();
+ void handle_global_filters(uint32_t docidLimit);
void freeze();
/**
diff --git a/searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp b/searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp
index ac14233f32d..74e67fb7fcd 100644
--- a/searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp
+++ b/searchlib/src/tests/attribute/tensorattribute/tensorattribute_test.cpp
@@ -847,8 +847,8 @@ TEST_F("NN blueprint handles strong filter", NearestNeighborBlueprintFixture)
filter->invalidateCachedCount();
auto strong_filter = GlobalFilter::create(std::move(filter));
bp->set_global_filter(*strong_filter);
- EXPECT_EQUAL(11u, bp->getState().estimate().estHits);
- EXPECT_FALSE(bp->may_approximate());
+ EXPECT_EQUAL(1u, bp->getState().estimate().estHits);
+ EXPECT_TRUE(bp->may_approximate());
}
TEST_F("NN blueprint handles weak filter", NearestNeighborBlueprintFixture)
diff --git a/searchlib/src/vespa/searchlib/memoryindex/field_index.cpp b/searchlib/src/vespa/searchlib/memoryindex/field_index.cpp
index bc74725d620..fecb8116f90 100644
--- a/searchlib/src/vespa/searchlib/memoryindex/field_index.cpp
+++ b/searchlib/src/vespa/searchlib/memoryindex/field_index.cpp
@@ -6,6 +6,7 @@
#include <vespa/searchlib/bitcompression/posocccompression.h>
#include <vespa/searchlib/queryeval/booleanmatchiteratorwrapper.h>
#include <vespa/searchlib/queryeval/searchiterator.h>
+#include <vespa/searchlib/queryeval/filter_wrapper.h>
#include <vespa/vespalib/btree/btree.hpp>
#include <vespa/vespalib/btree/btreeiterator.hpp>
#include <vespa/vespalib/btree/btreenode.hpp>
@@ -272,6 +273,13 @@ public:
_field_id, _posting_itr.size());
return result;
}
+
+ SearchIterator::UP createFilterSearch(bool, FilterConstraint) const override {
+ auto wrapper = std::make_unique<queryeval::FilterWrapper>(getState().numFields());
+ auto & tfmda = wrapper->tfmda();
+ wrapper->wrap(make_search_iterator<interleaved_features>(_posting_itr, _feature_store, _field_id, tfmda));
+ return wrapper;
+ }
};
}
diff --git a/searchlib/src/vespa/searchlib/queryeval/leaf_blueprints.h b/searchlib/src/vespa/searchlib/queryeval/leaf_blueprints.h
index d2e86ac65a4..5bba87c7091 100644
--- a/searchlib/src/vespa/searchlib/queryeval/leaf_blueprints.h
+++ b/searchlib/src/vespa/searchlib/queryeval/leaf_blueprints.h
@@ -15,11 +15,11 @@ class EmptyBlueprint : public SimpleLeafBlueprint
{
protected:
SearchIterator::UP createLeafSearch(const search::fef::TermFieldMatchDataArray &tfmda, bool strict) const override;
- SearchIterator::UP createFilterSearch(bool strict, FilterConstraint constraint) const override;
public:
EmptyBlueprint(const FieldSpecBaseList &fields);
EmptyBlueprint(const FieldSpecBase &field);
EmptyBlueprint();
+ SearchIterator::UP createFilterSearch(bool strict, FilterConstraint constraint) const override;
};
//-----------------------------------------------------------------------------
@@ -31,16 +31,14 @@ private:
SimpleResult _result;
protected:
- SearchIterator::UP
+ SearchIterator::UP
createLeafSearch(const search::fef::TermFieldMatchDataArray &tfmda, bool strict) const override;
- SearchIterator::UP
- createFilterSearch(bool strict, FilterConstraint constraint) const override;
-
public:
SimpleBlueprint(const SimpleResult &result);
~SimpleBlueprint();
SimpleBlueprint &tag(const vespalib::string &tag);
const vespalib::string &tag() const { return _tag; }
+ SearchIterator::UP createFilterSearch(bool strict, FilterConstraint constraint) const override;
};
//-----------------------------------------------------------------------------
diff --git a/searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.cpp b/searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.cpp
index 6a27e8a9f14..55342f91e93 100644
--- a/searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.cpp
+++ b/searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.cpp
@@ -9,6 +9,9 @@
#include <vespa/eval/tensor/dense/dense_tensor.h>
#include <vespa/searchlib/tensor/dense_tensor_attribute.h>
#include <vespa/searchlib/tensor/distance_function_factory.h>
+#include <vespa/log/log.h>
+
+LOG_SETUP(".searchlib.queryeval.nearest_neighbor_blueprint");
using vespalib::tensor::DenseTensorView;
using vespalib::tensor::DenseTensor;
@@ -84,19 +87,24 @@ NearestNeighborBlueprint::set_global_filter(const GlobalFilter &global_filter)
{
_global_filter = global_filter.shared_from_this();
auto nns_index = _attr_tensor.nearest_neighbor_index();
+ LOG(debug, "set_global_filter with: %s / %s / %s",
+ (_approximate ? "approximate" : "exact"),
+ (nns_index ? "nns_index" : "no_index"),
+ (_global_filter->has_filter() ? "has_filter" : "no_filter"));
if (_approximate && nns_index) {
uint32_t est_hits = _attr_tensor.getNumDocs();
if (_global_filter->has_filter()) {
uint32_t max_hits = _global_filter->filter()->countTrueBits();
+ LOG(debug, "set_global_filter getNumDocs: %u / max_hits %u", est_hits, max_hits);
if (max_hits * 10 < est_hits) {
- // too many hits filtered out, use brute force implementation:
- _approximate = false;
- return;
+ LOG(debug, "too many hits filtered out, consider using brute force implementation");
}
est_hits = std::min(est_hits, max_hits);
}
est_hits = std::min(est_hits, _target_num_hits);
setEstimate(HitEstimate(est_hits, false));
+ perform_top_k();
+ LOG(debug, "perform_top_k found %zu hits", _found_hits.size());
}
}
@@ -107,7 +115,7 @@ NearestNeighborBlueprint::perform_top_k()
if (_approximate && nns_index) {
auto lhs_type = _query_tensor->fast_type();
auto rhs_type = _attr_tensor.getTensorType();
- // different cell types should have be converted already
+ // different cell types should be converted already
if (lhs_type == rhs_type) {
auto lhs = _query_tensor->cellsRef();
uint32_t k = _target_num_hits;
@@ -121,13 +129,6 @@ NearestNeighborBlueprint::perform_top_k()
}
}
-void
-NearestNeighborBlueprint::fetchPostings(const ExecuteInfo &execInfo) {
- if (execInfo.isStrict()) {
- perform_top_k();
- }
-}
-
std::unique_ptr<SearchIterator>
NearestNeighborBlueprint::createLeafSearch(const search::fef::TermFieldMatchDataArray& tfmda, bool strict) const
{
diff --git a/searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.h b/searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.h
index a713c73ad32..3e402b46a43 100644
--- a/searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.h
+++ b/searchlib/src/vespa/searchlib/queryeval/nearest_neighbor_blueprint.h
@@ -49,7 +49,6 @@ public:
bool strict) const override;
void visitMembers(vespalib::ObjectVisitor& visitor) const override;
bool always_needs_unpack() const override;
- void fetchPostings(const ExecuteInfo &execInfo) override;
};
}
diff --git a/storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.cpp b/storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.cpp
index 5627ade2a11..fbdbac27b7c 100644
--- a/storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.cpp
+++ b/storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.cpp
@@ -14,6 +14,10 @@
#include <vespa/storageapi/message/stat.h>
#include <vespa/vespalib/stllike/hash_map.hpp>
#include <vespa/vespalib/util/exceptions.h>
+#ifndef XXH_INLINE_ALL
+# define XXH_INLINE_ALL // Let XXH64 be inlined for fixed hash size (bucket ID)
+#endif
+#include <xxhash.h>
#include <vespa/log/log.h>
LOG_SETUP(".persistence.filestor.handler.impl");
@@ -894,6 +898,11 @@ FileStorHandlerImpl::Disk::broadcast()
}
}
+uint64_t FileStorHandlerImpl::Disk::dispersed_bucket_bits(const document::Bucket& bucket) noexcept {
+ const uint64_t raw_id = bucket.getBucketId().getId();
+ return XXH64(&raw_id, sizeof(uint64_t), 0);
+}
+
bool
FileStorHandlerImpl::Disk::schedule(const std::shared_ptr<api::StorageMessage>& msg)
{
diff --git a/storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.h b/storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.h
index 7a4f9000e82..00714c291b7 100644
--- a/storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.h
+++ b/storage/src/vespa/storage/persistence/filestorage/filestorhandlerimpl.h
@@ -197,12 +197,7 @@ public:
std::string dumpQueue() const;
void dumpActiveHtml(std::ostream & os) const;
void dumpQueueHtml(std::ostream & os) const;
- static uint64_t dispersed_bucket_bits(const document::Bucket& bucket) noexcept {
- // Disperse bucket bits by multiplying with the 64-bit FNV-1 prime.
- // This avoids an inherent affinity between the LSB of a bucket's bits
- // and the stripe an operation ends up on.
- return bucket.getBucketId().getId() * 1099511628211ULL;
- }
+ static uint64_t dispersed_bucket_bits(const document::Bucket& bucket) noexcept;
// We make a fairly reasonable assumption that there will be less than 64k stripes.
uint16_t stripe_index(const document::Bucket& bucket) const noexcept {
return static_cast<uint16_t>(dispersed_bucket_bits(bucket) % _stripes.size());