aboutsummaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java2
-rw-r--r--configserver/src/main/java/com/yahoo/vespa/config/server/application/CompressedApplicationInputStream.java24
-rwxr-xr-xcontainer-core/src/main/java/com/yahoo/container/Container.java11
-rw-r--r--container-core/src/main/resources/configdefinitions/container.qr.def3
-rw-r--r--container-disc/src/main/java/com/yahoo/container/jdisc/ConfiguredApplication.java6
-rw-r--r--fileacquirer/abi-spec.json2
-rw-r--r--fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerFactory.java4
-rw-r--r--fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerImpl.java29
-rw-r--r--hosted-tenant-base/pom.xml16
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java19
-rw-r--r--node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionServiceProvider.java13
-rw-r--r--persistence/src/tests/spi/clusterstatetest.cpp2
-rw-r--r--persistence/src/vespa/persistence/conformancetest/conformancetest.cpp58
-rw-r--r--persistence/src/vespa/persistence/spi/docentry.cpp2
-rw-r--r--searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp35
-rw-r--r--searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp3
-rw-r--r--searchcore/src/tests/proton/matching/query_test.cpp8
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/query.cpp4
-rw-r--r--searchcore/src/vespa/searchcore/proton/matching/query.h3
-rw-r--r--searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.cpp27
-rw-r--r--searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.h16
-rw-r--r--searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp5
-rw-r--r--searchlib/src/tests/attribute/imported_search_context/imported_search_context_test.cpp4
-rw-r--r--searchlib/src/tests/attribute/searchable/attribute_searchable_adapter_test.cpp2
-rw-r--r--searchlib/src/tests/attribute/searchable/attribute_weighted_set_blueprint_test.cpp4
-rw-r--r--searchlib/src/tests/attribute/searchcontext/searchcontext_test.cpp4
-rw-r--r--searchlib/src/tests/queryeval/blueprint/intermediate_blueprints_test.cpp2
-rw-r--r--searchlib/src/tests/queryeval/dot_product/dot_product_test.cpp2
-rw-r--r--searchlib/src/tests/queryeval/equiv/equiv_test.cpp10
-rw-r--r--searchlib/src/tests/queryeval/fake_searchable/fake_searchable_test.cpp11
-rw-r--r--searchlib/src/tests/queryeval/same_element/same_element_test.cpp7
-rw-r--r--searchlib/src/tests/queryeval/simple_phrase/simple_phrase_test.cpp12
-rw-r--r--searchlib/src/tests/queryeval/sourceblender/sourceblender_test.cpp2
-rw-r--r--searchlib/src/tests/queryeval/weighted_set_term/weighted_set_term_test.cpp4
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/executeinfo.h9
-rw-r--r--searchlib/src/vespa/searchlib/queryeval/same_element_blueprint.cpp9
36 files changed, 239 insertions, 135 deletions
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java b/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
index e1629a6c2c3..fc60f1225a7 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/ApplicationRepository.java
@@ -1100,7 +1100,7 @@ public class ApplicationRepository implements com.yahoo.config.provision.Deploye
try {
return in.decompress(tempDir);
} catch (IOException e) {
- throw new IllegalArgumentException("Unable to decompress stream", e);
+ throw new IllegalArgumentException("Unable to decompress application stream", e);
}
}
diff --git a/configserver/src/main/java/com/yahoo/vespa/config/server/application/CompressedApplicationInputStream.java b/configserver/src/main/java/com/yahoo/vespa/config/server/application/CompressedApplicationInputStream.java
index 1b4ebb5cd61..9f5ae013447 100644
--- a/configserver/src/main/java/com/yahoo/vespa/config/server/application/CompressedApplicationInputStream.java
+++ b/configserver/src/main/java/com/yahoo/vespa/config/server/application/CompressedApplicationInputStream.java
@@ -61,22 +61,25 @@ public class CompressedApplicationInputStream implements AutoCloseable {
* Close this stream.
* @throws IOException if the stream could not be closed
*/
- public void close() throws IOException {
- reader.close();
- }
+ public void close() throws IOException { reader.close(); }
File decompress() throws IOException {
return decompress(uncheck(() -> java.nio.file.Files.createTempDirectory("decompress")).toFile());
}
public File decompress(File dir) throws IOException {
- decompressInto(dir.toPath());
- return dir;
+ try {
+ return decompressInto(dir.toPath());
+ } catch (IOException e) {
+ throw new IOException("Unable to decompress stream into " + dir.getAbsolutePath(), e);
+ }
}
- private void decompressInto(Path dir) throws IOException {
+ private File decompressInto(Path dir) throws IOException {
if (!Files.isDirectory(dir)) throw new IllegalArgumentException("Not a directory: " + dir.toAbsolutePath());
- log.log(Level.FINE, () -> "Application is in " + dir.toAbsolutePath());
+
+ String absolutePath = dir.toFile().getAbsolutePath();
+ log.log(Level.FINE, () -> "Decompress application into " + absolutePath);
int entries = 0;
Path tmpFile = null;
OutputStream tmpStream = null;
@@ -98,9 +101,10 @@ public class CompressedApplicationInputStream implements AutoCloseable {
if (tmpStream != null) tmpStream.close();
if (tmpFile != null) Files.deleteIfExists(tmpFile);
}
- if (entries == 0) {
- log.log(Level.WARNING, "Not able to decompress any entries to " + dir);
- }
+ if (entries == 0)
+ log.log(Level.WARNING, "Unable to decompress any entries into " + absolutePath);
+
+ return dir.toFile();
}
private static Path createTempFile(Path applicationDir) throws IOException {
diff --git a/container-core/src/main/java/com/yahoo/container/Container.java b/container-core/src/main/java/com/yahoo/container/Container.java
index c86f53d86bb..7f46c6d8a02 100755
--- a/container-core/src/main/java/com/yahoo/container/Container.java
+++ b/container-core/src/main/java/com/yahoo/container/Container.java
@@ -95,18 +95,11 @@ public class Container {
}
// Only intended for use by the Server instance.
- public void setupFileAcquirer(QrConfig.Filedistributor filedistributorConfig) {
+ public void setupFileAcquirer() {
if (usingCustomFileAcquirer)
return;
- if (filedistributorConfig.configid().isEmpty()) {
- if (fileAcquirer != null)
- logger.warning("Disabling file distribution");
- fileAcquirer = null;
- } else {
- fileAcquirer = FileAcquirerFactory.create(filedistributorConfig.configid());
- }
-
+ fileAcquirer = FileAcquirerFactory.create();
setPathAcquirer(fileAcquirer);
}
diff --git a/container-core/src/main/resources/configdefinitions/container.qr.def b/container-core/src/main/resources/configdefinitions/container.qr.def
index ba2207541e8..e49e334a299 100644
--- a/container-core/src/main/resources/configdefinitions/container.qr.def
+++ b/container-core/src/main/resources/configdefinitions/container.qr.def
@@ -5,7 +5,8 @@ namespace=container
### connection to the config system
## filedistributor rpc configuration
-filedistributor.configid reference default="" restart
+## TODO: Unused, remove in Vespa 9
+filedistributor.configid reference default=""
## Is RPC server enabled?
rpc.enabled bool default=false restart
diff --git a/container-disc/src/main/java/com/yahoo/container/jdisc/ConfiguredApplication.java b/container-disc/src/main/java/com/yahoo/container/jdisc/ConfiguredApplication.java
index 623d11cc473..6edfcecdda9 100644
--- a/container-disc/src/main/java/com/yahoo/container/jdisc/ConfiguredApplication.java
+++ b/container-disc/src/main/java/com/yahoo/container/jdisc/ConfiguredApplication.java
@@ -168,7 +168,7 @@ public final class ConfiguredApplication implements Application {
public void start() {
qrConfig = getConfig(QrConfig.class);
reconfigure(qrConfig.shutdown());
- hackToInitializeServer(qrConfig);
+ hackToInitializeServer();
ContainerBuilder builder = createBuilderWithGuiceBindings();
configurer = createConfigurer(builder.guiceModules().activate());
@@ -242,9 +242,9 @@ public final class ConfiguredApplication implements Application {
}
}
- private static void hackToInitializeServer(QrConfig config) {
+ private static void hackToInitializeServer() {
try {
- Container.get().setupFileAcquirer(config.filedistributor());
+ Container.get().setupFileAcquirer();
Container.get().setupUrlDownloader();
} catch (Exception e) {
log.log(Level.SEVERE, "Caught exception when initializing server. Exiting.", e);
diff --git a/fileacquirer/abi-spec.json b/fileacquirer/abi-spec.json
index 63e3ba71f2d..caec5f8629c 100644
--- a/fileacquirer/abi-spec.json
+++ b/fileacquirer/abi-spec.json
@@ -21,7 +21,7 @@
],
"methods" : [
"public void <init>()",
- "public static com.yahoo.filedistribution.fileacquirer.FileAcquirer create(java.lang.String)"
+ "public static com.yahoo.filedistribution.fileacquirer.FileAcquirer create()"
],
"fields" : [ ]
},
diff --git a/fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerFactory.java b/fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerFactory.java
index 24744728f78..8de5f87d8b0 100644
--- a/fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerFactory.java
+++ b/fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerFactory.java
@@ -9,8 +9,8 @@ package com.yahoo.filedistribution.fileacquirer;
*/
public class FileAcquirerFactory {
- public static FileAcquirer create(String configId) {
- return new FileAcquirerImpl(configId);
+ public static FileAcquirer create() {
+ return new FileAcquirerImpl();
}
}
diff --git a/fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerImpl.java b/fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerImpl.java
index b3c4382c397..ddddcfc2170 100644
--- a/fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerImpl.java
+++ b/fileacquirer/src/main/java/com/yahoo/filedistribution/fileacquirer/FileAcquirerImpl.java
@@ -1,9 +1,7 @@
// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.filedistribution.fileacquirer;
-import com.yahoo.cloud.config.filedistribution.FiledistributorrpcConfig;
import com.yahoo.config.FileReference;
-import com.yahoo.config.subscription.ConfigSubscriber;
import com.yahoo.jrt.ErrorCode;
import com.yahoo.jrt.Request;
import com.yahoo.jrt.Spec;
@@ -12,6 +10,7 @@ import com.yahoo.jrt.Supervisor;
import com.yahoo.jrt.Target;
import com.yahoo.jrt.Transport;
import com.yahoo.vespa.config.FileReferenceDoesNotExistException;
+
import java.io.File;
import java.time.Duration;
import java.util.concurrent.TimeUnit;
@@ -20,6 +19,8 @@ import java.util.concurrent.locks.ReentrantLock;
import java.util.logging.Level;
import java.util.logging.Logger;
+import static com.yahoo.net.HostName.getLocalhost;
+
/**
* Retrieves the path to a file or directory on the local file system
* that has been transferred with the vespa file distribution
@@ -43,15 +44,15 @@ class FileAcquirerImpl implements FileAcquirer {
private final Supervisor supervisor = new Supervisor(new Transport("fileaquirer"));
- private final ConfigSubscriber configSubscriber;
+ private class Connection {
- private class Connection implements ConfigSubscriber.SingleSubscriber<FiledistributorrpcConfig> {
- private final Lock targetLock = new ReentrantLock();
- private Target target;
+ private static final int configProxyRpcPort = 19090;
- private volatile Spec spec;
+ private final Lock targetLock = new ReentrantLock();
+ private final Spec spec = new Spec(getLocalhost(), configProxyRpcPort);
private long pauseTime = 0; //milliseconds
+ private Target target;
private long nextLogTime = 0;
private long logCount = 0;
@@ -85,7 +86,7 @@ class FileAcquirerImpl implements FileAcquirer {
private void logWarning() {
if (logCount == 0 || System.currentTimeMillis() > nextLogTime ) {
- log.warning("Could not connect to the config proxy '" + spec.toString() + "'" + " - " + this + "@" + System.identityHashCode(this));
+ log.warning("Could not connect to the config proxy '" + spec + "'" + " - " + this + "@" + System.identityHashCode(this));
nextLogTime = System.currentTimeMillis() +
Math.min(TimeUnit.DAYS.toMillis(1),
@@ -94,11 +95,6 @@ class FileAcquirerImpl implements FileAcquirer {
}
}
- @Override
- public void configure(FiledistributorrpcConfig filedistributorrpcConfig) {
- spec = new Spec(filedistributorrpcConfig.connectionspec());
- }
-
public Target getTarget(Timer timer) throws InterruptedException {
TimeUnit unit = TimeUnit.MILLISECONDS;
@@ -122,17 +118,10 @@ class FileAcquirerImpl implements FileAcquirer {
};
}
- public FileAcquirerImpl(String configId) {
- configSubscriber = new ConfigSubscriber();
- configSubscriber.subscribe(connection, FiledistributorrpcConfig.class, configId);
- }
-
public void shutdown() {
- configSubscriber.close();
supervisor.transport().shutdown().join();
}
-
/**
* Returns the path to a file or directory corresponding to the
* given file reference. File references are produced by the
diff --git a/hosted-tenant-base/pom.xml b/hosted-tenant-base/pom.xml
index 9851659bef0..7dd66003581 100644
--- a/hosted-tenant-base/pom.xml
+++ b/hosted-tenant-base/pom.xml
@@ -262,6 +262,22 @@
</configuration>
</execution>
<execution>
+ <id>enforce-no-compile-scope</id>
+ <goals>
+ <goal>enforce</goal>
+ </goals>
+ <configuration>
+ <rules>
+ <bannedDependencies>
+ <!-- Fail validation for provided container deps which will fail when embedded in application bundle -->
+ <excludes>
+ <exclude>org.slf4j:slf4j-api:*:jar:compile</exclude>
+ </excludes>
+ </bannedDependencies>
+ </rules>
+ </configuration>
+ </execution>
+ <execution>
<id>enforce-just-our-jupiter</id>
<goals>
<goal>enforce</goal>
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java
index dfbe41e31d7..a4a4c42d9c1 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/NodeRepository.java
@@ -7,6 +7,8 @@ import com.yahoo.concurrent.maintenance.JobControl;
import com.yahoo.config.provision.ApplicationTransaction;
import com.yahoo.config.provision.ClusterSpec;
import com.yahoo.config.provision.DockerImage;
+import com.yahoo.config.provision.EndpointsChecker.HealthChecker;
+import com.yahoo.config.provision.EndpointsChecker.HealthCheckerProvider;
import com.yahoo.config.provision.NodeFlavors;
import com.yahoo.config.provision.Zone;
import com.yahoo.config.provisioning.NodeRepositoryConfig;
@@ -19,6 +21,8 @@ import com.yahoo.vespa.hosted.provision.Node.State;
import com.yahoo.vespa.hosted.provision.applications.Applications;
import com.yahoo.vespa.hosted.provision.archive.ArchiveUriManager;
import com.yahoo.vespa.hosted.provision.autoscale.MetricsDb;
+import com.yahoo.vespa.hosted.provision.lb.LoadBalancer;
+import com.yahoo.vespa.hosted.provision.lb.LoadBalancerInstance;
import com.yahoo.vespa.hosted.provision.lb.LoadBalancers;
import com.yahoo.vespa.hosted.provision.maintenance.InfrastructureVersions;
import com.yahoo.vespa.hosted.provision.node.Agent;
@@ -34,6 +38,7 @@ import com.yahoo.vespa.hosted.provision.provisioning.FirmwareChecks;
import com.yahoo.vespa.hosted.provision.provisioning.HostResourcesCalculator;
import com.yahoo.vespa.hosted.provision.provisioning.NodeResourceLimits;
import com.yahoo.vespa.hosted.provision.provisioning.ProvisionServiceProvider;
+import com.yahoo.vespa.hosted.provision.provisioning.ProvisionServiceProvider.ProtoHealthChecker;
import com.yahoo.vespa.orchestrator.Orchestrator;
import java.time.Clock;
@@ -45,7 +50,7 @@ import java.util.Optional;
*
* @author bratseth
*/
-public class NodeRepository extends AbstractComponent {
+public class NodeRepository extends AbstractComponent implements HealthCheckerProvider {
private final CuratorDb db;
private final Clock clock;
@@ -67,6 +72,7 @@ public class NodeRepository extends AbstractComponent {
private final MetricsDb metricsDb;
private final Orchestrator orchestrator;
private final int spareCount;
+ private final ProtoHealthChecker healthChecker;
private final JacksonFlag<SharedHost> sharedHosts;
/**
@@ -142,6 +148,7 @@ public class NodeRepository extends AbstractComponent {
this.orchestrator = orchestrator;
this.spareCount = spareCount;
this.sharedHosts = PermanentFlags.SHARED_HOST.bindTo(flagSource());
+ this.healthChecker = provisionServiceProvider.getHealthChecker();
nodes.rewrite();
}
@@ -250,4 +257,14 @@ public class NodeRepository extends AbstractComponent {
return Optional.of(image).filter(s -> !s.isEmpty()).map(DockerImage::fromString);
}
+ @Override
+ public HealthChecker getHealthChecker() {
+ return endpoint -> healthChecker.healthy(endpoint,
+ loadBalancers.list(endpoint.applicationId())
+ .cluster(endpoint.clusterName())
+ .first()
+ .flatMap(LoadBalancer::instance)
+ .flatMap(LoadBalancerInstance::idSeed));
+ }
+
}
diff --git a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionServiceProvider.java b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionServiceProvider.java
index 601a4008110..7b7971318cd 100644
--- a/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionServiceProvider.java
+++ b/node-repository/src/main/java/com/yahoo/vespa/hosted/provision/provisioning/ProvisionServiceProvider.java
@@ -1,17 +1,20 @@
// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hosted.provision.provisioning;
+import com.yahoo.config.provision.EndpointsChecker.Availability;
+import com.yahoo.config.provision.EndpointsChecker.Endpoint;
import com.yahoo.config.provision.EndpointsChecker.HealthCheckerProvider;
import com.yahoo.vespa.hosted.provision.lb.LoadBalancerService;
import java.util.Optional;
+import java.util.UUID;
/**
* Injectable component that provides provision service for load-balancers and hosts
*
* @author freva
*/
-public interface ProvisionServiceProvider extends HealthCheckerProvider {
+public interface ProvisionServiceProvider {
Optional<LoadBalancerService> getLoadBalancerService();
@@ -19,4 +22,12 @@ public interface ProvisionServiceProvider extends HealthCheckerProvider {
HostResourcesCalculator getHostResourcesCalculator();
+ default ProtoHealthChecker getHealthChecker() {
+ return (endpoint, idSeed) -> Availability.ready;
+ }
+
+ interface ProtoHealthChecker {
+ Availability healthy(Endpoint endpoint, Optional<UUID> idSeed);
+ }
+
}
diff --git a/persistence/src/tests/spi/clusterstatetest.cpp b/persistence/src/tests/spi/clusterstatetest.cpp
index 863bde5c225..bbe563274ec 100644
--- a/persistence/src/tests/spi/clusterstatetest.cpp
+++ b/persistence/src/tests/spi/clusterstatetest.cpp
@@ -297,7 +297,7 @@ TEST(DocEntryTest, test_doctype_and_gid) {
DocEntry::UP e = DocEntry::create(Timestamp(9), DocumentMetaEnum::NONE, "doc_type", GlobalId::parse("gid(0xc4cef118f9f9649222750be2)"));
EXPECT_EQ(9, e->getTimestamp());
EXPECT_FALSE(e->isRemove());
- EXPECT_EQ(20, e->getSize());
+ EXPECT_EQ(44, e->getSize());
EXPECT_EQ(nullptr, e->getDocument());
EXPECT_EQ(nullptr, e->getDocumentId());
EXPECT_EQ("doc_type", e->getDocumentType());
diff --git a/persistence/src/vespa/persistence/conformancetest/conformancetest.cpp b/persistence/src/vespa/persistence/conformancetest/conformancetest.cpp
index f08d8ef2344..a40d6b01e14 100644
--- a/persistence/src/vespa/persistence/conformancetest/conformancetest.cpp
+++ b/persistence/src/vespa/persistence/conformancetest/conformancetest.cpp
@@ -26,6 +26,7 @@
#include <vespa/config-stor-distribution.h>
#include <limits>
#include <gtest/gtest.h>
+#include <gmock/gmock.h>
using document::BucketId;
using document::BucketSpace;
@@ -36,6 +37,8 @@ using document::IntFieldValue;
using storage::spi::test::makeSpiBucket;
using storage::spi::test::cloneDocEntry;
+using namespace ::testing;
+
namespace storage::spi {
using PersistenceProviderUP = std::unique_ptr<PersistenceProvider>;
@@ -58,8 +61,7 @@ getSpi(ConformanceTest::PersistenceFactory &factory, const document::TestDocMan
return result;
}
-enum SELECTION_FIELDS
-{
+enum class SelectionFields {
METADATA_ONLY = 0,
ALL_FIELDS = 1
};
@@ -68,18 +70,10 @@ CreateIteratorResult
createIterator(PersistenceProvider& spi,
const Bucket& b,
const Selection& sel,
- IncludedVersions versions = NEWEST_DOCUMENT_ONLY,
- int fields = ALL_FIELDS)
+ IncludedVersions versions = NEWEST_DOCUMENT_ONLY)
{
- document::FieldSet::SP fieldSet;
- if (fields & ALL_FIELDS) {
- fieldSet = std::make_shared<document::AllFields>();
- } else {
- fieldSet = std::make_shared<document::DocIdOnly>();
- }
-
Context context(Priority(0), Trace::TraceLevel(0));
- return spi.createIterator(b, std::move(fieldSet), sel, versions, context);
+ return spi.createIterator(b, std::make_shared<document::AllFields>(), sel, versions, context);
}
Selection
@@ -206,14 +200,21 @@ getEntriesFromChunks(const std::vector<Chunk>& chunks)
DocEntryList
iterateBucket(PersistenceProvider& spi,
const Bucket& bucket,
- IncludedVersions versions)
+ IncludedVersions versions,
+ SelectionFields fields = SelectionFields::ALL_FIELDS)
{
DocEntryList ret;
DocumentSelection docSel("");
Selection sel(docSel);
+ document::FieldSet::SP field_set;
+ if (fields == SelectionFields::ALL_FIELDS) {
+ field_set = std::make_shared<document::AllFields>();
+ } else {
+ field_set = std::make_shared<document::NoFields>();
+ }
Context context(Priority(0), Trace::TraceLevel(0));
- CreateIteratorResult iter = spi.createIterator(bucket, std::make_shared<document::AllFields>(), sel, versions, context);
+ CreateIteratorResult iter = spi.createIterator(bucket, std::move(field_set), sel, versions, context);
EXPECT_EQ(Result::ErrorType::NONE, iter.getErrorCode());
@@ -1487,6 +1488,35 @@ TEST_F(ConformanceTest, test_iterate_missing_bucket)
test_iterate_empty_or_missing_bucket(false);
}
+TEST_F(ConformanceTest, metadata_iteration_includes_doctype_and_gid)
+{
+ document::TestDocMan testDocMan;
+ _factory->clear();
+ PersistenceProviderUP spi(getSpi(*_factory, testDocMan));
+
+ Context context(Priority(0), Trace::TraceLevel(0));
+ Bucket bucket(makeSpiBucket(BucketId(8, 0x01)));
+ Document::SP doc1 = testDocMan.createRandomDocumentAtLocation(0x01, 1);
+ Document::SP doc2 = testDocMan.createRandomDocumentAtLocation(0x01, 2);
+ spi->createBucket(bucket);
+ EXPECT_EQ(Result(), Result(spi->put(bucket, Timestamp(1), doc1)));
+ EXPECT_EQ(Result(), Result(spi->put(bucket, Timestamp(2), doc2)));
+ EXPECT_EQ(Result(), Result(spi->remove(bucket, Timestamp(3), doc1->getId())));
+
+ auto entries = iterateBucket(*spi, bucket, NEWEST_DOCUMENT_OR_REMOVE, SelectionFields::METADATA_ONLY);
+ ASSERT_THAT(entries, SizeIs(2));
+
+ EXPECT_EQ(entries[0]->getMetaEnum(), DocumentMetaEnum::NONE);
+ EXPECT_EQ(entries[0]->getGid(), doc2->getId().getGlobalId());
+ EXPECT_EQ(entries[0]->getTimestamp(), Timestamp(2));
+ EXPECT_EQ(entries[0]->getDocumentType(), "testdoctype1");
+
+ EXPECT_EQ(entries[1]->getMetaEnum(), DocumentMetaEnum::REMOVE_ENTRY);
+ EXPECT_EQ(entries[1]->getGid(), doc1->getId().getGlobalId());
+ EXPECT_EQ(entries[1]->getTimestamp(), Timestamp(3));
+ EXPECT_EQ(entries[1]->getDocumentType(), "testdoctype1");
+}
+
TEST_F(ConformanceTest, testDeleteBucket)
{
document::TestDocMan testDocMan;
diff --git a/persistence/src/vespa/persistence/spi/docentry.cpp b/persistence/src/vespa/persistence/spi/docentry.cpp
index 5077af568ac..6e92223cb6e 100644
--- a/persistence/src/vespa/persistence/spi/docentry.cpp
+++ b/persistence/src/vespa/persistence/spi/docentry.cpp
@@ -71,7 +71,7 @@ DocEntryWithId::DocEntryWithId(Timestamp t, DocumentMetaEnum metaEnum, const Doc
{ }
DocEntryWithTypeAndGid::DocEntryWithTypeAndGid(Timestamp t, DocumentMetaEnum metaEnum, vespalib::stringref docType, GlobalId gid)
- : DocEntry(t, metaEnum, docType.size() + sizeof(gid)),
+ : DocEntry(t, metaEnum, sizeof(DocEntry) + docType.size() + sizeof(gid)),
_type(docType),
_gid(gid)
{ }
diff --git a/searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp b/searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp
index 48ce2015420..85883324080 100644
--- a/searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp
+++ b/searchcore/src/tests/proton/document_iterator/document_iterator_test.cpp
@@ -1,6 +1,7 @@
// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include <vespa/searchcore/proton/common/attribute_updater.h>
+#include <vespa/searchcore/proton/common/doctypename.h>
#include <vespa/searchcore/proton/common/pendinglidtracker.h>
#include <vespa/searchcore/proton/persistenceengine/document_iterator.h>
#include <vespa/searchcore/proton/persistenceengine/commit_and_wait_document_retriever.h>
@@ -30,6 +31,7 @@ using document::DocumentId;
using document::DocumentType;
using document::DoubleFieldValue;
using document::Field;
+using document::GlobalId;
using document::IntFieldValue;
using document::StringFieldValue;
using search::AttributeContext;
@@ -174,7 +176,6 @@ UnitDR::UnitDR(const document::DocumentType &dt, document::Document::UP d, Times
{}
UnitDR::~UnitDR() = default;
-
struct VisitRecordingUnitDR : UnitDR {
using VisitedLIDs = std::unordered_set<DocumentIdT>;
VisitedLIDs& visited_lids;
@@ -397,6 +398,15 @@ void checkEntry(const IterateResult &res, size_t idx, const Timestamp &timestamp
EXPECT_EQUAL(sizeof(DocEntry), res.getEntries()[idx]->getSize());
}
+void checkEntry(const IterateResult &res, size_t idx, const Timestamp &timestamp, DocumentMetaEnum flags,
+ const GlobalId &gid, vespalib::stringref doc_type_name)
+{
+ ASSERT_LESS(idx, res.getEntries().size());
+ auto expect = DocEntry::create(timestamp, flags, doc_type_name, gid);
+ EXPECT_TRUE(equal(*expect, *res.getEntries()[idx]));
+ EXPECT_EQUAL(sizeof(DocEntry) + sizeof(GlobalId) + doc_type_name.size(), res.getEntries()[idx]->getSize());
+}
+
void checkEntry(const IterateResult &res, size_t idx, const DocumentId &id, const Timestamp &timestamp)
{
ASSERT_LESS(idx, res.getEntries().size());
@@ -415,6 +425,10 @@ void checkEntry(const IterateResult &res, size_t idx, const Document &doc, const
EXPECT_GREATER(getSize(doc), 0u);
}
+GlobalId gid_of(vespalib::stringref id_str) {
+ return DocumentId(id_str).getGlobalId();
+}
+
TEST("require that custom retrievers work as expected") {
DocumentId id1("id:ns:document::1");
DocumentId id2("id:ns:document::2");
@@ -605,15 +619,15 @@ TEST("require that iterating all versions returns both documents and removes") {
TEST("require that using an empty field set returns meta-data only") {
DocumentIterator itr(bucket(5), std::make_shared<document::NoFields>(), selectAll(), newestV(), -1, false);
- itr.add(doc("id:ns:document::1", Timestamp(2), bucket(5)));
- itr.add(cat(doc("id:ns:document::2", Timestamp(3), bucket(5)),
- rem("id:ns:document::3", Timestamp(4), bucket(5))));
+ itr.add(DocTypeName("foo"), doc_with_fields("id:ns:foo::1", Timestamp(2), bucket(5)));
+ itr.add(DocTypeName("document"), cat(doc("id:ns:document::2", Timestamp(3), bucket(5)),
+ rem("id:ns:document::3", Timestamp(4), bucket(5))));
IterateResult res = itr.iterate(largeNum);
EXPECT_TRUE(res.isCompleted());
EXPECT_EQUAL(3u, res.getEntries().size());
- TEST_DO(checkEntry(res, 0, Timestamp(2), DocumentMetaEnum::NONE));
- TEST_DO(checkEntry(res, 1, Timestamp(3), DocumentMetaEnum::NONE));
- TEST_DO(checkEntry(res, 2, Timestamp(4), DocumentMetaEnum::REMOVE_ENTRY));
+ TEST_DO(checkEntry(res, 0, Timestamp(2), DocumentMetaEnum::NONE, gid_of("id:ns:foo::1"), "foo"));
+ TEST_DO(checkEntry(res, 1, Timestamp(3), DocumentMetaEnum::NONE, gid_of("id:ns:document::2"), "document"));
+ TEST_DO(checkEntry(res, 2, Timestamp(4), DocumentMetaEnum::REMOVE_ENTRY, gid_of("id:ns:document::3"), "document"));
}
TEST("require that entries in other buckets are skipped") {
@@ -656,12 +670,13 @@ TEST("require that maxBytes splits iteration results for meta-data only iteratio
IterateResult res1 = itr.iterate(2 * sizeof(DocEntry));
EXPECT_TRUE(!res1.isCompleted());
EXPECT_EQUAL(2u, res1.getEntries().size());
- TEST_DO(checkEntry(res1, 0, Timestamp(2), DocumentMetaEnum::NONE));
- TEST_DO(checkEntry(res1, 1, Timestamp(3), DocumentMetaEnum::REMOVE_ENTRY));
+ // Note: empty doc types since we did not pass in an explicit doc type alongside the retrievers
+ TEST_DO(checkEntry(res1, 0, Timestamp(2), DocumentMetaEnum::NONE, gid_of("id:ns:document::1"), ""));
+ TEST_DO(checkEntry(res1, 1, Timestamp(3), DocumentMetaEnum::REMOVE_ENTRY, gid_of("id:ns:document::2"), ""));
IterateResult res2 = itr.iterate(largeNum);
EXPECT_TRUE(res2.isCompleted());
- TEST_DO(checkEntry(res2, 0, Timestamp(4), DocumentMetaEnum::NONE));
+ TEST_DO(checkEntry(res2, 0, Timestamp(4), DocumentMetaEnum::NONE, gid_of("id:ns:document::3"), ""));
IterateResult res3 = itr.iterate(largeNum);
EXPECT_TRUE(res3.isCompleted());
diff --git a/searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp b/searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp
index f5c9e8baffa..c9951f0b60e 100644
--- a/searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp
+++ b/searchcore/src/tests/proton/documentmetastore/documentmetastore_test.cpp
@@ -23,7 +23,6 @@
#include <vespa/vespalib/test/insertion_operators.h>
#include <vespa/vespalib/util/exceptions.h>
#include <vespa/vespalib/util/hw_info.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/threadstackexecutor.h>
#include <filesystem>
#include <thread>
@@ -188,7 +187,7 @@ assertWhiteList(const SimpleResult &exp, Blueprint::UP whiteListBlueprint, bool
{
MatchDataLayout mdl;
MatchData::UP md = mdl.createMatchData();
- whiteListBlueprint->fetchPostings(search::queryeval::ExecuteInfo::create(strict));
+ whiteListBlueprint->fetchPostings(search::queryeval::ExecuteInfo::createForTest(strict));
whiteListBlueprint->setDocIdLimit(docIdLimit);
SearchIterator::UP sb = whiteListBlueprint->createSearch(*md, strict);
diff --git a/searchcore/src/tests/proton/matching/query_test.cpp b/searchcore/src/tests/proton/matching/query_test.cpp
index 928e13a9fae..0b87734cec2 100644
--- a/searchcore/src/tests/proton/matching/query_test.cpp
+++ b/searchcore/src/tests/proton/matching/query_test.cpp
@@ -711,7 +711,7 @@ void Test::requireThatQueryGluesEverythingTogether() {
EXPECT_EQUAL(1u, md->getNumTermFields());
query.optimize();
- query.fetchPostings(search::queryeval::ExecuteInfo::create(true, &requestContext.getDoom()));
+ query.fetchPostings(ExecuteInfo::create(true, 1.0F, &requestContext.getDoom()));
SearchIterator::UP search = query.createSearch(*md);
ASSERT_TRUE(search.get());
}
@@ -744,7 +744,7 @@ void checkQueryAddsLocation(const string &loc_in, const string &loc_out) {
MatchData::UP md = mdl.createMatchData();
EXPECT_EQUAL(2u, md->getNumTermFields());
- query.fetchPostings(search::queryeval::ExecuteInfo::create(true, &requestContext.getDoom()));
+ query.fetchPostings(ExecuteInfo::create(true, 1.0F, &requestContext.getDoom()));
SearchIterator::UP search = query.createSearch(*md);
ASSERT_TRUE(search.get());
if (!EXPECT_NOT_EQUAL(string::npos, search->asString().find(loc_out))) {
@@ -966,7 +966,7 @@ Test::requireThatWhiteListBlueprintCanBeUsed()
MatchData::UP md = mdl.createMatchData();
query.optimize();
- query.fetchPostings(search::queryeval::ExecuteInfo::create(true, &requestContext.getDoom()));
+ query.fetchPostings(ExecuteInfo::create(true, 1.0F, &requestContext.getDoom()));
SearchIterator::UP search = query.createSearch(*md);
SimpleResult exp = SimpleResult().addHit(1).addHit(5).addHit(7).addHit(11);
SimpleResult act;
@@ -1129,7 +1129,7 @@ public:
{
set_want_global_filter(want_global_filter);
}
- ~GlobalFilterBlueprint();
+ ~GlobalFilterBlueprint() override;
void set_global_filter(const GlobalFilter& filter_, double estimated_hit_ratio_) override {
filter = filter_.shared_from_this();
estimated_hit_ratio = estimated_hit_ratio_;
diff --git a/searchcore/src/vespa/searchcore/proton/matching/query.cpp b/searchcore/src/vespa/searchcore/proton/matching/query.cpp
index 955cd30714f..00a3f6bab7e 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/query.cpp
+++ b/searchcore/src/vespa/searchcore/proton/matching/query.cpp
@@ -247,7 +247,7 @@ Query::optimize()
}
void
-Query::fetchPostings(const search::queryeval::ExecuteInfo & executeInfo)
+Query::fetchPostings(const ExecuteInfo & executeInfo)
{
_blueprint->fetchPostings(executeInfo);
}
@@ -265,7 +265,7 @@ Query::handle_global_filter(const vespalib::Doom & doom, uint32_t docid_limit,
_blueprint = Blueprint::optimize(std::move(_blueprint));
LOG(debug, "blueprint after handle_global_filter:\n%s\n", _blueprint->asString().c_str());
// strictness may change if optimized order changed:
- fetchPostings(search::queryeval::ExecuteInfo::create(true, &doom));
+ fetchPostings(ExecuteInfo::create(true, 1.0F, &doom));
}
bool
diff --git a/searchcore/src/vespa/searchcore/proton/matching/query.h b/searchcore/src/vespa/searchcore/proton/matching/query.h
index c2d2d389d2c..1b5d6dbca60 100644
--- a/searchcore/src/vespa/searchcore/proton/matching/query.h
+++ b/searchcore/src/vespa/searchcore/proton/matching/query.h
@@ -23,6 +23,7 @@ class Query
private:
using Blueprint = search::queryeval::Blueprint;
using GlobalFilter = search::queryeval::GlobalFilter;
+ using ExecuteInfo = search::queryeval::ExecuteInfo;
search::query::Node::UP _query_tree;
Blueprint::UP _blueprint;
Blueprint::UP _whiteListBlueprint;
@@ -98,7 +99,7 @@ public:
* test to verify the original query without optimization.
**/
void optimize();
- void fetchPostings(const search::queryeval::ExecuteInfo & executeInfo) ;
+ void fetchPostings(const ExecuteInfo & executeInfo) ;
void handle_global_filter(const vespalib::Doom & doom, uint32_t docid_limit,
double global_filter_lower_limit, double global_filter_upper_limit,
diff --git a/searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.cpp b/searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.cpp
index 52ae32634a5..e9d233ef6ec 100644
--- a/searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.cpp
+++ b/searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.cpp
@@ -1,8 +1,10 @@
// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "document_iterator.h"
+#include "ipersistencehandler.h"
#include <vespa/persistence/spi/docentry.h>
#include <vespa/searchcore/proton/common/cachedselect.h>
+#include <vespa/searchcore/proton/common/doctypename.h>
#include <vespa/searchcore/proton/common/selectcontext.h>
#include <vespa/document/select/gid_filter.h>
#include <vespa/document/select/node.h>
@@ -18,7 +20,9 @@ using storage::spi::DocEntry;
using storage::spi::Timestamp;
using document::Document;
using document::DocumentId;
+using document::GlobalId;
using storage::spi::DocumentMetaEnum;
+using vespalib::stringref;
namespace proton {
@@ -30,6 +34,11 @@ createDocEntry(Timestamp timestamp, bool removed) {
}
std::unique_ptr<DocEntry>
+createDocEntry(Timestamp timestamp, bool removed, stringref doc_type, const GlobalId &gid) {
+ return DocEntry::create(timestamp, (removed ? DocumentMetaEnum::REMOVE_ENTRY : DocumentMetaEnum::NONE), doc_type, gid);
+}
+
+std::unique_ptr<DocEntry>
createDocEntry(Timestamp timestamp, bool removed, Document::UP doc, ssize_t defaultSerializedSize) {
if (doc) {
if (removed) {
@@ -92,17 +101,23 @@ DocumentIterator::DocumentIterator(const storage::spi::Bucket &bucket,
DocumentIterator::~DocumentIterator() = default;
void
+DocumentIterator::add(const DocTypeName &doc_type_name, IDocumentRetriever::SP retriever)
+{
+ _sources.emplace_back(doc_type_name, std::move(retriever));
+}
+
+void
DocumentIterator::add(IDocumentRetriever::SP retriever)
{
- _sources.push_back(std::move(retriever));
+ add(DocTypeName(""), std::move(retriever));
}
IterateResult
DocumentIterator::iterate(size_t maxBytes)
{
if ( ! _fetchedData ) {
- for (const IDocumentRetriever::SP & source : _sources) {
- fetchCompleteSource(*source, _list);
+ for (const auto & source : _sources) {
+ fetchCompleteSource(source.first, *source.second, _list);
}
_fetchedData = true;
}
@@ -235,7 +250,9 @@ private:
}
void
-DocumentIterator::fetchCompleteSource(const IDocumentRetriever & source, IterateResult::List & list)
+DocumentIterator::fetchCompleteSource(const DocTypeName & doc_type_name,
+ const IDocumentRetriever & source,
+ IterateResult::List & list)
{
IDocumentRetriever::ReadGuard sourceReadGuard(source.getReadGuard());
search::DocumentMetaData::Vector metaData;
@@ -269,7 +286,7 @@ DocumentIterator::fetchCompleteSource(const IDocumentRetriever & source, Iterate
for (uint32_t lid : lidsToFetch) {
const search::DocumentMetaData & meta = metaData[lidIndexMap[lid]];
assert(lid == meta.lid);
- list.push_back(createDocEntry(storage::spi::Timestamp(meta.timestamp), meta.removed));
+ list.push_back(createDocEntry(storage::spi::Timestamp(meta.timestamp), meta.removed, doc_type_name.getName(), meta.gid));
}
} else {
MatchVisitor visitor(matcher, metaData, lidIndexMap, _fields.get(), list, _defaultSerializedSize);
diff --git a/searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.h b/searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.h
index e307c249dc0..5d2b2af24b9 100644
--- a/searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.h
+++ b/searchcore/src/vespa/searchcore/proton/persistenceengine/document_iterator.h
@@ -4,6 +4,7 @@
#include "i_document_retriever.h"
#include <vespa/searchlib/common/idocumentmetastore.h>
+#include <vespa/searchcore/proton/common/doctypename.h>
#include <vespa/persistence/spi/bucket.h>
#include <vespa/persistence/spi/selection.h>
#include <vespa/persistence/spi/result.h>
@@ -12,10 +13,14 @@
namespace proton {
+class IPersistenceHandler;
+
class DocumentIterator
{
private:
using ReadConsistency = storage::spi::ReadConsistency;
+ using DocTypeNameAndRetriever = std::pair<DocTypeName, IDocumentRetriever::SP>;
+
const storage::spi::Bucket _bucket;;
const storage::spi::Selection _selection;
const storage::spi::IncludedVersions _versions;
@@ -25,14 +30,16 @@ private:
const bool _metaOnly;
const bool _ignoreMaxBytes;
bool _fetchedData;
- std::vector<IDocumentRetriever::SP> _sources;
+ std::vector<DocTypeNameAndRetriever> _sources;
size_t _nextItem;
storage::spi::IterateResult::List _list;
- bool checkMeta(const search::DocumentMetaData &meta) const;
- void fetchCompleteSource(const IDocumentRetriever & source, storage::spi::IterateResult::List & list);
- bool isWeakRead() const { return _readConsistency == ReadConsistency::WEAK; }
+ [[nodiscard]] bool checkMeta(const search::DocumentMetaData &meta) const;
+ void fetchCompleteSource(const DocTypeName & doc_type_name,
+ const IDocumentRetriever & source,
+ storage::spi::IterateResult::List & list);
+ [[nodiscard]] bool isWeakRead() const { return _readConsistency == ReadConsistency::WEAK; }
public:
DocumentIterator(const storage::spi::Bucket &bucket, document::FieldSet::SP fields,
@@ -40,6 +47,7 @@ public:
ssize_t defaultSerializedSize, bool ignoreMaxBytes,
ReadConsistency readConsistency=ReadConsistency::STRONG);
~DocumentIterator();
+ void add(const DocTypeName & doc_type_name, IDocumentRetriever::SP retriever);
void add(IDocumentRetriever::SP retriever);
storage::spi::IterateResult iterate(size_t maxBytes);
};
diff --git a/searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp b/searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp
index bf8915b2505..4208e696a08 100644
--- a/searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp
+++ b/searchcore/src/vespa/searchcore/proton/persistenceengine/persistenceengine.cpp
@@ -544,9 +544,10 @@ PersistenceEngine::createIterator(const Bucket &bucket, FieldSetSP fields, const
auto entry = std::make_unique<IteratorEntry>(context.getReadConsistency(), bucket, std::move(fields), selection,
versions, _defaultSerializedSize, _ignoreMaxBytes);
for (; snap.handlers().valid(); snap.handlers().next()) {
- IPersistenceHandler::RetrieversSP retrievers = snap.handlers().get()->getDocumentRetrievers(context.getReadConsistency());
+ auto *handler = snap.handlers().get();
+ IPersistenceHandler::RetrieversSP retrievers = handler->getDocumentRetrievers(context.getReadConsistency());
for (const auto & retriever : *retrievers) {
- entry->it.add(retriever);
+ entry->it.add(handler->doc_type_name(), retriever);
}
}
entry->handler_sequence = HandlerSnapshot::release(std::move(snap));
diff --git a/searchlib/src/tests/attribute/imported_search_context/imported_search_context_test.cpp b/searchlib/src/tests/attribute/imported_search_context/imported_search_context_test.cpp
index 17a393d97eb..79bdd83dc88 100644
--- a/searchlib/src/tests/attribute/imported_search_context/imported_search_context_test.cpp
+++ b/searchlib/src/tests/attribute/imported_search_context/imported_search_context_test.cpp
@@ -272,7 +272,7 @@ TEST_F("Strict iterator is marked as strict", Fixture) {
TEST_F("Non-strict blueprint with high hit rate is strict", Fixture(false, FastSearchConfig::ExplicitlyEnabled)) {
auto ctx = f.create_context(word_term("5678"));
- ctx->fetchPostings(queryeval::ExecuteInfo::create(false, 0.02));
+ ctx->fetchPostings(queryeval::ExecuteInfo::createForTest(false, 0.02));
TermFieldMatchData match;
auto iter = f.create_iterator(*ctx, match, false);
@@ -281,7 +281,7 @@ TEST_F("Non-strict blueprint with high hit rate is strict", Fixture(false, FastS
TEST_F("Non-strict blueprint with low hit rate is non-strict", Fixture(false, FastSearchConfig::ExplicitlyEnabled)) {
auto ctx = f.create_context(word_term("5678"));
- ctx->fetchPostings(queryeval::ExecuteInfo::create(false, 0.01));
+ ctx->fetchPostings(queryeval::ExecuteInfo::createForTest(false, 0.01));
TermFieldMatchData match;
auto iter = f.create_iterator(*ctx, match, false);
diff --git a/searchlib/src/tests/attribute/searchable/attribute_searchable_adapter_test.cpp b/searchlib/src/tests/attribute/searchable/attribute_searchable_adapter_test.cpp
index 03e338ee284..029c3130609 100644
--- a/searchlib/src/tests/attribute/searchable/attribute_searchable_adapter_test.cpp
+++ b/searchlib/src/tests/attribute/searchable/attribute_searchable_adapter_test.cpp
@@ -220,7 +220,7 @@ Result do_search(IAttributeManager &attribute_manager, const Node &node, bool st
Blueprint::UP bp = source.createBlueprint(requestContext, FieldSpec(field, fieldId, handle), node);
ASSERT_TRUE(bp);
Result result(bp->getState().estimate().estHits, bp->getState().estimate().empty);
- bp->fetchPostings(queryeval::ExecuteInfo::create(strict, 1.0));
+ bp->fetchPostings(queryeval::ExecuteInfo::createForTest(strict));
SearchIterator::UP iterator = bp->createSearch(*match_data, strict);
ASSERT_TRUE(iterator);
iterator->initRange(1, num_docs);
diff --git a/searchlib/src/tests/attribute/searchable/attribute_weighted_set_blueprint_test.cpp b/searchlib/src/tests/attribute/searchable/attribute_weighted_set_blueprint_test.cpp
index 4ec73a1d313..16597b8b615 100644
--- a/searchlib/src/tests/attribute/searchable/attribute_weighted_set_blueprint_test.cpp
+++ b/searchlib/src/tests/attribute/searchable/attribute_weighted_set_blueprint_test.cpp
@@ -109,7 +109,7 @@ struct WS {
FieldSpecList fields;
fields.add(FieldSpec(field, fieldId, handle, ac.getAttribute(field)->getIsFilter()));
queryeval::Blueprint::UP bp = searchable.createBlueprint(requestContext, fields, *node);
- bp->fetchPostings(queryeval::ExecuteInfo::create(strict));
+ bp->fetchPostings(queryeval::ExecuteInfo::createForTest(strict));
SearchIterator::UP sb = bp->createSearch(*md, strict);
return sb;
}
@@ -125,7 +125,7 @@ struct WS {
FieldSpecList fields;
fields.add(FieldSpec(field, fieldId, handle));
queryeval::Blueprint::UP bp = searchable.createBlueprint(requestContext, fields, *node);
- bp->fetchPostings(queryeval::ExecuteInfo::create(strict));
+ bp->fetchPostings(queryeval::ExecuteInfo::createForTest(strict));
SearchIterator::UP sb = bp->createSearch(*md, strict);
FakeResult result;
sb->initRange(1, 10);
diff --git a/searchlib/src/tests/attribute/searchcontext/searchcontext_test.cpp b/searchlib/src/tests/attribute/searchcontext/searchcontext_test.cpp
index 91b6efc26a6..343a5c8e38b 100644
--- a/searchlib/src/tests/attribute/searchcontext/searchcontext_test.cpp
+++ b/searchlib/src/tests/attribute/searchcontext/searchcontext_test.cpp
@@ -216,7 +216,7 @@ private:
// test search iterator unpacking
void fillForSearchIteratorUnpackingTest(IntegerAttribute * ia, bool extra);
void testSearchIteratorUnpacking(const AttributePtr & ptr, SearchContext & sc, bool extra, bool strict) {
- sc.fetchPostings(queryeval::ExecuteInfo::create(strict, 1.0));
+ sc.fetchPostings(queryeval::ExecuteInfo::createForTest(strict));
for (bool withElementId : {false, true}) {
testSearchIteratorUnpacking(ptr, sc, extra, strict, withElementId);
}
@@ -649,7 +649,7 @@ public:
~Verifier() override;
SearchIterator::UP
create(bool strict) const override {
- _sc->fetchPostings(queryeval::ExecuteInfo::create(strict, 1.0));
+ _sc->fetchPostings(queryeval::ExecuteInfo::createForTest(strict));
return _sc->createIterator(&_dummy, strict);
}
private:
diff --git a/searchlib/src/tests/queryeval/blueprint/intermediate_blueprints_test.cpp b/searchlib/src/tests/queryeval/blueprint/intermediate_blueprints_test.cpp
index 7a8250a2e16..a663944938c 100644
--- a/searchlib/src/tests/queryeval/blueprint/intermediate_blueprints_test.cpp
+++ b/searchlib/src/tests/queryeval/blueprint/intermediate_blueprints_test.cpp
@@ -122,7 +122,7 @@ TEST("test And propagates updated histestimate") {
bp.addChild(ap(MyLeafSpec(2000).create<RememberExecuteInfo>()->setSourceId(2)));
bp.optimize_self();
bp.setDocIdLimit(5000);
- bp.fetchPostings(ExecuteInfo::create(true));
+ bp.fetchPostings(ExecuteInfo::TRUE);
EXPECT_EQUAL(3u, bp.childCnt());
for (uint32_t i = 0; i < bp.childCnt(); i++) {
const RememberExecuteInfo & child = dynamic_cast<const RememberExecuteInfo &>(bp.getChild(i));
diff --git a/searchlib/src/tests/queryeval/dot_product/dot_product_test.cpp b/searchlib/src/tests/queryeval/dot_product/dot_product_test.cpp
index 4305f7da116..1538a9ce0df 100644
--- a/searchlib/src/tests/queryeval/dot_product/dot_product_test.cpp
+++ b/searchlib/src/tests/queryeval/dot_product/dot_product_test.cpp
@@ -73,7 +73,7 @@ struct DP {
FieldSpecList fields;
fields.add(FieldSpec(field, fieldId, handle, field_is_filter));
queryeval::Blueprint::UP bp = searchable.createBlueprint(requestContext, fields, *node);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
SearchIterator::UP sb = bp->createSearch(*md, strict);
EXPECT_TRUE(dynamic_cast<DotProductSearch*>(sb.get()) != 0);
sb->initFullRange();
diff --git a/searchlib/src/tests/queryeval/equiv/equiv_test.cpp b/searchlib/src/tests/queryeval/equiv/equiv_test.cpp
index 9b74f8a650f..c570c06a60b 100644
--- a/searchlib/src/tests/queryeval/equiv/equiv_test.cpp
+++ b/searchlib/src/tests/queryeval/equiv/equiv_test.cpp
@@ -1,12 +1,14 @@
// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
-#include <vespa/log/log.h>
-LOG_SETUP("equiv_test");
+
#include <vespa/searchlib/queryeval/leaf_blueprints.h>
#include <vespa/searchlib/queryeval/intermediate_blueprints.h>
#include <vespa/searchlib/queryeval/equiv_blueprint.h>
#include <vespa/searchlib/fef/matchdatalayout.h>
#include <vespa/vespalib/gtest/gtest.h>
+#include <vespa/log/log.h>
+LOG_SETUP("equiv_test");
+
using namespace search::queryeval;
using search::fef::MatchData;
using search::fef::MatchDataLayout;
@@ -17,7 +19,7 @@ using search::fef::FieldPositionsIterator;
class EquivTest : public ::testing::Test {
protected:
EquivTest();
- ~EquivTest();
+ ~EquivTest() override;
void test_equiv(bool strict, bool unpack_normal_features, bool unpack_interleaved_features);
};
@@ -57,7 +59,7 @@ EquivTest::test_equiv(bool strict, bool unpack_normal_features, bool unpack_inte
data.setNeedNormalFeatures(unpack_normal_features);
data.setNeedInterleavedFeatures(unpack_interleaved_features);
}
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
SearchIterator::UP search = bp->createSearch(*md, strict);
search->initFullRange();
diff --git a/searchlib/src/tests/queryeval/fake_searchable/fake_searchable_test.cpp b/searchlib/src/tests/queryeval/fake_searchable/fake_searchable_test.cpp
index 3bfdcd21ce6..8ddac327643 100644
--- a/searchlib/src/tests/queryeval/fake_searchable/fake_searchable_test.cpp
+++ b/searchlib/src/tests/queryeval/fake_searchable/fake_searchable_test.cpp
@@ -6,7 +6,6 @@
#include <vespa/searchlib/queryeval/fake_requestcontext.h>
#include <vespa/searchlib/queryeval/blueprint.h>
#include <vespa/searchlib/query/tree/intermediatenodes.h>
-#include <vespa/searchlib/query/tree/termnodes.h>
#include <vespa/searchlib/query/tree/simplequery.h>
#include <vespa/searchlib/fef/matchdata.h>
@@ -63,7 +62,7 @@ TEST_F(FakeSearchableTest, require_that_term_search_works) {
bool strict = (i == 0);
SCOPED_TRACE(strict ? "strict" : "non-strict");
MatchData::UP md = MatchData::makeTestInstance(100, 10);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
SearchIterator::UP search = bp->createSearch(*md, strict);
search->initFullRange();
@@ -117,7 +116,7 @@ TEST_F(FakeSearchableTest, require_that_phrase_search_works) {
bool strict = (i == 0);
SCOPED_TRACE(strict ? "strict" : "non-strict");
MatchData::UP md = MatchData::makeTestInstance(100, 10);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
SearchIterator::UP search = bp->createSearch(*md, strict);
search->initFullRange();
@@ -168,7 +167,7 @@ TEST_F(FakeSearchableTest, require_that_weigheted_set_search_works) {
bool strict = (i == 0);
SCOPED_TRACE(strict ? "strict" : "non-strict");
MatchData::UP md = MatchData::makeTestInstance(100, 10);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
SearchIterator::UP search = bp->createSearch(*md, strict);
search->initFullRange();
@@ -239,7 +238,7 @@ TEST_F(FakeSearchableTest, require_that_multi_field_search_works) {
bool strict = (i == 0);
SCOPED_TRACE(strict ? "strict" : "non-strict");
MatchData::UP md = MatchData::makeTestInstance(100, 10);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
SearchIterator::UP search = bp->createSearch(*md, strict);
search->initFullRange();
@@ -323,7 +322,7 @@ TEST_F(FakeSearchableTest, require_that_phrase_with_empty_child_works) {
bool strict = (i == 0);
SCOPED_TRACE(strict ? "strict" : "non-strict");
MatchData::UP md = MatchData::makeTestInstance(100, 10);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
SearchIterator::UP search = bp->createSearch(*md, strict);
search->initFullRange();
diff --git a/searchlib/src/tests/queryeval/same_element/same_element_test.cpp b/searchlib/src/tests/queryeval/same_element/same_element_test.cpp
index 7c5a4648925..d05e6c8e4f4 100644
--- a/searchlib/src/tests/queryeval/same_element/same_element_test.cpp
+++ b/searchlib/src/tests/queryeval/same_element/same_element_test.cpp
@@ -6,7 +6,6 @@
#include <vespa/searchlib/queryeval/simpleresult.h>
#include <vespa/searchlib/queryeval/same_element_blueprint.h>
#include <vespa/searchlib/queryeval/same_element_search.h>
-#include <vespa/searchlib/queryeval/emptysearch.h>
#include <vespa/searchcommon/attribute/i_search_context.h>
#include <vespa/searchlib/attribute/searchcontextelementiterator.h>
#include <vespa/vespalib/test/insertion_operators.h>
@@ -48,7 +47,7 @@ std::unique_ptr<SameElementBlueprint> make_blueprint(const std::vector<FakeResul
Blueprint::UP finalize(Blueprint::UP bp, bool strict) {
Blueprint::UP result = Blueprint::optimize(std::move(bp));
- result->fetchPostings(ExecuteInfo::create(strict));
+ result->fetchPostings(ExecuteInfo::createForTest(strict));
result->freeze();
return result;
}
@@ -87,7 +86,7 @@ TEST("require that matching elements can be identified") {
auto md = make_match_data();
auto search = bp->createSearch(*md, false);
search->initRange(1, 1000);
- SameElementSearch *se = dynamic_cast<SameElementSearch*>(search.get());
+ auto *se = dynamic_cast<SameElementSearch*>(search.get());
ASSERT_TRUE(se != nullptr);
TEST_DO(verify_elements(*se, 5, {3, 7}));
TEST_DO(verify_elements(*se, 10, {}));
@@ -148,7 +147,7 @@ TEST("require that attribute iterators are wrapped for element unpacking") {
auto bp = finalize(make_blueprint({a,b}, true), true);
auto md = make_match_data();
auto search = bp->createSearch(*md, false);
- SameElementSearch *se = dynamic_cast<SameElementSearch*>(search.get());
+ auto *se = dynamic_cast<SameElementSearch*>(search.get());
ASSERT_TRUE(se != nullptr);
ASSERT_EQUAL(se->children().size(), 2u);
EXPECT_TRUE(dynamic_cast<SearchContextElementIterator*>(se->children()[0].get()) != nullptr);
diff --git a/searchlib/src/tests/queryeval/simple_phrase/simple_phrase_test.cpp b/searchlib/src/tests/queryeval/simple_phrase/simple_phrase_test.cpp
index 29d4dd2c457..cbe497d6363 100644
--- a/searchlib/src/tests/queryeval/simple_phrase/simple_phrase_test.cpp
+++ b/searchlib/src/tests/queryeval/simple_phrase/simple_phrase_test.cpp
@@ -36,7 +36,7 @@ struct MyTerm : public search::queryeval::SimpleLeafBlueprint {
setEstimate(HitEstimate(hits, (hits == 0)));
}
SearchIterator::UP createLeafSearch(const search::fef::TermFieldMatchDataArray &, bool) const override {
- return SearchIterator::UP();
+ return {};
}
SearchIteratorUP createFilterSearch(bool strict, FilterConstraint constraint) const override {
return create_default_filter(strict, constraint);
@@ -143,13 +143,13 @@ public:
void
fetchPostings(bool useBlueprint)
{
- ExecuteInfo execInfo = ExecuteInfo::create(_strict);
+ ExecuteInfo execInfo = ExecuteInfo::createForTest(_strict);
if (useBlueprint) {
_phrase.fetchPostings(execInfo);
return;
}
- for (size_t i = 0; i < _children.size(); ++i) {
- _children[i]->fetchPostings(execInfo);
+ for (const auto & i : _children) {
+ i->fetchPostings(execInfo);
}
}
@@ -167,8 +167,8 @@ public:
childMatch.add(child_term_field_match_data);
}
SimplePhraseSearch::Children children;
- for (size_t i = 0; i < _children.size(); ++i) {
- children.push_back(_children[i]->createSearch(*_md, _strict));
+ for (const auto & i : _children) {
+ children.push_back(i->createSearch(*_md, _strict));
}
search = std::make_unique<SimplePhraseSearch>(std::move(children),
MatchData::UP(), childMatch, _order,
diff --git a/searchlib/src/tests/queryeval/sourceblender/sourceblender_test.cpp b/searchlib/src/tests/queryeval/sourceblender/sourceblender_test.cpp
index 77c9e1a8039..bb2e559106f 100644
--- a/searchlib/src/tests/queryeval/sourceblender/sourceblender_test.cpp
+++ b/searchlib/src/tests/queryeval/sourceblender/sourceblender_test.cpp
@@ -74,7 +74,7 @@ TEST("test strictness") {
blend_b->addChild(std::move(a_b));
blend_b->addChild(std::move(b_b));
Blueprint::UP bp(blend_b);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
SearchIterator::UP search = bp->createSearch(*md, strict);
search->initFullRange();
SearchIterator &blend = *search;
diff --git a/searchlib/src/tests/queryeval/weighted_set_term/weighted_set_term_test.cpp b/searchlib/src/tests/queryeval/weighted_set_term/weighted_set_term_test.cpp
index 2f3f0eb7392..fed4e7c9bbd 100644
--- a/searchlib/src/tests/queryeval/weighted_set_term/weighted_set_term_test.cpp
+++ b/searchlib/src/tests/queryeval/weighted_set_term/weighted_set_term_test.cpp
@@ -68,7 +68,7 @@ struct WS {
FieldSpecList fields;
fields.add(FieldSpec(field, fieldId, handle));
auto bp = searchable.createBlueprint(requestContext, fields, *node);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
auto sb = bp->createSearch(*md, strict);
return (dynamic_cast<WeightedSetTermSearch*>(sb.get()) != nullptr);
}
@@ -83,7 +83,7 @@ struct WS {
FieldSpecList fields;
fields.add(FieldSpec(field, fieldId, handle, field_is_filter));
auto bp = searchable.createBlueprint(requestContext, fields, *node);
- bp->fetchPostings(ExecuteInfo::create(strict));
+ bp->fetchPostings(ExecuteInfo::createForTest(strict));
auto sb = bp->createSearch(*md, strict);
sb->initFullRange();
FakeResult result;
diff --git a/searchlib/src/vespa/searchlib/queryeval/executeinfo.h b/searchlib/src/vespa/searchlib/queryeval/executeinfo.h
index 9e42bb4da8e..362b0826f67 100644
--- a/searchlib/src/vespa/searchlib/queryeval/executeinfo.h
+++ b/searchlib/src/vespa/searchlib/queryeval/executeinfo.h
@@ -22,16 +22,13 @@ public:
static ExecuteInfo create(bool strict, const ExecuteInfo & org) noexcept {
return {strict, org._hitRate, org.getDoom()};
}
- static ExecuteInfo create(bool strict, const vespalib::Doom * doom) noexcept {
- return create(strict, 1.0F, doom);
- }
static ExecuteInfo create(bool strict, float hitRate, const vespalib::Doom * doom) noexcept {
return {strict, hitRate, doom};
}
- static ExecuteInfo create(bool strict) noexcept {
- return create(strict, 1.0F);
+ static ExecuteInfo createForTest(bool strict) noexcept {
+ return createForTest(strict, 1.0F);
}
- static ExecuteInfo create(bool strict, float hitRate) noexcept {
+ static ExecuteInfo createForTest(bool strict, float hitRate) noexcept {
return create(strict, hitRate, nullptr);
}
private:
diff --git a/searchlib/src/vespa/searchlib/queryeval/same_element_blueprint.cpp b/searchlib/src/vespa/searchlib/queryeval/same_element_blueprint.cpp
index 4674aee8314..6b2faac847e 100644
--- a/searchlib/src/vespa/searchlib/queryeval/same_element_blueprint.cpp
+++ b/searchlib/src/vespa/searchlib/queryeval/same_element_blueprint.cpp
@@ -56,8 +56,13 @@ SameElementBlueprint::optimize_self()
void
SameElementBlueprint::fetchPostings(const ExecuteInfo &execInfo)
{
- for (size_t i = 0; i < _terms.size(); ++i) {
- _terms[i]->fetchPostings(ExecuteInfo::create(execInfo.isStrict() && (i == 0), execInfo));
+ if (_terms.empty()) return;
+ _terms[0]->fetchPostings(execInfo);
+ double hit_rate = execInfo.hitRate() * _terms[0]->hit_ratio();
+ for (size_t i = 1; i < _terms.size(); ++i) {
+ Blueprint & term = *_terms[i];
+ term.fetchPostings(ExecuteInfo::create(false, hit_rate, execInfo.getDoom()));
+ hit_rate = hit_rate * term.hit_ratio();
}
}