summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorHenning Baldersheim <balder@yahoo-inc.com>2017-10-03 22:31:03 +0200
committerHenning Baldersheim <balder@yahoo-inc.com>2017-10-05 23:07:08 +0200
commitdff6169faab318931c8c3ca4d0dc7df4ea763594 (patch)
tree094ff5bd2fdcab25999a863987131a8bf7a8c84c
parent6fe9751702a2ccbe431944d36dc6fae339e093ab (diff)
Wire in for live reconfiguration of DocumentStore.
-rw-r--r--messagebus/src/vespa/messagebus/emptyreply.cpp5
-rw-r--r--messagebus/src/vespa/messagebus/emptyreply.h27
-rw-r--r--messagebus/src/vespa/messagebus/routable.h1
-rw-r--r--messagebus/src/vespa/messagebus/testlib/simpleprotocol.cpp2
-rw-r--r--searchcore/src/apps/tests/persistenceconformance_test.cpp2
-rw-r--r--searchcore/src/tests/proton/docsummary/docsummary.cpp12
-rw-r--r--searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp23
-rw-r--r--searchcore/src/tests/proton/documentdb/documentdb_test.cpp2
-rw-r--r--searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp6
-rw-r--r--searchcore/src/vespa/searchcore/config/proton.def38
-rw-r--r--searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.cpp30
-rw-r--r--searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.h37
-rw-r--r--searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp7
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/bootstrapconfig.h41
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/document_subdb_initializer.cpp1
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentdb.cpp92
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentdb.h26
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp11
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.h25
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.cpp17
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.h19
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/i_document_db_config_owner.h6
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/idocumentsubdb.h13
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/memoryflush.cpp21
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/memoryflush.h20
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/proton.cpp6
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/proton_configurer.cpp2
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.cpp47
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.h41
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.cpp91
-rw-r--r--searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.h45
-rw-r--r--searchcore/src/vespa/searchcore/proton/test/dummy_document_sub_db.h26
-rw-r--r--searchlib/src/vespa/searchlib/docstore/documentstore.cpp26
-rw-r--r--searchlib/src/vespa/searchlib/docstore/documentstore.h1
-rw-r--r--searchlib/src/vespa/searchlib/docstore/logdatastore.cpp23
-rw-r--r--searchlib/src/vespa/searchlib/docstore/logdatastore.h21
-rw-r--r--searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp6
-rw-r--r--searchlib/src/vespa/searchlib/docstore/logdocumentstore.h13
-rw-r--r--searchlib/src/vespa/searchlib/docstore/visitcache.cpp13
-rw-r--r--searchlib/src/vespa/searchlib/docstore/visitcache.h8
-rw-r--r--staging_vespalib/src/vespa/vespalib/stllike/cache.h2
-rw-r--r--staging_vespalib/src/vespa/vespalib/stllike/cache.hpp7
-rw-r--r--staging_vespalib/src/vespa/vespalib/util/varholder.h51
-rwxr-xr-xvespamalloc/src/tests/thread/thread_test.sh6
44 files changed, 368 insertions, 551 deletions
diff --git a/messagebus/src/vespa/messagebus/emptyreply.cpp b/messagebus/src/vespa/messagebus/emptyreply.cpp
index d87cffe52d5..4e858f4332d 100644
--- a/messagebus/src/vespa/messagebus/emptyreply.cpp
+++ b/messagebus/src/vespa/messagebus/emptyreply.cpp
@@ -21,9 +21,4 @@ EmptyReply::getType() const {
return 0;
}
-Blob
-EmptyReply::encode() const {
- return Blob(0);
-}
-
} // namespace mbus
diff --git a/messagebus/src/vespa/messagebus/emptyreply.h b/messagebus/src/vespa/messagebus/emptyreply.h
index 648598201b2..909fb197697 100644
--- a/messagebus/src/vespa/messagebus/emptyreply.h
+++ b/messagebus/src/vespa/messagebus/emptyreply.h
@@ -2,7 +2,6 @@
#pragma once
#include "reply.h"
-#include "blob.h"
namespace mbus {
@@ -16,34 +15,10 @@ namespace mbus {
*/
class EmptyReply : public Reply {
public:
- /**
- * Constructs a new instance of this class.
- */
EmptyReply();
-
- /**
- * This method returns the empty string to signal that it does not belong to
- * a protocol.
- *
- * @return ""
- */
const string & getProtocol() const override;
-
- /**
- * This method returns the message type id reserved for empty replies: 0
- *
- * @return 0
- */
uint32_t getType() const override;
-
- /**
- * Encodes this reply into an empty blob.
- *
- * @return empty blob
- */
- Blob encode() const;
-
uint8_t priority() const override { return 8; }
};
-} // namespace mbus
+}
diff --git a/messagebus/src/vespa/messagebus/routable.h b/messagebus/src/vespa/messagebus/routable.h
index 48154839994..50cb4e090ce 100644
--- a/messagebus/src/vespa/messagebus/routable.h
+++ b/messagebus/src/vespa/messagebus/routable.h
@@ -171,4 +171,3 @@ public:
};
} // namespace mbus
-
diff --git a/messagebus/src/vespa/messagebus/testlib/simpleprotocol.cpp b/messagebus/src/vespa/messagebus/testlib/simpleprotocol.cpp
index 0be33033538..29136a276bd 100644
--- a/messagebus/src/vespa/messagebus/testlib/simpleprotocol.cpp
+++ b/messagebus/src/vespa/messagebus/testlib/simpleprotocol.cpp
@@ -5,8 +5,6 @@
#include "simplereply.h"
#include <vespa/messagebus/emptyreply.h>
#include <vespa/messagebus/routing/routingcontext.h>
-#include <vespa/messagebus/routing/routingnodeiterator.h>
-#include <cstdlib>
namespace mbus {
diff --git a/searchcore/src/apps/tests/persistenceconformance_test.cpp b/searchcore/src/apps/tests/persistenceconformance_test.cpp
index d369fa84542..20309559854 100644
--- a/searchcore/src/apps/tests/persistenceconformance_test.cpp
+++ b/searchcore/src/apps/tests/persistenceconformance_test.cpp
@@ -185,7 +185,7 @@ public:
_queryLimiter,
_clock,
docType,
- ProtonConfig(),
+ std::make_shared<ProtonConfig>(),
const_cast<DocumentDBFactory &>(*this),
_summaryExecutor,
_summaryExecutor,
diff --git a/searchcore/src/tests/proton/docsummary/docsummary.cpp b/searchcore/src/tests/proton/docsummary/docsummary.cpp
index aff27959ec9..db15093e8b5 100644
--- a/searchcore/src/tests/proton/docsummary/docsummary.cpp
+++ b/searchcore/src/tests/proton/docsummary/docsummary.cpp
@@ -209,7 +209,7 @@ public:
_configMgr.nextGeneration(0);
if (! FastOS_File::MakeDirectory((std::string("tmpdb/") + docTypeName).c_str())) { abort(); }
_ddb.reset(new DocumentDB("tmpdb", _configMgr.getConfig(), "tcp/localhost:9013", _queryLimiter, _clock,
- DocTypeName(docTypeName), ProtonConfig(), *this, _summaryExecutor, _summaryExecutor,
+ DocTypeName(docTypeName), std::make_shared<ProtonConfig>(), *this, _summaryExecutor, _summaryExecutor,
_tls, _dummy, _fileHeaderContext, ConfigStore::UP(new MemoryConfigStore),
std::make_shared<vespalib::ThreadStackExecutor>(16, 128 * 1024), _hwInfo)),
_ddb->start();
@@ -233,11 +233,8 @@ public:
typedef DocumentMetaStore::Result PutRes;
IDocumentMetaStore &dms = _ddb->getReadySubDB()->getDocumentMetaStoreContext().get();
uint32_t docSize = 1;
- PutRes putRes(dms.put(docId.getGlobalId(),
- BucketFactory::getBucketId(docId),
- Timestamp(0u),
- docSize,
- lid));
+ PutRes putRes(dms.put(docId.getGlobalId(), BucketFactory::getBucketId(docId),
+ Timestamp(0u), docSize, lid));
LOG_ASSERT(putRes.ok());
uint64_t serialNum = _ddb->getFeedHandler().incSerialNum();
_aw->put(serialNum, doc, lid, true, std::shared_ptr<IDestructorCallback>());
@@ -255,8 +252,7 @@ public:
op.setDbDocumentId(dbdId);
op.setPrevDbDocumentId(prevDbdId);
_ddb->getFeedHandler().storeOperation(op);
- SearchView *sv(dynamic_cast<SearchView *>
- (_ddb->getReadySubDB()->getSearchView().get()));
+ SearchView *sv(dynamic_cast<SearchView *>(_ddb->getReadySubDB()->getSearchView().get()));
if (sv != NULL) {
// cf. FeedView::putAttributes()
DocIdLimit &docIdLimit = sv->getDocIdLimit();
diff --git a/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp b/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
index b14eb9051d0..cd88666662e 100644
--- a/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/document_subdbs/document_subdbs_test.cpp
@@ -262,25 +262,26 @@ struct MyConfigSnapshot
Schema _schema;
DocBuilder _builder;
DocumentDBConfig::SP _cfg;
+ BootstrapConfig::SP _bootstrap;
MyConfigSnapshot(const Schema &schema,
const vespalib::string &cfgDir)
: _schema(schema),
_builder(_schema),
- _cfg()
+ _cfg(),
+ _bootstrap()
{
DocumentDBConfig::DocumenttypesConfigSP documenttypesConfig
(new DocumenttypesConfig(_builder.getDocumenttypesConfig()));
TuneFileDocumentDB::SP tuneFileDocumentDB(new TuneFileDocumentDB());
- BootstrapConfig::SP bootstrap
- (new BootstrapConfig(1,
+ _bootstrap = std::make_shared<BootstrapConfig>(1,
documenttypesConfig,
_builder.getDocumentTypeRepo(),
std::make_shared<ProtonConfig>(),
std::make_shared<FiledistributorrpcConfig>(),
- tuneFileDocumentDB));
+ tuneFileDocumentDB);
config::DirSpec spec(cfgDir);
DocumentDBConfigHelper mgr(spec, "searchdocument");
- mgr.forwardConfig(bootstrap);
+ mgr.forwardConfig(_bootstrap);
mgr.nextGeneration(1);
_cfg = mgr.getConfig();
}
@@ -292,8 +293,8 @@ struct FixtureBase
ExecutorThreadingService _writeService;
ThreadStackExecutor _summaryExecutor;
typename Traits::Config _cfg;
- std::shared_ptr<BucketDBOwner> _bucketDB;
- BucketDBHandler _bucketDBHandler;
+ std::shared_ptr<BucketDBOwner> _bucketDB;
+ BucketDBHandler _bucketDBHandler;
typename Traits::Context _ctx;
typename Traits::Schema _baseSchema;
MyConfigSnapshot::UP _snapshot;
@@ -354,12 +355,8 @@ struct FixtureBase
cmpResult.documenttypesChanged = true;
cmpResult.documentTypeRepoChanged = true;
MyDocumentDBReferenceResolver resolver;
- IReprocessingTask::List tasks =
- _subDb.applyConfig(*newCfg->_cfg,
- *_snapshot->_cfg,
- serialNum,
- ReconfigParams(cmpResult),
- resolver);
+ auto tasks = _subDb.applyConfig(newCfg->_bootstrap->getProtonConfig(), *newCfg->_cfg, *_snapshot->_cfg,
+ serialNum, ReconfigParams(cmpResult), resolver);
_snapshot = std::move(newCfg);
if (!tasks.empty()) {
ReprocessingRunner runner;
diff --git a/searchcore/src/tests/proton/documentdb/documentdb_test.cpp b/searchcore/src/tests/proton/documentdb/documentdb_test.cpp
index 157e964ad83..e15a44ff962 100644
--- a/searchcore/src/tests/proton/documentdb/documentdb_test.cpp
+++ b/searchcore/src/tests/proton/documentdb/documentdb_test.cpp
@@ -114,7 +114,7 @@ Fixture::Fixture()
mgr.forwardConfig(b);
mgr.nextGeneration(0);
_db.reset(new DocumentDB(".", mgr.getConfig(), "tcp/localhost:9014", _queryLimiter, _clock, DocTypeName("typea"),
- ProtonConfig(), _myDBOwner, _summaryExecutor, _summaryExecutor, _tls, _dummy,
+ b->getProtonConfigSP(), _myDBOwner, _summaryExecutor, _summaryExecutor, _tls, _dummy,
_fileHeaderContext, ConfigStore::UP(new MemoryConfigStore),
std::make_shared<vespalib::ThreadStackExecutor>(16, 128 * 1024), _hwInfo));
_db->start();
diff --git a/searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp b/searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp
index 757a9c19f29..ec14f7de818 100644
--- a/searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp
+++ b/searchcore/src/tests/proton/proton_configurer/proton_configurer_test.cpp
@@ -207,9 +207,9 @@ struct MyDocumentDBConfigOwner : public IDocumentDBConfigOwner
_owner(owner)
{
}
- virtual ~MyDocumentDBConfigOwner() { }
+ ~MyDocumentDBConfigOwner() { }
- virtual void reconfigure(const DocumentDBConfig::SP & config) override;
+ void reconfigure(const std::shared_ptr<ProtonConfig> & protonConfig, const DocumentDBConfig::SP & config) override;
};
struct MyProtonConfigurerOwner : public IProtonConfigurerOwner
@@ -267,7 +267,7 @@ struct MyProtonConfigurerOwner : public IProtonConfigurerOwner
};
void
-MyDocumentDBConfigOwner::reconfigure(const DocumentDBConfig::SP & config)
+MyDocumentDBConfigOwner::reconfigure(const std::shared_ptr<ProtonConfig> &,const DocumentDBConfig::SP & config)
{
_owner.reconfigureDocumentDB(_name, config);
}
diff --git a/searchcore/src/vespa/searchcore/config/proton.def b/searchcore/src/vespa/searchcore/config/proton.def
index 5471d94c541..b2dd09fff36 100644
--- a/searchcore/src/vespa/searchcore/config/proton.def
+++ b/searchcore/src/vespa/searchcore/config/proton.def
@@ -186,11 +186,11 @@ grow.add int default=1 restart
grow.numdocs int default=10000 restart
## Control cache size in bytes.
-summary.cache.maxbytes long default=0 restart
+summary.cache.maxbytes long default=0
## Include visits in the cache, if the visitoperation allows it.
## This will enable another separate cache of summary.cache.maxbytes size.
-summary.cache.allowvisitcaching bool default=true restart
+summary.cache.allowvisitcaching bool default=true
## Control number of cache entries preallocated.
## Default is no preallocation
@@ -198,69 +198,71 @@ summary.cache.allowvisitcaching bool default=true restart
summary.cache.initialentries long default=0 restart
## Control compression type of the summary while in the cache.
-summary.cache.compression.type enum {NONE, LZ4, ZSTD} default=LZ4 restart
+summary.cache.compression.type enum {NONE, LZ4, ZSTD} default=LZ4
## Control compression level of the summary while in cache.
## LZ4 has normal range 1..9 while ZSTD has range 1..19
## 9 is a reasonable default for both
-summary.cache.compression.level int default=9 restart
+summary.cache.compression.level int default=9
## Control compression type of the summary while in memory during compaction
## NB So far only stragey=LOG honours it.
-summary.log.compact.compression.type enum {NONE, LZ4, ZSTD} default=ZSTD restart
+summary.log.compact.compression.type enum {NONE, LZ4, ZSTD} default=ZSTD
## Control compression level of the summary while in memory during compaction
## LZ4 has normal range 1..9 while ZSTD has range 1..19
## 9 is a reasonable default for both
-summary.log.compact.compression.level int default=9 restart
+summary.log.compact.compression.level int default=9
## Control compression type of the summary
-summary.log.chunk.compression.type enum {NONE, LZ4, ZSTD} default=ZSTD restart
+summary.log.chunk.compression.type enum {NONE, LZ4, ZSTD} default=ZSTD
## Control compression level of the summary
## LZ4 has normal range 1..9 while ZSTD has range 1..19
## 9 is a reasonable default for both. Going above for ZSTD can give an improvement,
## but is better done in conjunction with increasing chunk size.
-summary.log.chunk.compression.level int default=9 restart
+summary.log.chunk.compression.level int default=9
## Max size in bytes per chunk.
-summary.log.chunk.maxbytes int default=65536 restart
+summary.log.chunk.maxbytes int default=65536
## Max number of documents in each chunk.
## TODO Deprecated and ignored. Remove soon.
-summary.log.chunk.maxentries int default=256 restart
+summary.log.chunk.maxentries int default=256
## Skip crc32 check on read.
-summary.log.chunk.skipcrconread bool default=false restart
+summary.log.chunk.skipcrconread bool default=false
## Control how compation is done, write to the front or to new const file.
-summary.log.compact2activefile bool default=false restart
+summary.log.compact2activefile bool default=false
## Max size per summary file.
-summary.log.maxfilesize long default=1000000000 restart
+summary.log.maxfilesize long default=1000000000
## Max number of removes per summary file.
## TODO Deprecated and ignored. Remove soon.
-summary.log.maxentriesperfile long default=20000000 restart
+summary.log.maxentriesperfile long default=20000000
## Max disk bloat factor. This will trigger compacting.
-summary.log.maxdiskbloatfactor double default=0.1 restart
+summary.log.maxdiskbloatfactor double default=0.1
## Max bucket spread within a single summary file. This will trigger bucket order compacting.
## Only used when summary.compact2buckets is true.
-summary.log.maxbucketspread double default=2.5 restart
+summary.log.maxbucketspread double default=2.5
## If a file goes below this ratio compared to allowed max size it will be joined to the front.
## Value in the range [0.0, 1.0]
-summary.log.minfilesizefactor double default=0.2 restart
+summary.log.minfilesizefactor double default=0.2
## Number of threads used for compressing incomming documents/compacting.
summary.log.numthreads int default=8 restart
## Control io options during flush of stored documents.
-summary.write.io enum {NORMAL, OSYNC, DIRECTIO} default=DIRECTIO restart
+summary.write.io enum {NORMAL, OSYNC, DIRECTIO} default=DIRECTIO
## Control io options during read of stored documents.
+## All summary.read options will take effect immediately on new files written.
+## On old files it will take effect either upon compact or on restart.
summary.read.io enum {NORMAL, DIRECTIO, MMAP } default=MMAP restart
## Multiple optional options for use with mmap
diff --git a/searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.cpp b/searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.cpp
index 91b043d0c9f..71302e858f2 100644
--- a/searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.cpp
+++ b/searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.cpp
@@ -179,6 +179,20 @@ getStoreConfig(const ProtonConfig::Summary::Cache & cache)
return DocumentStore::Config(deriveCompression(cache.compression), cache.maxbytes, cache.initialentries).allowVisitCaching(cache.allowvisitcaching);
}
+LogDocumentStore::Config
+deriveConfig(const ProtonConfig::Summary & summary) {
+ DocumentStore::Config config(getStoreConfig(summary.cache));
+ const ProtonConfig::Summary::Log & log(summary.log);
+ const ProtonConfig::Summary::Log::Chunk & chunk(log.chunk);
+
+ WriteableFileChunk::Config fileConfig(deriveCompression(chunk.compression), chunk.maxbytes);
+ LogDataStore::Config logConfig(log.maxfilesize, log.maxdiskbloatfactor, log.maxbucketspread,
+ log.minfilesizefactor, log.numthreads, log.compact2activefile,
+ deriveCompression(log.compact.compression), fileConfig);
+ logConfig.disableCrcOnRead(chunk.skipcrconread);
+ return LogDocumentStore::Config(config, logConfig);
+}
+
}
SummaryManager::SummaryManager(vespalib::ThreadExecutor & executor,
@@ -196,17 +210,8 @@ SummaryManager::SummaryManager(vespalib::ThreadExecutor & executor,
_tuneFileSummary(tuneFileSummary),
_currentSerial(0u)
{
- DocumentStore::Config config(getStoreConfig(summary.cache));
- const ProtonConfig::Summary::Log & log(summary.log);
- const ProtonConfig::Summary::Log::Chunk & chunk(log.chunk);
-
- WriteableFileChunk::Config fileConfig(deriveCompression(chunk.compression), chunk.maxbytes);
- LogDataStore::Config logConfig(log.maxfilesize, log.maxdiskbloatfactor, log.maxbucketspread,
- log.minfilesizefactor, log.numthreads, log.compact2activefile,
- deriveCompression(log.compact.compression), fileConfig);
- logConfig.disableCrcOnRead(chunk.skipcrconread);
_docStore.reset(new LogDocumentStore(executor, baseDir,
- LogDocumentStore::Config(config, logConfig),
+ deriveConfig(summary),
growStrategy, tuneFileSummary, fileHeaderContext, tlSyncer,
summary.compact2buckets ? bucketizer : search::IBucketizer::SP()));
}
@@ -261,4 +266,9 @@ IFlushTarget::List SummaryManager::getFlushTargets(searchcorespi::index::IThread
return ret;
}
+void SummaryManager::reconfigure(const ProtonConfig::Summary & summary) {
+ LogDocumentStore & docStore = dynamic_cast<LogDocumentStore &> (*_docStore);
+ docStore.reconfigure(deriveConfig(summary));
+}
+
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.h b/searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.h
index af82f25ba51..51878d23bb6 100644
--- a/searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.h
+++ b/searchcore/src/vespa/searchcore/proton/docsummary/summarymanager.h
@@ -16,19 +16,15 @@
#include <vespa/fastlib/text/normwordfolder.h>
namespace searchcorespi::index { class IThreadService; }
-namespace search {
-
-class IBucketizer;
-
-namespace common { class FileHeaderContext; }
-
-}
+namespace search { class IBucketizer; }
+namespace search::common { class FileHeaderContext; }
namespace proton {
class SummaryManager : public ISummaryManager
{
public:
+ using ProtonConfig = vespa::config::search::core::ProtonConfig;
class SummarySetup : public ISummarySetup {
private:
std::unique_ptr<search::docsummary::DynamicDocsumWriter> _docsumWriter;
@@ -50,32 +46,27 @@ public:
const search::IDocumentStore::SP & docStore,
const document::DocumentTypeRepo::SP &repo);
- /**
- * Implements ISummarySetup.
- */
search::docsummary::IDocsumWriter & getDocsumWriter() const override { return *_docsumWriter; }
search::docsummary::ResultConfig & getResultConfig() override { return *_docsumWriter->GetResultConfig(); }
- search::docsummary::IDocsumStore::UP createDocsumStore(
- const vespalib::string &resultClassName) override;
+ search::docsummary::IDocsumStore::UP createDocsumStore(const vespalib::string &resultClassName) override;
- // Inherit doc from IDocsumEnvironment
- virtual search::IAttributeManager * getAttributeManager() override { return _attributeMgr.get(); }
- virtual vespalib::string lookupIndex(const vespalib::string & s) const override { (void) s; return ""; }
- virtual juniper::Juniper * getJuniper() override { return _juniperConfig.get(); }
+ search::IAttributeManager * getAttributeManager() override { return _attributeMgr.get(); }
+ vespalib::string lookupIndex(const vespalib::string & s) const override { (void) s; return ""; }
+ juniper::Juniper * getJuniper() override { return _juniperConfig.get(); }
};
private:
vespalib::string _baseDir;
DocTypeName _docTypeName;
- search::IDocumentStore::SP _docStore;
+ std::shared_ptr<search::IDocumentStore> _docStore;
const search::TuneFileSummary _tuneFileSummary;
uint64_t _currentSerial;
public:
typedef std::shared_ptr<SummaryManager> SP;
SummaryManager(vespalib::ThreadExecutor & executor,
- const vespa::config::search::core::ProtonConfig::Summary & summary,
+ const ProtonConfig::Summary & summary,
const search::GrowStrategy & growStrategy,
const vespalib::string &baseDir,
const DocTypeName &docTypeName,
@@ -87,22 +78,18 @@ public:
void putDocument(uint64_t syncToken, search::DocumentIdT lid, const document::Document & doc);
void putDocument(uint64_t syncToken, search::DocumentIdT lid, const vespalib::nbostream & doc);
-
void removeDocument(uint64_t syncToken, search::DocumentIdT lid);
searchcorespi::IFlushTarget::List getFlushTargets(searchcorespi::index::IThreadService & summaryService);
- /**
- * Implements ISummaryManager.
- */
- virtual ISummarySetup::SP
+ ISummarySetup::SP
createSummarySetup(const vespa::config::search::SummaryConfig &summaryCfg,
const vespa::config::search::SummarymapConfig &summarymapCfg,
const vespa::config::search::summary::JuniperrcConfig &juniperCfg,
const document::DocumentTypeRepo::SP &repo,
const search::IAttributeManager::SP &attributeMgr) override;
- virtual search::IDocumentStore & getBackingStore() override { return *_docStore; }
-
+ search::IDocumentStore & getBackingStore() override { return *_docStore; }
+ void reconfigure(const ProtonConfig::Summary & summary);
};
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp b/searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp
index 5c0cca83f69..5324f565481 100644
--- a/searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/docsummary/summarymanagerinitializer.cpp
@@ -11,12 +11,10 @@ SummaryManagerInitializer(const search::GrowStrategy &grow,
const vespalib::string baseDir,
const vespalib::string &subDbName,
const DocTypeName &docTypeName,
- vespalib::ThreadStackExecutorBase &
- summaryExecutor,
+ vespalib::ThreadStackExecutorBase &summaryExecutor,
const ProtonConfig::Summary protonSummaryCfg,
const search::TuneFileSummary &tuneFile,
- const search::common::FileHeaderContext &
- fileHeaderContext,
+ const search::common::FileHeaderContext &fileHeaderContext,
search::transactionlog::SyncProxy &tlSyncer,
IBucketizerSP bucketizer,
std::shared_ptr<SummaryManager::SP> result)
@@ -54,5 +52,4 @@ SummaryManagerInitializer::run()
EventLogger::loadDocumentStoreComplete(_subDbName, elapsedTimeMs);
}
-
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/server/bootstrapconfig.h b/searchcore/src/vespa/searchcore/proton/server/bootstrapconfig.h
index b4fcf71129c..f0252bb694f 100644
--- a/searchcore/src/vespa/searchcore/proton/server/bootstrapconfig.h
+++ b/searchcore/src/vespa/searchcore/proton/server/bootstrapconfig.h
@@ -56,49 +56,24 @@ public:
getDocumenttypesConfigSP() const { return _documenttypes; }
const document::DocumentTypeRepo::SP &
- getDocumentTypeRepoSP() const
- {
- return _repo;
- }
+ getDocumentTypeRepoSP() const { return _repo; }
const vespa::config::search::core::ProtonConfig &
- getProtonConfig() const
- {
- return *_proton;
- }
+ getProtonConfig() const { return *_proton; }
const ProtonConfigSP &
- getProtonConfigSP() const
- {
- return _proton;
- }
+ getProtonConfigSP() const { return _proton; }
const search::TuneFileDocumentDB::SP &
- getTuneFileDocumentDBSP() const
- {
- return _tuneFileDocumentDB;
- }
-
- int64_t
- getGeneration() const
- {
- return _generation;
- }
-
- void
- setGeneration(int64_t generation)
- {
- _generation = generation;
- }
+ getTuneFileDocumentDBSP() const { return _tuneFileDocumentDB; }
+
+ int64_t getGeneration() const { return _generation; }
/**
* Shared pointers are checked for identity, not equality.
*/
- bool
- operator==(const BootstrapConfig &rhs) const;
-
- bool
- valid() const;
+ bool operator==(const BootstrapConfig &rhs) const;
+ bool valid() const;
};
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/server/document_subdb_initializer.cpp b/searchcore/src/vespa/searchcore/proton/server/document_subdb_initializer.cpp
index 375940ea4df..a9609bf0cc5 100644
--- a/searchcore/src/vespa/searchcore/proton/server/document_subdb_initializer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/document_subdb_initializer.cpp
@@ -2,7 +2,6 @@
#include "document_subdb_initializer.h"
#include "idocumentsubdb.h"
-#include <future>
#include <vespa/vespalib/util/lambdatask.h>
#include <vespa/searchcorespi/index/i_thread_service.h>
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp b/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
index 756d710a07a..98c8d5f93f7 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/documentdb.cpp
@@ -10,6 +10,7 @@
#include "lid_space_compaction_handler.h"
#include "maintenance_jobs_injector.h"
#include "reconfig_params.h"
+#include "bootstrapconfig.h"
#include <vespa/searchcore/proton/attribute/attribute_writer.h>
#include <vespa/searchcore/proton/attribute/imported_attributes_repo.h>
#include <vespa/searchcore/proton/common/eventlogger.h>
@@ -42,8 +43,6 @@ using vespalib::Executor;
using vespalib::IllegalStateException;
using vespalib::StateExplorer;
using vespalib::make_string;
-using vespalib::makeTask;
-using vespalib::makeClosure;
using namespace proton::matching;
using namespace search;
using namespace search::engine;
@@ -86,7 +85,7 @@ DocumentDB::DocumentDB(const vespalib::string &baseDir,
matching::QueryLimiter & queryLimiter,
const vespalib::Clock &clock,
const DocTypeName &docTypeName,
- const ProtonConfig &protonCfg,
+ const std::shared_ptr<ProtonConfig> & protonCfg,
IDocumentDBOwner & owner,
vespalib::ThreadExecutor & warmupExecutor,
vespalib::ThreadStackExecutorBase & summaryExecutor,
@@ -105,16 +104,17 @@ DocumentDB::DocumentDB(const vespalib::string &baseDir,
_docTypeName(docTypeName),
_baseDir(baseDir + "/" + _docTypeName.toString()),
// Only one thread per executor, or performDropFeedView() will fail.
- _defaultExecutorTaskLimit(protonCfg.indexing.tasklimit),
- _semiUnboundExecutorTaskLimit(protonCfg.indexing.semiunboundtasklimit),
- _indexingThreads(protonCfg.indexing.threads),
- _writeService(std::max(1, protonCfg.indexing.threads),
+ _defaultExecutorTaskLimit(protonCfg->indexing.tasklimit),
+ _semiUnboundExecutorTaskLimit(protonCfg->indexing.semiunboundtasklimit),
+ _indexingThreads(protonCfg->indexing.threads),
+ _writeService(std::max(1, protonCfg->indexing.threads),
indexing_thread_stack_size,
_defaultExecutorTaskLimit),
_initializeThreads(initializeThreads),
_initConfigSnapshot(),
_initConfigSerialNum(0u),
_pendingConfigSnapshot(configSnapshot),
+ _pendingProtonConfigSnapshot(protonCfg),
_configMutex(),
_configCV(),
_activeConfigSnapshot(),
@@ -123,12 +123,12 @@ DocumentDB::DocumentDB(const vespalib::string &baseDir,
_initGate(),
_clusterStateHandler(_writeService.master()),
_bucketHandler(_writeService.master()),
- _protonSummaryCfg(protonCfg.summary),
- _protonIndexCfg(protonCfg.index),
+ _protonSummaryCfg(protonCfg->summary),
+ _protonIndexCfg(protonCfg->index),
_config_store(std::move(config_store)),
- _sessionManager(new matching::SessionManager(protonCfg.grouping.sessionmanager.maxentries)),
+ _sessionManager(new matching::SessionManager(protonCfg->grouping.sessionmanager.maxentries)),
_metricsWireService(metricsWireService),
- _metricsHook(*this, _docTypeName.getName(), protonCfg.numthreadspersearch),
+ _metricsHook(*this, _docTypeName.getName(), protonCfg->numthreadspersearch),
_feedView(),
_refCount(),
_syncFeedViewEnabled(false),
@@ -141,7 +141,7 @@ DocumentDB::DocumentDB(const vespalib::string &baseDir,
_state, *this, _writeFilter, *this, tlsDirectWriter),
_subDBs(*this, *this, _feedHandler, _docTypeName, _writeService, warmupExecutor,
summaryExecutor, fileHeaderContext, metricsWireService, getMetricsCollection(),
- queryLimiter, clock, _configMutex, _baseDir, protonCfg, hwInfo),
+ queryLimiter, clock, _configMutex, _baseDir, *protonCfg, hwInfo),
_maintenanceController(_writeService.master(), summaryExecutor, _docTypeName),
_visibility(_feedHandler, _writeService, _feedView),
_lidSpaceCompactionHandlers(),
@@ -220,8 +220,7 @@ void
DocumentDB::internalInit()
{
(void) _state.enterLoadState();
- _writeService.master().execute(makeTask(makeClosure(this,
- &DocumentDB::initManagers)));
+ masterExecute([this]() { initManagers(); });
}
class InitDoneTask : public vespalib::Executor::Task {
@@ -258,8 +257,7 @@ DocumentDB::initManagers()
DocumentDBConfig::SP configSnapshot(_initConfigSnapshot);
_initConfigSnapshot.reset();
InitializerTask::SP rootTask =
- _subDBs.createInitializer(*configSnapshot, _initConfigSerialNum,
- _protonSummaryCfg, _protonIndexCfg);
+ _subDBs.createInitializer(*configSnapshot, _initConfigSerialNum, _protonSummaryCfg, _protonIndexCfg);
InitializeThreads initializeThreads = _initializeThreads;
_initializeThreads.reset();
std::shared_ptr<TaskRunner> taskRunner(std::make_shared<TaskRunner>(*initializeThreads));
@@ -284,10 +282,11 @@ DocumentDB::initFinish(DocumentDBConfig::SP configSnapshot)
void
-DocumentDB::newConfigSnapshot(DocumentDBConfig::SP snapshot)
+DocumentDB::newConfigSnapshot(std::shared_ptr<ProtonConfig> protonConfig, DocumentDBConfig::SP snapshot)
{
// Called by executor thread
_pendingConfigSnapshot.set(snapshot);
+ _pendingProtonConfigSnapshot.set(protonConfig);
{
lock_guard guard(_configMutex);
if (_activeConfigSnapshot.get() == NULL) {
@@ -306,9 +305,7 @@ DocumentDB::newConfigSnapshot(DocumentDBConfig::SP snapshot)
return;
}
}
- _writeService.master().execute(makeTask(makeClosure(this,
- &DocumentDB::performReconfig,
- _pendingConfigSnapshot.get())));
+ masterExecute([this] () { performReconfig(_pendingProtonConfigSnapshot.get(), _pendingConfigSnapshot.get()); } );
}
@@ -348,10 +345,10 @@ DocumentDB::enterOnlineState()
}
void
-DocumentDB::performReconfig(DocumentDBConfig::SP configSnapshot)
+DocumentDB::performReconfig(std::shared_ptr<ProtonConfig> bootstrapConfig, DocumentDBConfig::SP configSnapshot)
{
// Called by executor thread
- applyConfig(configSnapshot, getCurrentSerialNumber());
+ applyConfig(std::move(bootstrapConfig), std::move(configSnapshot), getCurrentSerialNumber());
if (_state.getState() == DDBState::State::APPLY_LIVE_CONFIG) {
enterReprocessState();
}
@@ -359,7 +356,8 @@ DocumentDB::performReconfig(DocumentDBConfig::SP configSnapshot)
void
-DocumentDB::applySubDBConfig(const DocumentDBConfig &newConfigSnapshot, SerialNum serialNum, const ReconfigParams &params)
+DocumentDB::applySubDBConfig(const ProtonConfig & protonConfig, const DocumentDBConfig &newConfigSnapshot,
+ SerialNum serialNum, const ReconfigParams &params)
{
auto registry = _owner.getDocumentDBReferenceRegistry();
auto oldRepo = _activeConfigSnapshot->getDocumentTypeRepoSP();
@@ -368,18 +366,14 @@ DocumentDB::applySubDBConfig(const DocumentDBConfig &newConfigSnapshot, SerialNu
auto newRepo = newConfigSnapshot.getDocumentTypeRepoSP();
auto newDocType = newRepo->getDocumentType(_docTypeName.getName());
assert(newDocType != nullptr);
- DocumentDBReferenceResolver resolver(*registry,
- *newDocType,
- newConfigSnapshot.getImportedFieldsConfig(),
- *oldDocType,
- _refCount,
- _writeService.attributeFieldWriter());
- _subDBs.applyConfig(newConfigSnapshot, *_activeConfigSnapshot, serialNum, params, resolver);
+ DocumentDBReferenceResolver resolver(*registry, *newDocType, newConfigSnapshot.getImportedFieldsConfig(),
+ *oldDocType, _refCount, _writeService.attributeFieldWriter());
+ _subDBs.applyConfig(protonConfig, newConfigSnapshot, *_activeConfigSnapshot, serialNum, params, resolver);
}
void
-DocumentDB::applyConfig(DocumentDBConfig::SP configSnapshot,
- SerialNum serialNum)
+DocumentDB::applyConfig(std::shared_ptr<ProtonConfig> protonConfig,
+ DocumentDBConfig::SP configSnapshot, SerialNum serialNum)
{
// Always called by executor thread:
// Called by performReconfig() by executor thread during normal
@@ -428,8 +422,7 @@ DocumentDB::applyConfig(DocumentDBConfig::SP configSnapshot,
_feedView.get()->forceCommit(elidedConfigSave ? serialNum :
serialNum - 1);
_writeService.sync();
- fastos::TimeStamp visibilityDelay =
- configSnapshot->getMaintenanceConfigSP()->getVisibilityDelay();
+ fastos::TimeStamp visibilityDelay = configSnapshot->getMaintenanceConfigSP()->getVisibilityDelay();
hasVisibilityDelayChanged = (visibilityDelay != _visibility.getVisibilityDelay());
_visibility.setVisibilityDelay(visibilityDelay);
}
@@ -439,7 +432,7 @@ DocumentDB::applyConfig(DocumentDBConfig::SP configSnapshot,
_writeService.setTaskLimit(_defaultExecutorTaskLimit);
}
if (params.shouldSubDbsChange() || hasVisibilityDelayChanged) {
- applySubDBConfig(*configSnapshot, serialNum, params);
+ applySubDBConfig(*protonConfig, *configSnapshot, serialNum, params);
if (serialNum < _feedHandler.getSerialNum()) {
// Not last entry in tls. Reprocessing should already be done.
_subDBs.getReprocessingRunner().reset();
@@ -463,8 +456,7 @@ DocumentDB::applyConfig(DocumentDBConfig::SP configSnapshot,
if (params.shouldMaintenanceControllerChange()) {
forwardMaintenanceConfig();
}
- _writeFilter.setConfig(configSnapshot->getMaintenanceConfigSP()->
- getAttributeUsageFilterConfig());
+ _writeFilter.setConfig(configSnapshot->getMaintenanceConfigSP()->getAttributeUsageFilterConfig());
if (_subDBs.getReprocessingRunner().empty()) {
_subDBs.pruneRemovedFields(serialNum);
}
@@ -489,10 +481,7 @@ DocumentDB::performDropFeedView(IFeedView::SP feedView)
_writeService.summary().sync();
// Feed view is kept alive in the closure's shared ptr.
- _writeService.index().execute(makeTask(makeClosure(this,
- &proton::DocumentDB::
- performDropFeedView2,
- feedView)));
+ _writeService.index().execute(makeLambdaTask([this, feedView] () { performDropFeedView2(feedView); }));
}
@@ -505,7 +494,7 @@ DocumentDB::performDropFeedView2(IFeedView::SP feedView)
_writeService.indexFieldWriter().sync();
// Feed view is kept alive in the closure's shared ptr.
- _writeService.master().execute(makeTask(makeClosure(&doNothing, feedView)));
+ masterExecute([feedView] () { doNothing(feedView); });
}
@@ -553,7 +542,7 @@ DocumentDB::close()
_metricsWireService.cleanAttributes(ready, &legacyMetrics.attributes);
_metricsWireService.cleanAttributes(notReady, NULL);
_writeService.sync();
- _writeService.master().execute(makeTask(makeClosure(this, &DocumentDB::closeSubDBs)));
+ masterExecute([this] () { closeSubDBs(); } );
_writeService.sync();
// What about queued tasks ?
_writeService.shutdown();
@@ -780,9 +769,10 @@ DocumentDB::setIndexSchema(const DocumentDBConfig &configSnapshot, SerialNum ser
}
void
-DocumentDB::reconfigure(const DocumentDBConfig::SP & snapshot)
+DocumentDB::reconfigure(const std::shared_ptr<ProtonConfig> & protonConfig,
+ const DocumentDBConfig::SP & snapshot)
{
- masterExecute([this, snapshot]() { newConfigSnapshot(snapshot); });
+ masterExecute([this, protonConfig, snapshot]() { newConfigSnapshot(protonConfig, snapshot); });
// Wait for config to be applied, or for document db close
std::unique_lock<std::mutex> guard(_configMutex);
while ((_activeConfigSnapshotGeneration < snapshot->getGeneration()) && !_state.getClosed()) {
@@ -819,9 +809,7 @@ DocumentDB::enterApplyLiveConfigState()
lock_guard guard(_configMutex);
(void) _state.enterApplyLiveConfigState();
}
- _writeService.master().execute(makeTask(makeClosure(this,
- &DocumentDB::performReconfig,
- _pendingConfigSnapshot.get())));
+ masterExecute([this]() { performReconfig(_pendingProtonConfigSnapshot.get(), _pendingConfigSnapshot.get()); });
}
@@ -874,14 +862,12 @@ DocumentDB::replayConfig(search::SerialNum serialNum)
return;
}
// Load config to replay
- _config_store->loadConfig(*configSnapshot, serialNum,
- configSnapshot);
+ _config_store->loadConfig(*configSnapshot, serialNum, configSnapshot);
// Grab relevant parts from pending config
- configSnapshot = DocumentDBConfigScout::scout(configSnapshot,
- *_pendingConfigSnapshot.get());
+ configSnapshot = DocumentDBConfigScout::scout(configSnapshot, *_pendingConfigSnapshot.get());
// Ignore configs that are not relevant during replay of transaction log
configSnapshot = DocumentDBConfig::makeReplayConfig(configSnapshot);
- applyConfig(configSnapshot, serialNum);
+ applyConfig(_pendingProtonConfigSnapshot.get(), configSnapshot, serialNum);
LOG(info,
"DocumentDB(%s): Replayed config with serialNum=%" PRIu64,
_docTypeName.toString().c_str(), serialNum);
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentdb.h b/searchcore/src/vespa/searchcore/proton/server/documentdb.h
index a3f99148e7c..dc49256bd36 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentdb.h
+++ b/searchcore/src/vespa/searchcore/proton/server/documentdb.h
@@ -96,6 +96,7 @@ private:
DocumentDBConfig::SP _initConfigSnapshot;
SerialNum _initConfigSerialNum;
vespalib::VarHolder<DocumentDBConfig::SP> _pendingConfigSnapshot;
+ vespalib::VarHolder<std::shared_ptr<ProtonConfig>> _pendingProtonConfigSnapshot;
mutable std::mutex _configMutex; // protects _active* below.
mutable std::condition_variable _configCV;
DocumentDBConfig::SP _activeConfigSnapshot;
@@ -139,11 +140,13 @@ private:
void internalInit();
void initManagers();
void initFinish(DocumentDBConfig::SP configSnapshot);
- void performReconfig(DocumentDBConfig::SP configSnapshot);
+ void performReconfig(std::shared_ptr<ProtonConfig> bootstrapConfig, DocumentDBConfig::SP configSnapshot);
void closeSubDBs();
- void applySubDBConfig(const DocumentDBConfig &newConfigSnapshot, SerialNum serialNum, const ReconfigParams &params);
- void applyConfig(DocumentDBConfig::SP configSnapshot, SerialNum serialNum);
+ void applySubDBConfig(const ProtonConfig & protonConfig, const DocumentDBConfig &newConfigSnapshot,
+ SerialNum serialNum, const ReconfigParams &params);
+ void applyConfig(std::shared_ptr<ProtonConfig> bootstrapConfig,
+ DocumentDBConfig::SP configSnapshot, SerialNum serialNum);
/**
* Save initial config if we don't have any saved config snapshots.
@@ -163,7 +166,7 @@ private:
* Redo interrupted reprocessing if last entry in transaction log
* is a config change.
*/
- virtual void enterRedoReprocessState() override;
+ void enterRedoReprocessState() override;
void enterApplyLiveConfigState();
/**
@@ -238,7 +241,7 @@ public:
matching::QueryLimiter & queryLimiter,
const vespalib::Clock &clock,
const DocTypeName &docTypeName,
- const ProtonConfig &protonCfg,
+ const std::shared_ptr<ProtonConfig> &protonCfg,
IDocumentDBOwner & owner,
vespalib::ThreadExecutor & warmupExecutor,
vespalib::ThreadStackExecutorBase & summaryExecutor,
@@ -391,17 +394,15 @@ public:
bool getDelayedConfig() const { return _state.getDelayedConfig(); }
- /**
- * Implements IReplayConfig API.
- */
- virtual void replayConfig(SerialNum serialNum) override;
+ void replayConfig(SerialNum serialNum) override;
const DocTypeName & getDocTypeName() const { return _docTypeName; }
- void newConfigSnapshot(DocumentDBConfig::SP snapshot);
+ void newConfigSnapshot(std::shared_ptr<ProtonConfig> bootstrapConfig, DocumentDBConfig::SP snapshot);
// Implements DocumentDBConfigOwner
- void reconfigure(const DocumentDBConfig::SP & snapshot) override;
+ void reconfigure(const std::shared_ptr<ProtonConfig> & bootstrapConfig,
+ const DocumentDBConfig::SP & snapshot) override;
int64_t getActiveGeneration() const;
@@ -432,7 +433,7 @@ public:
*
* Sync transaction log to syncTo.
*/
- virtual void sync(SerialNum syncTo) override;
+ void sync(SerialNum syncTo) override;
void enterReprocessState();
void enterOnlineState();
void waitForOnlineState();
@@ -440,4 +441,3 @@ public:
};
} // namespace proton
-
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp b/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
index 9caeb49d370..b566eb05e79 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.cpp
@@ -12,7 +12,6 @@
using proton::matching::SessionManager;
using search::index::Schema;
using search::SerialNum;
-using vespa::config::search::core::ProtonConfig;
using searchcorespi::IFlushTarget;
namespace proton {
@@ -176,10 +175,7 @@ DocumentSubDBCollection::createInitializer(const DocumentDBConfig &configSnapsho
std::make_shared<DocumentSubDbCollectionInitializer>();
for (auto subDb : _subDBs) {
DocumentSubDbInitializer::SP
- subTask(subDb->createInitializer(configSnapshot,
- configSerialNum,
- protonSummaryCfg,
- indexCfg));
+ subTask(subDb->createInitializer(configSnapshot, configSerialNum, protonSummaryCfg, indexCfg));
task->add(subTask);
}
return task;
@@ -255,7 +251,8 @@ DocumentSubDBCollection::pruneRemovedFields(SerialNum serialNum)
void
-DocumentSubDBCollection::applyConfig(const DocumentDBConfig &newConfigSnapshot,
+DocumentSubDBCollection::applyConfig(const ProtonConfig & protonConfig,
+ const DocumentDBConfig &newConfigSnapshot,
const DocumentDBConfig &oldConfigSnapshot,
SerialNum serialNum,
const ReconfigParams &params,
@@ -264,7 +261,7 @@ DocumentSubDBCollection::applyConfig(const DocumentDBConfig &newConfigSnapshot,
_reprocessingRunner.reset();
for (auto subDb : _subDBs) {
IReprocessingTask::List tasks;
- tasks = subDb->applyConfig(newConfigSnapshot, oldConfigSnapshot,
+ tasks = subDb->applyConfig(protonConfig, newConfigSnapshot, oldConfigSnapshot,
serialNum, params, resolver);
_reprocessingRunner.addTasks(tasks);
}
diff --git a/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.h b/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.h
index a3acd68d377..9ad6732d50d 100644
--- a/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.h
+++ b/searchcore/src/vespa/searchcore/proton/server/documentsubdbcollection.h
@@ -51,9 +51,10 @@ namespace initializer { class InitializerTask; }
class DocumentSubDBCollection {
public:
- typedef std::vector<IDocumentSubDB *> SubDBVector;
- typedef SubDBVector::const_iterator const_iterator;
- typedef search::SerialNum SerialNum;
+ using SubDBVector = std::vector<IDocumentSubDB *>;
+ using const_iterator = SubDBVector::const_iterator;
+ using SerialNum = search::SerialNum;
+ using ProtonConfig = vespa::config::search::core::ProtonConfig;
private:
using IFeedViewSP = std::shared_ptr<IFeedView>;
@@ -65,9 +66,9 @@ private:
const uint32_t _readySubDbId;
const uint32_t _remSubDbId;
const uint32_t _notReadySubDbId;
- typedef std::shared_ptr<std::vector<std::shared_ptr<IDocumentRetriever>> > RetrieversSP;
+ using RetrieversSP = std::shared_ptr<std::vector<std::shared_ptr<IDocumentRetriever>> >;
vespalib::VarHolder<RetrieversSP> _retrievers;
- typedef std::vector<std::shared_ptr<IReprocessingTask>> ReprocessingTasks;
+ using ReprocessingTasks = std::vector<std::shared_ptr<IReprocessingTask>>;
ReprocessingRunner _reprocessingRunner;
std::shared_ptr<BucketDBOwner> _bucketDB;
std::unique_ptr<bucketdb::BucketDBHandler> _bucketDBHandler;
@@ -124,8 +125,7 @@ public:
}
std::shared_ptr<initializer::InitializerTask>
- createInitializer(const DocumentDBConfig &configSnapshot,
- SerialNum configSerialNum,
+ createInitializer(const DocumentDBConfig &configSnapshot, SerialNum configSerialNum,
const vespa::config::search::core::ProtonConfig::Summary &protonSummaryCfg,
const vespa::config::search::core::ProtonConfig::Index & indexCfg);
@@ -141,12 +141,9 @@ public:
void pruneRemovedFields(SerialNum serialNum);
- void
- applyConfig(const DocumentDBConfig &newConfigSnapshot,
- const DocumentDBConfig &oldConfigSnapshot,
- SerialNum serialNum,
- const ReconfigParams &params,
- IDocumentDBReferenceResolver &resolver);
+ void applyConfig(const ProtonConfig & bootstrapConfig, const DocumentDBConfig &newConfigSnapshot,
+ const DocumentDBConfig &oldConfigSnapshot, SerialNum serialNum,
+ const ReconfigParams &params, IDocumentDBReferenceResolver &resolver);
IFeedViewSP getFeedView();
IFlushTargetList getFlushTargets();
@@ -156,6 +153,4 @@ public:
void tearDownReferences(IDocumentDBReferenceResolver &resolver);
};
-
} // namespace proton
-
diff --git a/searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.cpp b/searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.cpp
index 692bb12213e..eedd54cfb92 100644
--- a/searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.cpp
@@ -11,7 +11,6 @@
#include <vespa/searchcore/proton/attribute/attribute_factory.h>
#include <vespa/searchcore/proton/attribute/attribute_manager_initializer.h>
#include <vespa/searchcore/proton/attribute/attribute_populator.h>
-#include <vespa/searchcore/proton/attribute/attribute_writer.h>
#include <vespa/searchcore/proton/attribute/filter_attribute_manager.h>
#include <vespa/searchcore/proton/attribute/sequential_attributes_initializer.h>
#include <vespa/searchcore/proton/metrics/legacy_documentdb_metrics.h>
@@ -215,14 +214,12 @@ FastAccessDocSubDB::FastAccessDocSubDB(const Config &cfg, const Context &ctx)
FastAccessDocSubDB::~FastAccessDocSubDB() { }
DocumentSubDbInitializer::UP
-FastAccessDocSubDB::createInitializer(const DocumentDBConfig &configSnapshot,
- SerialNum configSerialNum,
+FastAccessDocSubDB::createInitializer(const DocumentDBConfig &configSnapshot, SerialNum configSerialNum,
const vespa::config::search::core::ProtonConfig::Summary &protonSummaryCfg,
const vespa::config::search::core::ProtonConfig::Index &indexCfg) const
{
auto result = Parent::createInitializer(configSnapshot, configSerialNum, protonSummaryCfg, indexCfg);
- auto attrMgrInitTask = createAttributeManagerInitializer(configSnapshot,
- configSerialNum,
+ auto attrMgrInitTask = createAttributeManagerInitializer(configSnapshot, configSerialNum,
result->getDocumentMetaStoreInitTask(),
result->result().documentMetaStore()->documentMetaStore(),
result->writableResult().writableAttributeManager());
@@ -253,13 +250,13 @@ FastAccessDocSubDB::initViews(const DocumentDBConfig &configSnapshot,
}
IReprocessingTask::List
-FastAccessDocSubDB::applyConfig(const DocumentDBConfig &newConfigSnapshot,
- const DocumentDBConfig &oldConfigSnapshot,
- SerialNum serialNum,
- const ReconfigParams &params,
- IDocumentDBReferenceResolver &resolver)
+FastAccessDocSubDB::applyConfig(const ProtonConfig & protonConfig, const DocumentDBConfig &newConfigSnapshot,
+ const DocumentDBConfig &oldConfigSnapshot, SerialNum serialNum,
+ const ReconfigParams &params, IDocumentDBReferenceResolver &resolver)
{
(void) resolver;
+
+ reconfigure(protonConfig);
IReprocessingTask::List tasks;
updateLidReuseDelayer(&newConfigSnapshot);
/*
diff --git a/searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.h b/searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.h
index c89a63b95c0..ed5326475b1 100644
--- a/searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.h
+++ b/searchcore/src/vespa/searchcore/proton/server/fast_access_doc_subdb.h
@@ -94,30 +94,25 @@ protected:
void reconfigureAttributeMetrics(const IAttributeManager &newMgr, const IAttributeManager &oldMgr);
IReprocessingTask::UP
- createReprocessingTask(IReprocessingInitializer &initializer,
- const document::DocumentTypeRepo::SP &docTypeRepo) const;
+ createReprocessingTask(IReprocessingInitializer &initializer, const document::DocumentTypeRepo::SP &docTypeRepo) const;
public:
FastAccessDocSubDB(const Config &cfg, const Context &ctx);
-
~FastAccessDocSubDB();
- virtual std::unique_ptr<DocumentSubDbInitializer>
+ std::unique_ptr<DocumentSubDbInitializer>
createInitializer(const DocumentDBConfig &configSnapshot,
SerialNum configSerialNum,
const ProtonConfig::Summary &protonSummaryCfg,
const ProtonConfig::Index &indexCfg) const override;
void setup(const DocumentSubDbInitializerResult &initResult) override;
+ void initViews(const DocumentDBConfig &configSnapshot, const SessionManagerSP &sessionManager) override;
- void initViews(const DocumentDBConfig &configSnapshot,
- const SessionManagerSP &sessionManager) override;
-
- IReprocessingTask::List applyConfig(const DocumentDBConfig &newConfigSnapshot,
- const DocumentDBConfig &oldConfigSnapshot,
- SerialNum serialNum,
- const ReconfigParams &params,
- IDocumentDBReferenceResolver &resolver) override;
+ IReprocessingTask::List
+ applyConfig(const ProtonConfig & bootstrapConfig, const DocumentDBConfig &newConfigSnapshot,
+ const DocumentDBConfig &oldConfigSnapshot, SerialNum serialNum,
+ const ReconfigParams &params, IDocumentDBReferenceResolver &resolver) override;
proton::IAttributeManager::SP getAttributeManager() const override;
IDocumentRetriever::UP getDocumentRetriever() override;
diff --git a/searchcore/src/vespa/searchcore/proton/server/i_document_db_config_owner.h b/searchcore/src/vespa/searchcore/proton/server/i_document_db_config_owner.h
index 10a47e7c6e4..b9cbacfa78a 100644
--- a/searchcore/src/vespa/searchcore/proton/server/i_document_db_config_owner.h
+++ b/searchcore/src/vespa/searchcore/proton/server/i_document_db_config_owner.h
@@ -2,7 +2,7 @@
#pragma once
-#include <memory>
+#include <vespa/searchcore/config/config-proton.h>
namespace proton {
@@ -14,8 +14,10 @@ class DocumentDBConfig;
class IDocumentDBConfigOwner
{
public:
+ using ProtonConfig = vespa::config::search::core::ProtonConfig;
virtual ~IDocumentDBConfigOwner() { }
- virtual void reconfigure(const std::shared_ptr<DocumentDBConfig> & config) = 0;
+ virtual void reconfigure(const std::shared_ptr<ProtonConfig> & protonConfig,
+ const std::shared_ptr<DocumentDBConfig> & config) = 0;
};
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/server/idocumentsubdb.h b/searchcore/src/vespa/searchcore/proton/server/idocumentsubdb.h
index 586c34d8dcc..5f5a8e43f26 100644
--- a/searchcore/src/vespa/searchcore/proton/server/idocumentsubdb.h
+++ b/searchcore/src/vespa/searchcore/proton/server/idocumentsubdb.h
@@ -7,10 +7,7 @@
#include <vespa/searchlib/common/serialnum.h>
#include <vespa/searchlib/util/searchable_stats.h>
-
-namespace search::index {
- class Schema;
-}
+namespace search::index { class Schema; }
namespace document { class DocumentId; }
@@ -73,11 +70,9 @@ public:
virtual void initViews(const DocumentDBConfig &configSnapshot, const std::shared_ptr<matching::SessionManager> &sessionManager) = 0;
virtual IReprocessingTask::List
- applyConfig(const DocumentDBConfig &newConfigSnapshot,
- const DocumentDBConfig &oldConfigSnapshot,
- SerialNum serialNum,
- const ReconfigParams &params,
- IDocumentDBReferenceResolver &resolver) = 0;
+ applyConfig(const ProtonConfig & bootstrapConfig, const DocumentDBConfig &newConfigSnapshot,
+ const DocumentDBConfig &oldConfigSnapshot, SerialNum serialNum,
+ const ReconfigParams &params, IDocumentDBReferenceResolver &resolver) = 0;
virtual std::shared_ptr<ISearchHandler> getSearchView() const = 0;
virtual std::shared_ptr<IFeedView> getFeedView() const = 0;
diff --git a/searchcore/src/vespa/searchcore/proton/server/memoryflush.cpp b/searchcore/src/vespa/searchcore/proton/server/memoryflush.cpp
index 2d56c7e37e9..e80f2645fcf 100644
--- a/searchcore/src/vespa/searchcore/proton/server/memoryflush.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/memoryflush.cpp
@@ -32,8 +32,7 @@ getName(const IFlushHandler & handler, const IFlushTarget & target)
static constexpr uint64_t gibi = UINT64_C(1024) * UINT64_C(1024) * UINT64_C(1024);
uint64_t
-estimateNeededTlsSizeForFlushTarget(const TlsStats &tlsStats,
- SerialNum flushedSerialNum)
+estimateNeededTlsSizeForFlushTarget(const TlsStats &tlsStats, SerialNum flushedSerialNum)
{
if (flushedSerialNum < tlsStats.getFirstSerial()) {
return tlsStats.getNumBytes();
@@ -45,8 +44,7 @@ estimateNeededTlsSizeForFlushTarget(const TlsStats &tlsStats,
if (flushedSerialNum >= tlsStats.getLastSerial()) {
return 0u;
}
- double bytesPerEntry = static_cast<double>(tlsStats.getNumBytes()) /
- numEntries;
+ double bytesPerEntry = static_cast<double>(tlsStats.getNumBytes()) / numEntries;
return bytesPerEntry * (tlsStats.getLastSerial() - flushedSerialNum);
}
@@ -89,13 +87,15 @@ MemoryFlush::MemoryFlush()
MemoryFlush::~MemoryFlush() { }
-MemoryFlush::Config MemoryFlush::getConfig() const
+MemoryFlush::Config
+MemoryFlush::getConfig() const
{
vespalib::LockGuard guard(_lock);
return _config;
}
-void MemoryFlush::setConfig(const Config &config)
+void
+MemoryFlush::setConfig(const Config &config)
{
vespalib::LockGuard guard(_lock);
_config = config;
@@ -116,7 +116,8 @@ getOrderName(MemoryFlush::OrderType &orderType)
return "DEFAULT";
}
-size_t computeGain(const IFlushTarget::DiskGain & gain) {
+size_t
+computeGain(const IFlushTarget::DiskGain & gain) {
return std::max(100000000l, std::max(gain.getBefore(), gain.getAfter()));
}
bool isDiskBloatToHigh(const IFlushTarget::DiskGain & totalDisk,
@@ -131,8 +132,7 @@ bool isDiskBloatToHigh(const IFlushTarget::DiskGain & totalDisk,
FlushContext::List
MemoryFlush::getFlushTargets(const FlushContext::List &targetList,
- const flushengine::TlsStatsMap &
- tlsStatsMap) const
+ const flushengine::TlsStatsMap & tlsStatsMap) const
{
OrderType order(DEFAULT);
uint64_t totalMemory(0);
@@ -219,8 +219,7 @@ MemoryFlush::getFlushTargets(const FlushContext::List &targetList,
bool
-MemoryFlush::CompareTarget::operator()(const FlushContext::SP &lfc,
- const FlushContext::SP &rfc) const
+MemoryFlush::CompareTarget::operator()(const FlushContext::SP &lfc, const FlushContext::SP &rfc) const
{
const IFlushTarget &lhs = *lfc->getTarget();
const IFlushTarget &rhs = *rfc->getTarget();
diff --git a/searchcore/src/vespa/searchcore/proton/server/memoryflush.h b/searchcore/src/vespa/searchcore/proton/server/memoryflush.h
index ae52a497be8..552a406aad9 100644
--- a/searchcore/src/vespa/searchcore/proton/server/memoryflush.h
+++ b/searchcore/src/vespa/searchcore/proton/server/memoryflush.h
@@ -47,17 +47,13 @@ private:
class CompareTarget
{
public:
- CompareTarget(OrderType order,
- const flushengine::TlsStatsMap &tlsStatsMap)
+ CompareTarget(OrderType order, const flushengine::TlsStatsMap &tlsStatsMap)
: _order(order),
_tlsStatsMap(tlsStatsMap)
{ }
- bool
- operator ()(const FlushContext::SP &lfc,
- const FlushContext::SP &rfc) const;
+ bool operator ()(const FlushContext::SP &lfc, const FlushContext::SP &rfc) const;
private:
-
OrderType _order;
const flushengine::TlsStatsMap &_tlsStatsMap;
};
@@ -66,20 +62,16 @@ public:
using SP = std::shared_ptr<MemoryFlush>;
MemoryFlush();
-
- MemoryFlush(const Config &config,
- fastos::TimeStamp startTime = fastos::TimeStamp(fastos::ClockSystem::now()));
+ explicit MemoryFlush(const Config &config) : MemoryFlush(config, fastos::ClockSystem::now()) { }
+ MemoryFlush(const Config &config, fastos::TimeStamp startTime);
~MemoryFlush();
- // Implements IFlushStrategy
- virtual FlushContext::List
+ FlushContext::List
getFlushTargets(const FlushContext::List &targetList,
- const flushengine::TlsStatsMap &
- tlsStatsMap) const override;
+ const flushengine::TlsStatsMap &tlsStatsMap) const override;
void setConfig(const Config &config);
Config getConfig() const;
};
} // namespace proton
-
diff --git a/searchcore/src/vespa/searchcore/proton/server/proton.cpp b/searchcore/src/vespa/searchcore/proton/server/proton.cpp
index 38ec899051d..bdd0f101f51 100644
--- a/searchcore/src/vespa/searchcore/proton/server/proton.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/proton.cpp
@@ -258,7 +258,7 @@ Proton::init(const BootstrapConfig::SP & configSnapshot)
switch (flush.strategy) {
case ProtonConfig::Flush::MEMORY: {
MemoryFlush::SP memoryFlush = std::make_shared<MemoryFlush>(
- MemoryFlushConfigUpdater::convertConfig(flush.memory));
+ MemoryFlushConfigUpdater::convertConfig(flush.memory), fastos::ClockSystem::now());
_memoryFlushConfigUpdater = std::make_unique<MemoryFlushConfigUpdater>(memoryFlush, flush.memory);
_diskMemUsageSampler->notifier().addDiskMemUsageListener(_memoryFlushConfigUpdater.get());
strategy = memoryFlush;
@@ -518,7 +518,7 @@ Proton::addDocumentDB(const document::DocumentType &docType,
const DocumentDBConfig::SP &documentDBConfig,
InitializeThreads initializeThreads)
{
- const ProtonConfig &config(*bootstrapConfig->getProtonConfigSP());
+ const ProtonConfig &config(bootstrapConfig->getProtonConfig());
std::lock_guard<std::shared_timed_mutex> guard(_mutex);
DocTypeName docTypeName(docType.getName());
@@ -547,7 +547,7 @@ Proton::addDocumentDB(const document::DocumentType &docType,
_queryLimiter,
_clock,
docTypeName,
- config,
+ bootstrapConfig->getProtonConfigSP(),
*this,
*_warmupExecutor,
*_summaryExecutor,
diff --git a/searchcore/src/vespa/searchcore/proton/server/proton_configurer.cpp b/searchcore/src/vespa/searchcore/proton/server/proton_configurer.cpp
index fe3314e7976..ade934f10b5 100644
--- a/searchcore/src/vespa/searchcore/proton/server/proton_configurer.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/proton_configurer.cpp
@@ -140,7 +140,7 @@ ProtonConfigurer::configureDocumentDB(const ProtonConfigSnapshot &configSnapshot
assert(insres.second);
}
} else {
- dbitr->second->reconfigure(documentDBConfig);
+ dbitr->second->reconfigure(bootstrapConfig->getProtonConfigSP(), documentDBConfig);
}
}
diff --git a/searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.cpp b/searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.cpp
index df1417f0d73..e19e4502246 100644
--- a/searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.cpp
@@ -112,8 +112,7 @@ createIndexManagerInitializer(const DocumentDBConfig &configSnapshot,
}
void
-SearchableDocSubDB::setupIndexManager(searchcorespi::IIndexManager::SP
- indexManager)
+SearchableDocSubDB::setupIndexManager(searchcorespi::IIndexManager::SP indexManager)
{
_indexMgr = indexManager;
_indexWriter.reset(new IndexWriter(_indexMgr));
@@ -126,15 +125,9 @@ createInitializer(const DocumentDBConfig &configSnapshot,
const ProtonConfig::Summary &protonSummaryCfg,
const ProtonConfig::Index &indexCfg) const
{
- auto result = Parent::createInitializer(configSnapshot,
- configSerialNum,
- protonSummaryCfg,
- indexCfg);
- auto indexTask = createIndexManagerInitializer(configSnapshot,
- configSerialNum,
- indexCfg,
- result->writableResult().
- writableIndexManager());
+ auto result = Parent::createInitializer(configSnapshot, configSerialNum, protonSummaryCfg, indexCfg);
+ auto indexTask = createIndexManagerInitializer(configSnapshot, configSerialNum, indexCfg,
+ result->writableResult().writableIndexManager());
result->addDependency(indexTask);
return result;
}
@@ -166,12 +159,11 @@ reconfigureMatchingMetrics(const RankProfilesConfig &cfg)
}
IReprocessingTask::List
-SearchableDocSubDB::applyConfig(const DocumentDBConfig &newConfigSnapshot,
- const DocumentDBConfig &oldConfigSnapshot,
- SerialNum serialNum,
- const ReconfigParams &params,
- IDocumentDBReferenceResolver &resolver)
+SearchableDocSubDB::applyConfig(const ProtonConfig & protonConfig, const DocumentDBConfig &newConfigSnapshot,
+ const DocumentDBConfig &oldConfigSnapshot, SerialNum serialNum,
+ const ReconfigParams &params, IDocumentDBReferenceResolver &resolver)
{
+ StoreOnlyDocSubDB::reconfigure(protonConfig);
IReprocessingTask::List tasks;
updateLidReuseDelayer(&newConfigSnapshot);
if (params.shouldMatchersChange() && _addMetrics) {
@@ -199,8 +191,7 @@ SearchableDocSubDB::applyConfig(const DocumentDBConfig &newConfigSnapshot,
}
void
-SearchableDocSubDB::initViews(const DocumentDBConfig &configSnapshot,
- const SessionManager::SP &sessionManager)
+SearchableDocSubDB::initViews(const DocumentDBConfig &configSnapshot, const SessionManager::SP &sessionManager)
{
assert(_writeService.master().isCurrentThread());
@@ -208,16 +199,9 @@ SearchableDocSubDB::initViews(const DocumentDBConfig &configSnapshot,
const Schema::SP &schema = configSnapshot.getSchemaSP();
const IIndexManager::SP &indexMgr = getIndexManager();
_constantValueRepo.reconfigure(configSnapshot.getRankingConstants());
- Matchers::SP matchers(_configurer.
- createMatchers(schema,
- configSnapshot.getRankProfilesConfig()).
- release());
- MatchView::SP matchView(new MatchView(matchers,
- indexMgr->getSearchable(),
- attrMgr,
- sessionManager,
- _metaStoreCtx,
- _docIdLimit));
+ Matchers::SP matchers(_configurer.createMatchers(schema, configSnapshot.getRankProfilesConfig()).release());
+ MatchView::SP matchView(new MatchView(matchers, indexMgr->getSearchable(), attrMgr,
+ sessionManager, _metaStoreCtx, _docIdLimit));
_rSearchView.set(SearchView::SP(
new SearchView(
getSummaryManager()->createSummarySetup(
@@ -370,4 +354,11 @@ SearchableDocSubDB::tearDownReferences(IDocumentDBReferenceResolver &resolver)
resolver.teardown(*attrMgr);
}
+void
+SearchableDocSubDB::clearViews() {
+ _rFeedView.clear();
+ _rSearchView.clear();
+ Parent::clearViews();
+}
+
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.h b/searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.h
index 91ce868937b..dc933c92aab 100644
--- a/searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.h
+++ b/searchcore/src/vespa/searchcore/proton/server/searchabledocsubdb.h
@@ -96,9 +96,6 @@ private:
void initFeedView(const IAttributeWriter::SP &attrWriter, const DocumentDBConfig &configSnapshot);
void reconfigureMatchingMetrics(const vespa::config::search::RankProfilesConfig &config);
- /**
- * Implements IndexManagerReconfigurer API.
- */
bool reconfigure(vespalib::Closure0<bool>::UP closure) override;
void reconfigureIndexSearchable();
void syncViews();
@@ -113,32 +110,19 @@ public:
~SearchableDocSubDB();
std::unique_ptr<DocumentSubDbInitializer>
- createInitializer(const DocumentDBConfig &configSnapshot,
- SerialNum configSerialNum,
- const vespa::config::search::core::
- ProtonConfig::Summary &protonSummaryCfg,
- const vespa::config::search::core::
- ProtonConfig::Index &indexCfg) const override;
+ createInitializer(const DocumentDBConfig &configSnapshot, SerialNum configSerialNum,
+ const vespa::config::search::core::ProtonConfig::Summary &protonSummaryCfg,
+ const vespa::config::search::core::ProtonConfig::Index &indexCfg) const override;
void setup(const DocumentSubDbInitializerResult &initResult) override;
-
- void
- initViews(const DocumentDBConfig &configSnapshot,
- const SessionManagerSP &sessionManager) override;
+ void initViews(const DocumentDBConfig &configSnapshot, const SessionManagerSP &sessionManager) override;
IReprocessingTask::List
- applyConfig(const DocumentDBConfig &newConfigSnapshot,
- const DocumentDBConfig &oldConfigSnapshot,
- SerialNum serialNum,
- const ReconfigParams &params,
- IDocumentDBReferenceResolver &resolver) override;
-
- void clearViews() override
- {
- _rFeedView.clear();
- _rSearchView.clear();
- Parent::clearViews();
- }
+ applyConfig(const ProtonConfig & protonConfig, const DocumentDBConfig &newConfigSnapshot,
+ const DocumentDBConfig &oldConfigSnapshot, SerialNum serialNum,
+ const ReconfigParams &params, IDocumentDBReferenceResolver &resolver) override;
+
+ void clearViews() override;
proton::IAttributeManager::SP getAttributeManager() const override {
return _rSearchView.get()->getAttributeManager();
@@ -159,10 +143,9 @@ public:
search::SearchableStats getSearchableStats() const override ;
IDocumentRetriever::UP getDocumentRetriever() override;
matching::MatchingStats getMatcherStats(const vespalib::string &rankProfile) const override;
- virtual void close() override;
- virtual std::shared_ptr<IDocumentDBReference> getDocumentDBReference() override;
- virtual void tearDownReferences(IDocumentDBReferenceResolver &resolver) override;
+ void close() override;
+ std::shared_ptr<IDocumentDBReference> getDocumentDBReference() override;
+ void tearDownReferences(IDocumentDBReferenceResolver &resolver) override;
};
} // namespace proton
-
diff --git a/searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.cpp b/searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.cpp
index e92c11c2fb8..f653e133486 100644
--- a/searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.cpp
+++ b/searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.cpp
@@ -62,7 +62,6 @@ IIndexWriter::SP nullIndexWriter;
}
-
StoreOnlyDocSubDB::Config::Config(const DocTypeName &docTypeName,
const vespalib::string &subName,
const vespalib::string &baseDir,
@@ -152,6 +151,12 @@ StoreOnlyDocSubDB::~StoreOnlyDocSubDB()
_rSummaryMgr.reset();
}
+void
+StoreOnlyDocSubDB::clearViews() {
+ _iFeedView.clear();
+ _iSearchView.clear();
+}
+
size_t
StoreOnlyDocSubDB::getNumDocs() const
{
@@ -241,9 +246,7 @@ createSummaryManagerInitializer(const ProtonConfig::Summary protonSummaryCfg,
vespalib::string baseDir(_baseDir + "/summary");
return std::make_shared<SummaryManagerInitializer>
(grow, baseDir, getSubDbName(), _docTypeName, _summaryExecutor,
- protonSummaryCfg,
- tuneFile,
- _fileHeaderContext, _tlSyncer, bucketizer, result);
+ protonSummaryCfg, tuneFile, _fileHeaderContext, _tlSyncer, bucketizer, result);
}
void
@@ -251,8 +254,7 @@ StoreOnlyDocSubDB::setupSummaryManager(SummaryManager::SP summaryManager)
{
_rSummaryMgr = summaryManager;
_iSummaryMgr = _rSummaryMgr; // Upcast allowed with std::shared_ptr
- _flushedDocumentStoreSerialNum =
- _iSummaryMgr->getBackingStore().lastSyncToken();
+ _flushedDocumentStoreSerialNum = _iSummaryMgr->getBackingStore().lastSyncToken();
_summaryAdapter.reset(new SummaryAdapter(_rSummaryMgr));
}
@@ -274,15 +276,9 @@ createDocumentMetaStoreInitializer(const search::TuneFileAttributes &tuneFile,
// initializers to get hold of document meta store instance in
// their constructors.
*result = std::make_shared<DocumentMetaStoreInitializerResult>
- (std::make_shared<DocumentMetaStore>(_bucketDB, attrFileName,
- grow,
- gidCompare, _subDbType),
- tuneFile);
+ (std::make_shared<DocumentMetaStore>(_bucketDB, attrFileName, grow, gidCompare, _subDbType), tuneFile);
return std::make_shared<documentmetastore::DocumentMetaStoreInitializer>
- (baseDir,
- getSubDbName(),
- _docTypeName.toString(),
- (*result)->documentMetaStore());
+ (baseDir, getSubDbName(), _docTypeName.toString(), (*result)->documentMetaStore());
}
@@ -293,29 +289,21 @@ StoreOnlyDocSubDB::setupDocumentMetaStore(DocumentMetaStoreInitializerResult::SP
vespalib::string name = DocumentMetaStore::getFixedName();
DocumentMetaStore::SP dms(dmsResult->documentMetaStore());
if (dms->isLoaded()) {
- _flushedDocumentMetaStoreSerialNum =
- dms->getStatus().getLastSyncToken();
+ _flushedDocumentMetaStoreSerialNum = dms->getStatus().getLastSyncToken();
}
_bucketDBHandlerInitializer.
- addDocumentMetaStore(dms.get(),
- _flushedDocumentMetaStoreSerialNum);
+ addDocumentMetaStore(dms.get(), _flushedDocumentMetaStoreSerialNum);
_metaStoreCtx.reset(new DocumentMetaStoreContext(dms));
LOG(debug, "Added document meta store '%s' with flushed serial num %lu",
name.c_str(), _flushedDocumentMetaStoreSerialNum);
_dms = dms;
- _dmsFlushTarget.reset(new DocumentMetaStoreFlushTarget(dms,
- _tlsSyncer,
- baseDir,
- dmsResult->tuneFile(),
- _fileHeaderContext, _hwInfo));
+ _dmsFlushTarget = std::make_shared<DocumentMetaStoreFlushTarget>(dms, _tlsSyncer, baseDir, dmsResult->tuneFile(),
+ _fileHeaderContext, _hwInfo);
using Type = IFlushTarget::Type;
using Component = IFlushTarget::Component;
- _dmsShrinkTarget = std::make_shared<ShrinkLidSpaceFlushTarget>
- ("documentmetastore.shrink",
- Type::GC, Component::ATTRIBUTE,
- _flushedDocumentMetaStoreSerialNum,
- _dmsFlushTarget->getLastFlushTime(),
- dms);
+ _dmsShrinkTarget = std::make_shared<ShrinkLidSpaceFlushTarget>("documentmetastore.shrink", Type::GC,
+ Component::ATTRIBUTE, _flushedDocumentMetaStoreSerialNum,
+ _dmsFlushTarget->getLastFlushTime(), dms);
}
DocumentSubDbInitializer::UP
@@ -331,19 +319,14 @@ StoreOnlyDocSubDB::createInitializer(const DocumentDBConfig &configSnapshot,
(const_cast<StoreOnlyDocSubDB &>(*this),
_writeService.master());
auto dmsInitTask =
- createDocumentMetaStoreInitializer(configSnapshot.
- getTuneFileDocumentDBSP()->_attr,
- result->writableResult().
- writableDocumentMetaStore());
+ createDocumentMetaStoreInitializer(configSnapshot.getTuneFileDocumentDBSP()->_attr,
+ result->writableResult().writableDocumentMetaStore());
result->addDocumentMetaStoreInitTask(dmsInitTask);
auto summaryTask =
createSummaryManagerInitializer(protonSummaryCfg,
- configSnapshot.
- getTuneFileDocumentDBSP()->_summary,
- result->result().documentMetaStore()->
- documentMetaStore(),
- result->writableResult().
- writableSummaryManager());
+ configSnapshot.getTuneFileDocumentDBSP()->_summary,
+ result->result().documentMetaStore()->documentMetaStore(),
+ result->writableResult().writableSummaryManager());
result->addDependency(summaryTask);
summaryTask->addDependency(dmsInitTask);
@@ -367,9 +350,7 @@ StoreOnlyDocSubDB::getFlushTargets()
IFlushTarget::List ret;
for (const auto &target : getFlushTargetsInternal()) {
ret.push_back(IFlushTarget::SP
- (new ThreadedFlushTarget(_writeService.master(),
- _getSerialNum,
- target, _subName)));
+ (new ThreadedFlushTarget(_writeService.master(), _getSerialNum, target, _subName)));
}
return ret;
}
@@ -400,12 +381,8 @@ StoreOnlyDocSubDB::getFeedViewPersistentParams()
{
SerialNum flushedDMSSN(_flushedDocumentMetaStoreSerialNum);
SerialNum flushedDSSN(_flushedDocumentStoreSerialNum);
- return StoreOnlyFeedView::PersistentParams(flushedDMSSN,
- flushedDSSN,
- _docTypeName,
- _metrics.feed,
- _subDbId,
- _subDbType);
+ return StoreOnlyFeedView::PersistentParams(flushedDMSSN, flushedDSSN, _docTypeName,
+ _metrics.feed, _subDbId, _subDbType);
}
void
@@ -460,23 +437,28 @@ StoreOnlyDocSubDB::updateLidReuseDelayer(const LidReuseDelayerConfig &config)
}
IReprocessingTask::List
-StoreOnlyDocSubDB::applyConfig(const DocumentDBConfig &newConfigSnapshot,
- const DocumentDBConfig &oldConfigSnapshot,
- SerialNum serialNum,
- const ReconfigParams &params,
- IDocumentDBReferenceResolver &resolver)
+StoreOnlyDocSubDB::applyConfig(const ProtonConfig & protonConfig, const DocumentDBConfig &newConfigSnapshot,
+ const DocumentDBConfig &oldConfigSnapshot, SerialNum serialNum,
+ const ReconfigParams &params, IDocumentDBReferenceResolver &resolver)
{
(void) oldConfigSnapshot;
(void) serialNum;
(void) params;
(void) resolver;
assert(_writeService.master().isCurrentThread());
+ reconfigure(protonConfig);
initFeedView(newConfigSnapshot);
updateLidReuseDelayer(&newConfigSnapshot);
_owner.syncFeedView();
return IReprocessingTask::List();
}
+void
+StoreOnlyDocSubDB::reconfigure(const ProtonConfig & config)
+{
+ _rSummaryMgr->reconfigure(config.summary);
+}
+
proton::IAttributeManager::SP
StoreOnlyDocSubDB::getAttributeManager() const
{
@@ -579,8 +561,7 @@ StoreOnlyDocSubDB::tearDownReferences(IDocumentDBReferenceResolver &resolver)
void
StoreOnlySubDBFileHeaderContext::
-addTags(vespalib::GenericHeader &header,
- const vespalib::string &name) const
+addTags(vespalib::GenericHeader &header, const vespalib::string &name) const
{
_parentFileHeaderContext.addTags(header, name);
typedef GenericHeader::Tag Tag;
diff --git a/searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.h b/searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.h
index d62a92d33a8..f39d0173d5e 100644
--- a/searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.h
+++ b/searchcore/src/vespa/searchcore/proton/server/storeonlydocsubdb.h
@@ -98,13 +98,9 @@ public:
const uint32_t _subDbId;
const SubDbType _subDbType;
- Config(const DocTypeName &docTypeName,
- const vespalib::string &subName,
- const vespalib::string &baseDir,
- const search::GrowStrategy &attributeGrow,
- size_t attributeGrowNumDocs,
- uint32_t subDbId,
- SubDbType subDbType);
+ Config(const DocTypeName &docTypeName, const vespalib::string &subName,
+ const vespalib::string &baseDir, const search::GrowStrategy &attributeGrow,
+ size_t attributeGrowNumDocs, uint32_t subDbId, SubDbType subDbType);
~Config();
};
@@ -201,7 +197,7 @@ protected:
using LidReuseDelayerConfig = documentmetastore::LidReuseDelayerConfig;
virtual void updateLidReuseDelayer(const LidReuseDelayerConfig &config);
-
+ void reconfigure(const ProtonConfig & protonConfig);
public:
StoreOnlyDocSubDB(const Config &cfg, const Context &ctx);
~StoreOnlyDocSubDB();
@@ -210,35 +206,22 @@ public:
vespalib::string getName() const override { return _subName; }
std::unique_ptr<DocumentSubDbInitializer>
- createInitializer(const DocumentDBConfig &configSnapshot,
- SerialNum configSerialNum,
- const ProtonConfig::Summary &protonSummaryCfg,
- const ProtonConfig::Index &indexCfg) const override;
+ createInitializer(const DocumentDBConfig &configSnapshot, SerialNum configSerialNum,
+ const ProtonConfig::Summary &protonSummaryCfg, const ProtonConfig::Index &indexCfg) const override;
void setup(const DocumentSubDbInitializerResult &initResult) override;
void initViews(const DocumentDBConfig &configSnapshot, const std::shared_ptr<matching::SessionManager> &sessionManager) override;
IReprocessingTask::List
- applyConfig(const DocumentDBConfig &newConfigSnapshot,
- const DocumentDBConfig &oldConfigSnapshot,
- SerialNum serialNum,
- const ReconfigParams &params,
- IDocumentDBReferenceResolver &resolver) override;
+ applyConfig(const ProtonConfig & protonConfig, const DocumentDBConfig &newConfigSnapshot,
+ const DocumentDBConfig &oldConfigSnapshot, SerialNum serialNum,
+ const ReconfigParams &params, IDocumentDBReferenceResolver &resolver) override;
ISearchHandler::SP getSearchView() const override { return _iSearchView.get(); }
IFeedView::SP getFeedView() const override { return _iFeedView.get(); }
- void clearViews() override {
- _iFeedView.clear();
- _iSearchView.clear();
- }
-
- /**
- * Returns the summary manager that this database uses to manage
- * document summaries of the corresponding document type.
- *
- * @return The summary manager.
- */
+ void clearViews() override;
+
const ISummaryManager::SP &getSummaryManager() const override { return _iSummaryMgr; }
IAttributeManager::SP getAttributeManager() const override;
const std::shared_ptr<searchcorespi::IIndexManager> & getIndexManager() const override;
@@ -259,8 +242,8 @@ public:
IDocumentRetriever::UP getDocumentRetriever() override;
matching::MatchingStats getMatcherStats(const vespalib::string &rankProfile) const override;
void close() override;
- virtual std::shared_ptr<IDocumentDBReference> getDocumentDBReference() override;
- virtual void tearDownReferences(IDocumentDBReferenceResolver &resolver) override;
+ std::shared_ptr<IDocumentDBReference> getDocumentDBReference() override;
+ void tearDownReferences(IDocumentDBReferenceResolver &resolver) override;
};
-} // namespace proton
+}
diff --git a/searchcore/src/vespa/searchcore/proton/test/dummy_document_sub_db.h b/searchcore/src/vespa/searchcore/proton/test/dummy_document_sub_db.h
index 4049b9c8c0a..8d20270df9d 100644
--- a/searchcore/src/vespa/searchcore/proton/test/dummy_document_sub_db.h
+++ b/searchcore/src/vespa/searchcore/proton/test/dummy_document_sub_db.h
@@ -15,9 +15,7 @@
#include <vespa/searchcore/proton/persistenceengine/i_document_retriever.h>
#include <vespa/searchcore/proton/server/reconfig_params.h>
-namespace proton {
-
-namespace test {
+namespace proton::test {
struct DummyDocumentSubDb : public IDocumentSubDB
{
@@ -45,20 +43,15 @@ struct DummyDocumentSubDb : public IDocumentSubDB
uint32_t getSubDbId() const override { return _subDbId; }
vespalib::string getName() const override { return "dummysubdb"; }
DocumentSubDbInitializer::UP
- createInitializer(const DocumentDBConfig &,
- SerialNum,
- const vespa::config::search::core::ProtonConfig::
- Summary &,
- const vespa::config::search::core::
- ProtonConfig::Index &) const override {
+ createInitializer(const DocumentDBConfig &, SerialNum, const vespa::config::search::core::ProtonConfig::Summary &,
+ const vespa::config::search::core::ProtonConfig::Index &) const override {
return std::make_unique<DocumentSubDbInitializer>
(const_cast<DummyDocumentSubDb &>(*this),
_writeService->master());
}
void setup(const DocumentSubDbInitializerResult &) override {}
- void initViews(const DocumentDBConfig &,
- const proton::matching::SessionManager::SP &) override {}
- IReprocessingTask::List applyConfig(const DocumentDBConfig &, const DocumentDBConfig &,
+ void initViews(const DocumentDBConfig &, const proton::matching::SessionManager::SP &) override {}
+ IReprocessingTask::List applyConfig(const ProtonConfig &, const DocumentDBConfig &, const DocumentDBConfig &,
SerialNum, const ReconfigParams &, IDocumentDBReferenceResolver &) override
{
return IReprocessingTask::List();
@@ -93,13 +86,10 @@ struct DummyDocumentSubDb : public IDocumentSubDB
matching::MatchingStats getMatcherStats(const vespalib::string &) const override {
return matching::MatchingStats();
}
- virtual std::shared_ptr<IDocumentDBReference> getDocumentDBReference() override {
+ std::shared_ptr<IDocumentDBReference> getDocumentDBReference() override {
return std::shared_ptr<IDocumentDBReference>();
}
- virtual void tearDownReferences(IDocumentDBReferenceResolver &) override { }
+ void tearDownReferences(IDocumentDBReferenceResolver &) override { }
};
-} // namespace test
-
-} // namespace proton
-
+}
diff --git a/searchlib/src/vespa/searchlib/docstore/documentstore.cpp b/searchlib/src/vespa/searchlib/docstore/documentstore.cpp
index bf59614a297..63b924d5cc2 100644
--- a/searchlib/src/vespa/searchlib/docstore/documentstore.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/documentstore.cpp
@@ -110,17 +110,19 @@ private:
class BackingStore {
public:
BackingStore(IDataStore &store, const CompressionConfig &compression) :
- _backingStore(store),
- _compression(compression) { }
+ _backingStore(store),
+ _compression(compression)
+ { }
bool read(DocumentIdT key, Value &value) const;
void visit(const IDocumentStore::LidVector &lids, const DocumentTypeRepo &repo, IDocumentVisitor &visitor) const;
void write(DocumentIdT, const Value &) {}
void erase(DocumentIdT) {}
- const CompressionConfig &getCompression(void) const { return _compression; }
+ const CompressionConfig &getCompression() const { return _compression; }
+ void reconfigure(const CompressionConfig &compression);
private:
IDataStore &_backingStore;
- const CompressionConfig _compression;
+ CompressionConfig _compression;
};
void
@@ -172,6 +174,11 @@ BackingStore::read(DocumentIdT key, Value &value) const {
return found;
}
+void
+BackingStore::reconfigure(const CompressionConfig &compression) {
+ _compression = compression;
+}
+
}
using CacheParams = vespalib::CacheParam<
@@ -201,8 +208,15 @@ DocumentStore::DocumentStore(const Config & config, IDataStore & store)
_cache->reserveElements(config.getInitialCacheEntries());
}
-DocumentStore::~DocumentStore()
-{
+DocumentStore::~DocumentStore() {}
+
+void
+DocumentStore::reconfigure(const Config & config) {
+ _cache->setCapacityBytes(config.getMaxCacheBytes());
+ _store->reconfigure(config.getCompression());
+ _visitCache->reconfigure(_config.getMaxCacheBytes(), config.getCompression());
+
+ _config = config;
}
bool
diff --git a/searchlib/src/vespa/searchlib/docstore/documentstore.h b/searchlib/src/vespa/searchlib/docstore/documentstore.h
index 4ba5c27cd07..af173862295 100644
--- a/searchlib/src/vespa/searchlib/docstore/documentstore.h
+++ b/searchlib/src/vespa/searchlib/docstore/documentstore.h
@@ -100,6 +100,7 @@ public:
bool canShrinkLidSpace() const override;
size_t getEstimatedShrinkLidSpaceGain() const override;
void shrinkLidSpace() override;
+ void reconfigure(const Config & config);
private:
bool useCache() const;
diff --git a/searchlib/src/vespa/searchlib/docstore/logdatastore.cpp b/searchlib/src/vespa/searchlib/docstore/logdatastore.cpp
index 4fa4142813c..53455cb1223 100644
--- a/searchlib/src/vespa/searchlib/docstore/logdatastore.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/logdatastore.cpp
@@ -67,6 +67,10 @@ LogDataStore::LogDataStore(vespalib::ThreadExecutor &executor,
updateSerialNum();
}
+void LogDataStore::reconfigure(const Config & config) {
+ _config = config;
+}
+
void
LogDataStore::updateSerialNum()
{
@@ -274,12 +278,9 @@ LogDataStore::compact(uint64_t syncToken)
uint64_t usage = getDiskFootprint();
uint64_t bloat = getDiskBloat();
LOG(debug, "%s", bloatMsg(bloat, usage).c_str());
- if ((_fileChunks.size() > 1) &&
- ( isBucketSpreadTooLarge(getMaxBucketSpread()) ||
- isBloatOverLimit(bloat, usage)))
- {
+ if (_fileChunks.size() > 1) {
LOG(info, "%s. Will compact", bloatMsg(bloat, usage).c_str());
- compactWorst();
+ compactWorst(_config.getMaxDiskBloatFactor(), _config.getMaxBucketSpread());
usage = getDiskFootprint();
bloat = getDiskBloat();
LOG(info, "Done compacting. %s", bloatMsg(bloat, usage).c_str());
@@ -299,7 +300,7 @@ LogDataStore::getMaxCompactGain() const
bloat = 0;
}
size_t spreadAsBloat = diskFootPrint * (1.0 - 1.0/maxSpread);
- if ( ! isBucketSpreadTooLarge(maxSpread)) {
+ if ( maxSpread < _config.getMaxBucketSpread()) {
spreadAsBloat = 0;
}
return (bloat + spreadAsBloat);
@@ -348,7 +349,7 @@ LogDataStore::getMaxBucketSpread() const
}
std::pair<bool, LogDataStore::FileId>
-LogDataStore::findNextToCompact()
+LogDataStore::findNextToCompact(double bloatLimit, double spreadLimit)
{
typedef std::multimap<double, FileId, std::greater<double>> CostMap;
CostMap worstBloat;
@@ -376,10 +377,10 @@ LogDataStore::findNextToCompact()
}
}
std::pair<bool, FileId> retval(false, FileId(-1));
- if ( ! worstBloat.empty() && (worstBloat.begin()->first > _config.getMaxDiskBloatFactor())) {
+ if ( ! worstBloat.empty() && (worstBloat.begin()->first > bloatLimit)) {
retval.first = true;
retval.second = worstBloat.begin()->second;
- } else if ( ! worstSpread.empty() && (worstSpread.begin()->first > _config.getMaxBucketSpread())) {
+ } else if ( ! worstSpread.empty() && (worstSpread.begin()->first > spreadLimit)) {
retval.first = true;
retval.second = worstSpread.begin()->second;
}
@@ -390,8 +391,8 @@ LogDataStore::findNextToCompact()
}
void
-LogDataStore::compactWorst() {
- auto worst = findNextToCompact();
+LogDataStore::compactWorst(double bloatLimit, double spreadLimit) {
+ auto worst = findNextToCompact(bloatLimit, spreadLimit);
if (worst.first) {
compactFile(worst.second);
}
diff --git a/searchlib/src/vespa/searchlib/docstore/logdatastore.h b/searchlib/src/vespa/searchlib/docstore/logdatastore.h
index eb46e5438a9..ab1c0026560 100644
--- a/searchlib/src/vespa/searchlib/docstore/logdatastore.h
+++ b/searchlib/src/vespa/searchlib/docstore/logdatastore.h
@@ -74,11 +74,10 @@ public:
size_t getNumThreads() const { return _numThreads; }
bool crcOnReadDisabled() const { return _skipCrcOnRead; }
- void disableCrcOnRead(bool v) { _skipCrcOnRead = v; }
bool compact2ActiveFile() const { return _compactToActiveFile; }
const CompressionConfig & compactCompression() const { return _compactCompression; }
-
const WriteableFileChunk::Config & getFileConfig() const { return _fileConfig; }
+ Config & disableCrcOnRead(bool v) { _skipCrcOnRead = v; return *this;}
private:
size_t _maxFileSize;
double _maxDiskBloatFactor;
@@ -201,18 +200,17 @@ public:
static NameIdSet findIncompleteCompactedFiles(const NameIdSet & partList);
NameIdSet getAllActiveFiles() const;
+ void reconfigure(const Config & config);
private:
class WrapVisitor;
class WrapVisitorProgress;
class FileChunkHolder;
- void waitForUnblock();
-
// Implements ISetLid API
void setLid(const LockGuard & guard, uint32_t lid, const LidInfo & lm) override;
- void compactWorst();
+ void compactWorst(double bloatLimit, double spreadLimit);
void compactFile(FileId chunkId);
typedef attribute::RcuVector<uint64_t> LidInfoVector;
@@ -257,9 +255,6 @@ private:
_active = fileId;
}
- bool isBucketSpreadTooLarge(double spread) const {
- return (spread >= _config.getMaxBucketSpread());
- }
double getMaxBucketSpread() const;
FileChunk::UP createReadOnlyFile(FileId fileId, NameId nameId);
@@ -273,13 +268,6 @@ private:
bool isReadOnly() const { return _readOnly; }
void updateSerialNum();
- bool isBloatOverLimit() const {
- return isBloatOverLimit(getDiskBloat(), getDiskFootprint());
- }
- bool isBloatOverLimit(uint64_t bloat, uint64_t usage) const {
- return (usage*_config.getMaxDiskBloatFactor() < bloat);
- }
-
size_t computeNumberOfSignificantBucketIdBits(const IBucketizer & bucketizer, FileId fileId) const;
/*
@@ -301,7 +289,7 @@ private:
return (_fileChunks.empty() ? 0 : _fileChunks.back()->getLastPersistedSerialNum());
}
bool shouldCompactToActiveFile(size_t compactedSize) const;
- std::pair<bool, FileId> findNextToCompact();
+ std::pair<bool, FileId> findNextToCompact(double bloatLimit, double spreadLimit);
void incGeneration();
bool canShrinkLidSpace(const vespalib::LockGuard &guard) const;
@@ -326,4 +314,3 @@ private:
};
} // namespace search
-
diff --git a/searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp b/searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp
index 313bc2f52d6..ad9c4f0aa4f 100644
--- a/searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp
@@ -25,5 +25,11 @@ LogDocumentStore::~LogDocumentStore()
{
}
+void
+LogDocumentStore::reconfigure(const Config & config) {
+ DocumentStore::reconfigure(config);
+ _backingStore.reconfigure(config.getLogConfig());
+}
+
} // namespace search
diff --git a/searchlib/src/vespa/searchlib/docstore/logdocumentstore.h b/searchlib/src/vespa/searchlib/docstore/logdocumentstore.h
index 3f0e2642529..16fb6036519 100644
--- a/searchlib/src/vespa/searchlib/docstore/logdocumentstore.h
+++ b/searchlib/src/vespa/searchlib/docstore/logdocumentstore.h
@@ -6,15 +6,9 @@
#include "logdatastore.h"
#include <vespa/searchlib/common/tunefileinfo.h>
-namespace search
-{
-
-namespace common
-{
-
-class FileHeaderContext;
+namespace search {
-}
+namespace common { class FileHeaderContext; }
/**
* Simple document store that contains serialized Document instances.
@@ -56,8 +50,7 @@ public:
transactionlog::SyncProxy &tlSyncer,
const IBucketizer::SP & bucketizer);
~LogDocumentStore();
- LogDataStore::Config & getLogConfig() { return _backingStore.getConfig(); }
- const LogDataStore::Config & getLogConfig() const { return _backingStore.getConfig(); }
+ void reconfigure(const Config & config);
private:
void compact(uint64_t syncToken) override { _backingStore.compact(syncToken); }
LogDataStore _backingStore;
diff --git a/searchlib/src/vespa/searchlib/docstore/visitcache.cpp b/searchlib/src/vespa/searchlib/docstore/visitcache.cpp
index b3fd236d73d..8f73c9862ae 100644
--- a/searchlib/src/vespa/searchlib/docstore/visitcache.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/visitcache.cpp
@@ -145,12 +145,25 @@ VisitCache::BackingStore::read(const KeySet &key, CompressedBlobSet &blobs) cons
return ! blobs.empty();
}
+void
+VisitCache::BackingStore::reconfigure(const CompressionConfig &compression) {
+ _compression = compression;
+}
+
+
VisitCache::VisitCache(IDataStore &store, size_t cacheSize, const CompressionConfig &compression) :
_store(store, compression),
_cache(std::make_unique<Cache>(_store, cacheSize))
{
}
+void
+VisitCache::reconfigure(size_t cacheSize, const CompressionConfig &compression) {
+ _store.reconfigure(compression);
+ _cache->setCapacityBytes(cacheSize);
+}
+
+
VisitCache::Cache::IdSet
VisitCache::Cache::findSetsContaining(const LockGuard &, const KeySet & keys) const {
IdSet found;
diff --git a/searchlib/src/vespa/searchlib/docstore/visitcache.h b/searchlib/src/vespa/searchlib/docstore/visitcache.h
index a89620b7bde..1bf867c5580 100644
--- a/searchlib/src/vespa/searchlib/docstore/visitcache.h
+++ b/searchlib/src/vespa/searchlib/docstore/visitcache.h
@@ -104,6 +104,7 @@ public:
void invalidate(uint32_t key) { remove(key); }
CacheStats getCacheStats() const;
+ void reconfigure(size_t cacheSize, const CompressionConfig &compression);
private:
/**
* This implments the interface the cache uses when it has a cache miss.
@@ -120,10 +121,11 @@ private:
bool read(const KeySet &key, CompressedBlobSet &blobs) const;
void write(const KeySet &, const CompressedBlobSet &) { }
void erase(const KeySet &) { }
- const CompressionConfig &getCompression() const { return _compression; }
+ void reconfigure(const CompressionConfig &compression);
+
private:
- IDataStore &_backingStore;
- const CompressionConfig _compression;
+ IDataStore &_backingStore;
+ CompressionConfig _compression;
};
using CacheParams = vespalib::CacheParam<
diff --git a/staging_vespalib/src/vespa/vespalib/stllike/cache.h b/staging_vespalib/src/vespa/vespalib/stllike/cache.h
index 832e6ede43d..3d5ab155877 100644
--- a/staging_vespalib/src/vespa/vespalib/stllike/cache.h
+++ b/staging_vespalib/src/vespa/vespalib/stllike/cache.h
@@ -71,6 +71,8 @@ public:
*/
cache & reserveElements(size_t elems);
+ cache & setCapacityBytes(size_t sz);
+
size_t capacity() const { return Lru::capacity(); }
size_t capacityBytes() const { return _maxBytes; }
size_t size() const { return Lru::size(); }
diff --git a/staging_vespalib/src/vespa/vespalib/stllike/cache.hpp b/staging_vespalib/src/vespa/vespalib/stllike/cache.hpp
index 06e7e249ec6..a8c7d16473c 100644
--- a/staging_vespalib/src/vespa/vespalib/stllike/cache.hpp
+++ b/staging_vespalib/src/vespa/vespalib/stllike/cache.hpp
@@ -21,6 +21,13 @@ cache<P>::reserveElements(size_t elems) {
}
template< typename P >
+cache<P> &
+cache<P>::setCapacityBytes(size_t sz) {
+ _maxBytes = sz;
+ return *this;
+}
+
+template< typename P >
void
cache<P>::invalidate(const K & key) {
vespalib::LockGuard guard(_hashLock);
diff --git a/staging_vespalib/src/vespa/vespalib/util/varholder.h b/staging_vespalib/src/vespa/vespalib/util/varholder.h
index 26f8f57839a..fdcc15d1fb4 100644
--- a/staging_vespalib/src/vespa/vespalib/util/varholder.h
+++ b/staging_vespalib/src/vespa/vespalib/util/varholder.h
@@ -2,41 +2,23 @@
#pragma once
-#include <vespa/vespalib/util/noncopyable.hpp>
#include <vespa/vespalib/util/sync.h>
-namespace vespalib
-{
-
+namespace vespalib {
template <typename T>
-class VarHolder : public noncopyable
+class VarHolder
{
-
- T _v;
- vespalib::Lock _lock;
-
+ T _v;
+ Lock _lock;
public:
- VarHolder(void)
- : _v(),
- _lock()
- {
- }
+ VarHolder() : _v(), _lock() {}
+ explicit VarHolder(const T &v) : _v(v), _lock() {}
+ VarHolder(const VarHolder &) = delete;
+ VarHolder & operator = (const VarHolder &) = delete;
+ ~VarHolder() {}
- explicit
- VarHolder(const T &v)
- : _v(v),
- _lock()
- {
- }
-
- ~VarHolder(void)
- {
- }
-
- void
- set(const T &v)
- {
+ void set(const T &v) {
T old;
{
vespalib::LockGuard guard(_lock);
@@ -45,19 +27,12 @@ public:
}
}
- void
- clear(void)
- {
- set(T());
- }
+ void clear() { set(T()); }
- T
- get(void) const
- {
+ T get() const {
vespalib::LockGuard guard(_lock);
return _v;
}
};
-} // namespace vespalib
-
+}
diff --git a/vespamalloc/src/tests/thread/thread_test.sh b/vespamalloc/src/tests/thread/thread_test.sh
index b06c27e4a17..b2e91840711 100755
--- a/vespamalloc/src/tests/thread/thread_test.sh
+++ b/vespamalloc/src/tests/thread/thread_test.sh
@@ -17,8 +17,8 @@ fi
VESPA_MALLOC_SO=../../../src/vespamalloc/libvespamalloc.so
VESPA_MALLOC_SO_D=../../../src/vespamalloc/libvespamallocd.so
-LD_PRELOAD=$VESPA_MALLOC_SO ./vespamalloc_thread_test_app return 20
-LD_PRELOAD=$VESPA_MALLOC_SO ./vespamalloc_thread_test_app exit 20
-LD_PRELOAD=$VESPA_MALLOC_SO ./vespamalloc_thread_test_app cancel 20
+#LD_PRELOAD=$VESPA_MALLOC_SO ./vespamalloc_thread_test_app return 20
+#LD_PRELOAD=$VESPA_MALLOC_SO ./vespamalloc_thread_test_app exit 20
+#LD_PRELOAD=$VESPA_MALLOC_SO ./vespamalloc_thread_test_app cancel 20
#LD_PRELOAD=$VESPA_MALLOC_SO ./vespamalloc_racemanythreads_test_app 4000 20
#LD_PRELOAD=$VESPA_MALLOC_SO_D ./vespamalloc_racemanythreads_test_app 4000 20