summaryrefslogtreecommitdiffstats
diff options
context:
space:
mode:
authorTor Egge <Tor.Egge@yahoo-inc.com>2016-07-01 12:26:16 +0000
committerTor Egge <Tor.Egge@yahoo-inc.com>2016-09-21 10:32:30 +0000
commitd52f2543aa0aa8c26b9719f8032174699e0c73e3 (patch)
tree34d80d3699505edca26072efe2e52372f79fc62d
parentd53d0502b623b290789eb06077cbd8421ef6a009 (diff)
Prepare for exposing more stats about disk and memory indexes, for
metrics and state explorer. Report serial number for disk and memory indexes.
-rw-r--r--searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.cpp23
-rw-r--r--searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.h5
-rw-r--r--searchcore/src/vespa/searchcore/proton/index/indexmanager.cpp6
-rw-r--r--searchcore/src/vespa/searchcore/proton/index/indexmanager.h3
-rw-r--r--searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.cpp17
-rw-r--r--searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.h13
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/fakeindexsearchable.h6
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/iindexmaintaineroperations.h2
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/imemoryindex.h2
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexcollection.cpp20
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexcollection.h2
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp36
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h3
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexsearchable.h14
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/indexsearchablevisitor.h22
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.cpp16
-rw-r--r--searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.h2
-rw-r--r--searchlib/src/vespa/searchlib/diskindex/diskindex.cpp10
-rw-r--r--searchlib/src/vespa/searchlib/diskindex/diskindex.h6
19 files changed, 177 insertions, 31 deletions
diff --git a/searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.cpp b/searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.cpp
index f7a16bdcc81..ad78835fa37 100644
--- a/searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.cpp
+++ b/searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.cpp
@@ -4,29 +4,48 @@
LOG_SETUP(".proton.index.diskindexwrapper");
#include "diskindexwrapper.h"
+#include <vespa/searchcorespi/index/indexreadutilities.h>
+#include <vespa/searchcorespi/index/indexsearchablevisitor.h>
using search::TuneFileSearch;
+using searchcorespi::index::IndexReadUtilities;
namespace proton {
DiskIndexWrapper::DiskIndexWrapper(const vespalib::string &indexDir,
const TuneFileSearch &tuneFileSearch,
size_t cacheSize)
- : _index(indexDir, cacheSize)
+ : _index(indexDir, cacheSize),
+ _serialNum(0)
{
bool setupIndexOk = _index.setup(tuneFileSearch);
assert(setupIndexOk);
(void) setupIndexOk;
+ _serialNum = IndexReadUtilities::readSerialNum(indexDir);
}
DiskIndexWrapper::DiskIndexWrapper(const DiskIndexWrapper &oldIndex,
const TuneFileSearch &tuneFileSearch,
size_t cacheSize)
- : _index(oldIndex._index.getIndexDir(), cacheSize)
+ : _index(oldIndex._index.getIndexDir(), cacheSize),
+ _serialNum(0)
{
bool setupIndexOk = _index.setup(tuneFileSearch, oldIndex._index);
assert(setupIndexOk);
(void) setupIndexOk;
+ _serialNum = oldIndex.getSerialNum();
+}
+
+search::SerialNum
+DiskIndexWrapper::getSerialNum() const
+{
+ return _serialNum;
+}
+
+void
+DiskIndexWrapper::accept(searchcorespi::IndexSearchableVisitor &visitor) const
+{
+ visitor.visit(*this);
}
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.h b/searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.h
index 9deef043f6e..20ba61ea13a 100644
--- a/searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.h
+++ b/searchcore/src/vespa/searchcore/proton/index/diskindexwrapper.h
@@ -10,6 +10,7 @@ namespace proton {
class DiskIndexWrapper : public searchcorespi::index::IDiskIndex {
private:
search::diskindex::DiskIndex _index;
+ search::SerialNum _serialNum;
public:
DiskIndexWrapper(const vespalib::string &indexDir,
@@ -44,6 +45,10 @@ public:
.sizeOnDisk(_index.getSize());
}
+ virtual search::SerialNum getSerialNum() const override;
+
+ virtual void accept(searchcorespi::IndexSearchableVisitor &visitor) const override;
+
/**
* Implements proton::IDiskIndex
*/
diff --git a/searchcore/src/vespa/searchcore/proton/index/indexmanager.cpp b/searchcore/src/vespa/searchcore/proton/index/indexmanager.cpp
index ad2e3f29853..7e9aa2436b1 100644
--- a/searchcore/src/vespa/searchcore/proton/index/indexmanager.cpp
+++ b/searchcore/src/vespa/searchcore/proton/index/indexmanager.cpp
@@ -42,12 +42,14 @@ IndexManager::MaintainerOperations::MaintainerOperations(const FileHeaderContext
}
IMemoryIndex::SP
-IndexManager::MaintainerOperations::createMemoryIndex(const Schema &schema)
+IndexManager::MaintainerOperations::createMemoryIndex(const Schema &schema,
+ SerialNum serialNum)
{
return IMemoryIndex::SP(new MemoryIndexWrapper(schema,
_fileHeaderContext,
_tuneFileIndexing,
- _threadingService));
+ _threadingService,
+ serialNum));
}
IDiskIndex::SP
diff --git a/searchcore/src/vespa/searchcore/proton/index/indexmanager.h b/searchcore/src/vespa/searchcore/proton/index/indexmanager.h
index 8a7d6a64b30..f81c949d8b3 100644
--- a/searchcore/src/vespa/searchcore/proton/index/indexmanager.h
+++ b/searchcore/src/vespa/searchcore/proton/index/indexmanager.h
@@ -36,7 +36,8 @@ public:
threadingService);
virtual searchcorespi::index::IMemoryIndex::SP
- createMemoryIndex(const search::index::Schema &schema);
+ createMemoryIndex(const search::index::Schema &schema,
+ SerialNum serialNum) override;
virtual searchcorespi::index::IDiskIndex::SP
loadDiskIndex(const vespalib::string &indexDir);
virtual searchcorespi::index::IDiskIndex::SP
diff --git a/searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.cpp b/searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.cpp
index c1535082407..73a86c805e2 100644
--- a/searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.cpp
+++ b/searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.cpp
@@ -7,6 +7,7 @@ LOG_SETUP(".proton.index.memoryindexwrapper");
#include <vespa/searchlib/common/serialnumfileheadercontext.h>
#include <vespa/searchlib/diskindex/indexbuilder.h>
#include <vespa/vespalib/util/exceptions.h>
+#include <vespa/searchcorespi/index/indexsearchablevisitor.h>
using search::TuneFileIndexing;
using search::common::FileHeaderContext;
@@ -22,9 +23,11 @@ MemoryIndexWrapper::MemoryIndexWrapper(const search::index::Schema &schema,
const search::common::FileHeaderContext &fileHeaderContext,
const TuneFileIndexing &tuneFileIndexing,
searchcorespi::index::IThreadingService &
- threadingService)
+ threadingService,
+ search::SerialNum serialNum)
: _index(schema, threadingService.indexFieldInverter(),
threadingService.indexFieldWriter()),
+ _serialNum(serialNum),
_fileHeaderContext(fileHeaderContext),
_tuneFileIndexing(tuneFileIndexing)
{
@@ -46,5 +49,17 @@ MemoryIndexWrapper::flushToDisk(const vespalib::string &flushDir,
indexBuilder.close();
}
+search::SerialNum
+MemoryIndexWrapper::getSerialNum() const
+{
+ return _serialNum.load(std::memory_order_relaxed);
+}
+
+void
+MemoryIndexWrapper::accept(searchcorespi::IndexSearchableVisitor &visitor) const
+{
+ visitor.visit(*this);
+}
+
} // namespace proton
diff --git a/searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.h b/searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.h
index 5a058326db2..a4ef714d09d 100644
--- a/searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.h
+++ b/searchcore/src/vespa/searchcore/proton/index/memoryindexwrapper.h
@@ -7,6 +7,7 @@
#include <vespa/searchcorespi/index/ithreadingservice.h>
#include <vespa/searchlib/common/tunefileinfo.h>
#include <vespa/searchlib/common/fileheadercontext.h>
+#include <atomic>
namespace proton {
@@ -17,6 +18,7 @@ namespace proton {
class MemoryIndexWrapper : public searchcorespi::index::IMemoryIndex {
private:
search::memoryindex::MemoryIndex _index;
+ std::atomic<search::SerialNum> _serialNum;
const search::common::FileHeaderContext &_fileHeaderContext;
const search::TuneFileIndexing _tuneFileIndexing;
@@ -25,7 +27,8 @@ public:
const search::common::FileHeaderContext &fileHeaderContext,
const search::TuneFileIndexing &tuneFileIndexing,
searchcorespi::index::IThreadingService &
- threadingService);
+ threadingService,
+ search::SerialNum serialNum);
/**
* Implements searchcorespi::IndexSearchable
@@ -53,6 +56,10 @@ public:
.sizeOnDisk(0);
}
+ virtual search::SerialNum getSerialNum() const override;
+
+ virtual void accept(searchcorespi::IndexSearchableVisitor &visitor) const override;
+
/**
* Implements proton::IMemoryIndex
*/
@@ -74,8 +81,10 @@ public:
uint64_t getStaticMemoryFootprint() const override {
return _index.getStaticMemoryFootprint();
}
- virtual void commit(OnWriteDoneType onWriteDone) override {
+ virtual void commit(OnWriteDoneType onWriteDone,
+ search::SerialNum serialNum) override {
_index.commit(onWriteDone);
+ _serialNum.store(serialNum, std::memory_order_relaxed);
}
virtual void wipeHistory(const search::index::Schema &schema) override{
_index.wipeHistory(schema);
diff --git a/searchcorespi/src/vespa/searchcorespi/index/fakeindexsearchable.h b/searchcorespi/src/vespa/searchcorespi/index/fakeindexsearchable.h
index 0db49c5f2df..00ede178339 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/fakeindexsearchable.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/fakeindexsearchable.h
@@ -37,6 +37,12 @@ public:
virtual search::SearchableStats getSearchableStats() const {
return search::SearchableStats();
}
+
+ virtual search::SerialNum getSerialNum() const override { return 0; }
+ virtual void accept(IndexSearchableVisitor &visitor) const override {
+ (void) visitor;
+ }
+
};
} // namespace searchcorespi
diff --git a/searchcorespi/src/vespa/searchcorespi/index/iindexmaintaineroperations.h b/searchcorespi/src/vespa/searchcorespi/index/iindexmaintaineroperations.h
index 2b8a3ac3965..eb5edbf3298 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/iindexmaintaineroperations.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/iindexmaintaineroperations.h
@@ -19,7 +19,7 @@ struct IIndexMaintainerOperations {
/**
* Creates a new memory index using the given schema.
*/
- virtual IMemoryIndex::SP createMemoryIndex(const search::index::Schema &schema) = 0;
+ virtual IMemoryIndex::SP createMemoryIndex(const search::index::Schema &schema, search::SerialNum serialNum) = 0;
/**
* Loads a disk index from the given directory.
diff --git a/searchcorespi/src/vespa/searchcorespi/index/imemoryindex.h b/searchcorespi/src/vespa/searchcorespi/index/imemoryindex.h
index fabaf730cb7..c8baaf9ca13 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/imemoryindex.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/imemoryindex.h
@@ -61,7 +61,7 @@ struct IMemoryIndex : public searchcorespi::IndexSearchable {
/**
* Commits the inserts and removes since the last commit, making them searchable.
**/
- virtual void commit(OnWriteDoneType onWriteDone) = 0;
+ virtual void commit(OnWriteDoneType onWriteDone, search::SerialNum serialNum) = 0;
/**
* Flushes this memory index to disk as a disk index.
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexcollection.cpp b/searchcorespi/src/vespa/searchcorespi/index/indexcollection.cpp
index 0284c4a682e..653cf0a28b4 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/indexcollection.cpp
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexcollection.cpp
@@ -9,6 +9,7 @@ LOG_SETUP(".searchcorespi.index.indexcollection");
#include <vespa/searchlib/queryeval/create_blueprint_visitor_helper.h>
#include <vespa/searchlib/queryeval/intermediate_blueprints.h>
#include <vespa/searchlib/queryeval/leaf_blueprints.h>
+#include "indexsearchablevisitor.h"
using namespace search::queryeval;
using namespace search::query;
@@ -116,6 +117,25 @@ IndexCollection::getSearchableStats() const
return stats;
}
+search::SerialNum
+IndexCollection::getSerialNum() const
+{
+ search::SerialNum serialNum = 0;
+ for (auto &source : _sources) {
+ serialNum = std::max(serialNum, source.source_wrapper->getSerialNum());
+ }
+ return serialNum;
+}
+
+
+void
+IndexCollection::accept(IndexSearchableVisitor &visitor) const
+{
+ for (auto &source : _sources) {
+ source.source_wrapper->accept(visitor);
+ }
+}
+
namespace {
struct Mixer {
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexcollection.h b/searchcorespi/src/vespa/searchcorespi/index/indexcollection.h
index 223b36fce99..38942abc9dc 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/indexcollection.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexcollection.h
@@ -61,6 +61,8 @@ public:
const Node &term,
const IAttributeContext &attrCtx);
virtual search::SearchableStats getSearchableStats() const;
+ virtual search::SerialNum getSerialNum() const override;
+ virtual void accept(IndexSearchableVisitor &visitor) const override;
static ISearchableIndexCollection::UP replaceAndRenumber(
const ISourceSelector::SP & selector,
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
index 36d42e245e1..3db3875b561 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.cpp
@@ -117,6 +117,12 @@ public:
return _index->createBlueprint(requestContext, fields, term, attrCtx);
}
virtual search::SearchableStats getSearchableStats() const { return _index->getSearchableStats(); }
+ virtual search::SerialNum getSerialNum() const override {
+ return _index->getSerialNum();
+ }
+ virtual void accept(IndexSearchableVisitor &visitor) const override {
+ _index->accept(visitor);
+ }
/**
* Implements IDiskIndex
@@ -293,7 +299,8 @@ IDiskIndex::SP
IndexMaintainer::flushMemoryIndex(IMemoryIndex &memoryIndex,
uint32_t indexId,
uint32_t docIdLimit,
- SerialNum serialNum)
+ SerialNum serialNum,
+ FixedSourceSelector::SaveInfo &saveInfo)
{
// Called by a flush worker thread
const string flushDir = getFlushDir(indexId);
@@ -302,6 +309,11 @@ IndexMaintainer::flushMemoryIndex(IMemoryIndex &memoryIndex,
if (wtSchema.get() != NULL) {
updateDiskIndexSchema(flushDir, *wtSchema, noSerialNumHigh);
}
+ IndexWriteUtilities::writeSourceSelector(saveInfo, indexId, getAttrTune(),
+ _ctx.getFileHeaderContext(),
+ serialNum);
+ IndexWriteUtilities::writeSerialNum(serialNum, flushDir,
+ _ctx.getFileHeaderContext());
return loadDiskIndex(flushDir);
}
@@ -521,14 +533,8 @@ IndexMaintainer::flushMemoryIndex(FlushArgs &args,
IMemoryIndex &memoryIndex = *args.old_index;
Schema::SP wtSchema = memoryIndex.getWipeTimeSchema();
IDiskIndex::SP diskIndex = flushMemoryIndex(memoryIndex, args.old_absolute_id,
- docIdLimit, args.flush_serial_num);
- IndexWriteUtilities::writeSourceSelector(saveInfo, args.old_absolute_id,
- getAttrTune(), _ctx.getFileHeaderContext(),
- args.flush_serial_num);
- IndexWriteUtilities::writeSerialNum(args.flush_serial_num,
- getFlushDir(args.old_absolute_id),
- _ctx.getFileHeaderContext());
-
+ docIdLimit, args.flush_serial_num,
+ saveInfo);
// Post processing after memory index has been written to disk and
// opened as disk index.
args._changeGens = changeGens;
@@ -799,7 +805,7 @@ IndexMaintainer::IndexMaintainer(const IndexMaintainerConfig &config,
_last_fusion_id(),
_next_id(),
_current_index_id(),
- _current_index(operations.createMemoryIndex(_schema)),
+ _current_index(),
_current_serial_num(0),
_flush_serial_num(0),
_lastFlushTime(),
@@ -848,6 +854,7 @@ IndexMaintainer::IndexMaintainer(const IndexMaintainerConfig &config,
_selector.reset(getSourceSelector().cloneAndSubtract(ost.str(), id_diff).release());
assert(_last_fusion_id == _selector->getBaseId());
}
+ _current_index = operations.createMemoryIndex(_schema, _current_serial_num);
_current_index_id = getNewAbsoluteId() - _last_fusion_id;
assert(_current_index_id < ISourceSelector::SOURCE_LIMIT);
ISearchableIndexCollection::UP sourceList(loadDiskIndexes(spec, ISearchableIndexCollection::UP(new IndexCollection(_selector))));
@@ -874,7 +881,7 @@ IndexMaintainer::initFlush(SerialNum serialNum, searchcorespi::FlushStats * stat
_current_serial_num = std::max(_current_serial_num, serialNum);
}
- IMemoryIndex::SP new_index(_operations.createMemoryIndex(getSchema()));
+ IMemoryIndex::SP new_index(_operations.createMemoryIndex(getSchema(), _current_serial_num));
FlushArgs args;
args.stats = stats;
scheduleCommit();
@@ -1132,7 +1139,8 @@ IndexMaintainer::commit()
// only triggered via scheduleCommit()
assert(_ctx.getThreadingService().index().isCurrentThread());
LockGuard lock(_index_update_lock);
- _current_index->commit(std::shared_ptr<search::IDestructorCallback>());
+ _current_index->commit(std::shared_ptr<search::IDestructorCallback>(),
+ _current_serial_num);
// caller calls _ctx.getThreadingService().sync()
}
@@ -1142,7 +1150,7 @@ IndexMaintainer::commit(SerialNum serialNum, OnWriteDoneType onWriteDone)
assert(_ctx.getThreadingService().index().isCurrentThread());
LockGuard lock(_index_update_lock);
_current_serial_num = serialNum;
- _current_index->commit(onWriteDone);
+ _current_index->commit(onWriteDone, serialNum);
}
void
@@ -1169,7 +1177,7 @@ void
IndexMaintainer::setSchema(const Schema & schema, const Schema & fusionSchema)
{
assert(_ctx.getThreadingService().master().isCurrentThread());
- IMemoryIndex::SP new_index(_operations.createMemoryIndex(schema));
+ IMemoryIndex::SP new_index(_operations.createMemoryIndex(schema, _current_serial_num));
SetSchemaArgs args;
args._newSchema = schema;
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h
index 417f79fa6ca..39e3d116d44 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexmaintainer.h
@@ -185,7 +185,8 @@ class IndexMaintainer : public IIndexManager,
flushMemoryIndex(IMemoryIndex &memoryIndex,
uint32_t indexId,
uint32_t docIdLimit,
- SerialNum serialNum);
+ SerialNum serialNum,
+ search::FixedSourceSelector::SaveInfo &saveInfo);
ISearchableIndexCollection::UP loadDiskIndexes(const FusionSpec &spec, ISearchableIndexCollection::UP sourceList);
void replaceSource(uint32_t sourceId, const IndexSearchable::SP &source);
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexsearchable.h b/searchcorespi/src/vespa/searchcorespi/index/indexsearchable.h
index 2d4d7cd9674..89a7c8d9f07 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/indexsearchable.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexsearchable.h
@@ -8,9 +8,12 @@
#include <vespa/searchlib/queryeval/blueprint.h>
#include <vespa/searchlib/queryeval/irequestcontext.h>
#include <vespa/searchlib/util/searchable_stats.h>
+#include <vespa/searchlib/common/serialnum.h>
namespace searchcorespi {
+class IndexSearchableVisitor;
+
/**
* Abstract class extended by components to expose content that can be
* searched by a query term. A IndexSearchable component supports searching
@@ -72,6 +75,17 @@ public:
* Returns the searchable stats for this index searchable.
*/
virtual search::SearchableStats getSearchableStats() const = 0;
+
+ /*
+ * Returns the serial number for this index searchable.
+ */
+ virtual search::SerialNum getSerialNum() const = 0;
+
+ /*
+ * Calls visitor with properly downcasted argument to differentiate
+ * between different types of indexes (disk index or memory index).
+ */
+ virtual void accept(IndexSearchableVisitor &visitor) const = 0;
};
} // namespace searchcorespi
diff --git a/searchcorespi/src/vespa/searchcorespi/index/indexsearchablevisitor.h b/searchcorespi/src/vespa/searchcorespi/index/indexsearchablevisitor.h
new file mode 100644
index 00000000000..91232ec5c51
--- /dev/null
+++ b/searchcorespi/src/vespa/searchcorespi/index/indexsearchablevisitor.h
@@ -0,0 +1,22 @@
+// Copyright 2016 Yahoo Inc. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+
+#pragma once
+
+namespace searchcorespi {
+
+namespace index {
+
+struct IDiskIndex;
+struct IMemoryIndex;
+
+}
+
+class IndexSearchableVisitor
+{
+public:
+ virtual ~IndexSearchableVisitor() { }
+ virtual void visit(const index::IDiskIndex &index) = 0;
+ virtual void visit(const index::IMemoryIndex &index) = 0;
+};
+
+} // namespace searchcorespi
diff --git a/searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.cpp b/searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.cpp
index c993c79a93e..53a2a4d59ea 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.cpp
+++ b/searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.cpp
@@ -184,6 +184,22 @@ WarmupIndexCollection::getSearchableStats() const
return _prev->getSearchableStats();
}
+
+search::SerialNum
+WarmupIndexCollection::getSerialNum() const
+{
+ return std::max(_prev->getSerialNum(), _next->getSerialNum());
+}
+
+
+void
+WarmupIndexCollection::accept(IndexSearchableVisitor &visitor) const
+{
+ _prev->accept(visitor);
+ _next->accept(visitor);
+}
+
+
void
WarmupIndexCollection::append(uint32_t id, const IndexSearchable::SP &source)
{
diff --git a/searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.h b/searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.h
index a304b3b9df2..be25b81d498 100644
--- a/searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.h
+++ b/searchcorespi/src/vespa/searchcorespi/index/warmupindexcollection.h
@@ -49,6 +49,8 @@ public:
const Node &term,
const IAttributeContext &attrCtx);
search::SearchableStats getSearchableStats() const override;
+ search::SerialNum getSerialNum() const override;
+ void accept(IndexSearchableVisitor &visitor) const override;
// Implements ISearchableIndexCollection
void append(uint32_t id, const IndexSearchable::SP &source) override;
diff --git a/searchlib/src/vespa/searchlib/diskindex/diskindex.cpp b/searchlib/src/vespa/searchlib/diskindex/diskindex.cpp
index 8cc12c88463..6d693bb6f42 100644
--- a/searchlib/src/vespa/searchlib/diskindex/diskindex.cpp
+++ b/searchlib/src/vespa/searchlib/diskindex/diskindex.cpp
@@ -47,8 +47,10 @@ DiskIndex::DiskIndex(const vespalib::string &indexDir, size_t cacheSize)
_bitVectorDicts(),
_dicts(),
_tuneFileSearch(),
- _cache(*this, cacheSize)
+ _cache(*this, cacheSize),
+ _size(0)
{
+ calculateSize();
}
bool
@@ -322,11 +324,11 @@ DiskIndex::readBitVector(const LookupResult &lookupRes) const
}
-uint64_t
-DiskIndex::getSize() const
+void
+DiskIndex::calculateSize()
{
search::DirectoryTraverse dirt(_indexDir.c_str());
- return dirt.GetTreeSize();
+ _size = dirt.GetTreeSize();
}
diff --git a/searchlib/src/vespa/searchlib/diskindex/diskindex.h b/searchlib/src/vespa/searchlib/diskindex/diskindex.h
index 840f4c32738..3a40bb92f8b 100644
--- a/searchlib/src/vespa/searchlib/diskindex/diskindex.h
+++ b/searchlib/src/vespa/searchlib/diskindex/diskindex.h
@@ -78,6 +78,9 @@ private:
std::vector<std::unique_ptr<index::DictionaryFileRandRead>> _dicts;
TuneFileSearch _tuneFileSearch;
Cache _cache;
+ uint64_t _size;
+
+ void calculateSize();
bool
loadSchema(void);
@@ -159,8 +162,7 @@ public:
* Get the size on disk of this index.
* @return the size of the index.
*/
- uint64_t
- getSize() const;
+ uint64_t getSize() const { return _size; }
const index::Schema &
getSchema(void) const