aboutsummaryrefslogtreecommitdiffstats
path: root/searchcore/src/vespa/searchcorespi/index/indexmaintainer.cpp
diff options
context:
space:
mode:
Diffstat (limited to 'searchcore/src/vespa/searchcorespi/index/indexmaintainer.cpp')
-rw-r--r--searchcore/src/vespa/searchcorespi/index/indexmaintainer.cpp238
1 files changed, 147 insertions, 91 deletions
diff --git a/searchcore/src/vespa/searchcorespi/index/indexmaintainer.cpp b/searchcore/src/vespa/searchcorespi/index/indexmaintainer.cpp
index f6b05f639ed..bbd17be9b5a 100644
--- a/searchcore/src/vespa/searchcorespi/index/indexmaintainer.cpp
+++ b/searchcore/src/vespa/searchcorespi/index/indexmaintainer.cpp
@@ -1,11 +1,14 @@
-// Copyright Yahoo. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
+// Copyright Vespa.ai. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
#include "indexmaintainer.h"
+#include "disk_indexes.h"
#include "diskindexcleaner.h"
#include "eventlogger.h"
#include "fusionrunner.h"
+#include "indexcollection.h"
#include "indexflushtarget.h"
#include "indexfusiontarget.h"
+#include "indexmaintainerconfig.h"
#include "indexreadutilities.h"
#include "indexwriteutilities.h"
#include "index_disk_dir.h"
@@ -93,13 +96,13 @@ SerialNum noSerialNumHigh = std::numeric_limits<SerialNum>::max();
class DiskIndexWithDestructorCallback : public IDiskIndex {
private:
std::shared_ptr<IDestructorCallback> _callback;
- IDiskIndex::SP _index;
+ std::shared_ptr<IDiskIndex> _index;
IndexDiskDir _index_disk_dir;
IndexDiskLayout& _layout;
DiskIndexes& _disk_indexes;
public:
- DiskIndexWithDestructorCallback(IDiskIndex::SP index,
+ DiskIndexWithDestructorCallback(std::shared_ptr<IDiskIndex> index,
std::shared_ptr<IDestructorCallback> callback,
IndexDiskLayout& layout,
DiskIndexes& disk_indexes) noexcept
@@ -116,7 +119,7 @@ public:
/**
* Implements searchcorespi::IndexSearchable
*/
- Blueprint::UP
+ std::unique_ptr<Blueprint>
createBlueprint(const IRequestContext & requestContext,
const FieldSpec &field,
const Node &term) override
@@ -125,7 +128,7 @@ public:
fsl.add(field);
return _index->createBlueprint(requestContext, fsl, term);
}
- Blueprint::UP
+ std::unique_ptr<Blueprint>
createBlueprint(const IRequestContext & requestContext,
const FieldSpecList &fields,
const Node &term) override
@@ -184,10 +187,11 @@ IndexMaintainer::FusionArgs::~FusionArgs() = default;
IndexMaintainer::SetSchemaArgs::SetSchemaArgs() = default;
IndexMaintainer::SetSchemaArgs::~SetSchemaArgs() = default;
-uint32_t
-IndexMaintainer::getNewAbsoluteId()
+void
+IndexMaintainer::set_id_for_new_memory_index()
{
- return _next_id++;
+ _current_index_id = _next_id++ - _last_fusion_id;
+ assert(_current_index_id < ISourceSelector::SOURCE_LIMIT);
}
string
@@ -221,7 +225,7 @@ IndexMaintainer::reopenDiskIndexes(ISearchableIndexCollection &coll)
LOG(error, "Could not open schema '%s'", schemaName.c_str());
}
if (trimmedSchema != d->getSchema()) {
- IDiskIndex::SP newIndex(reloadDiskIndex(*d));
+ auto newIndex(reloadDiskIndex(*d));
coll.replace(coll.getSourceId(i), newIndex);
hasReopenedAnything = true;
}
@@ -265,9 +269,9 @@ IndexMaintainer::updateActiveFusionPrunedSchema(const Schema &schema)
{
assert(_ctx.getThreadingService().master().isCurrentThread());
for (;;) {
- Schema::SP activeFusionSchema;
- Schema::SP activeFusionPrunedSchema;
- Schema::SP newActiveFusionPrunedSchema;
+ std::shared_ptr<Schema> activeFusionSchema;
+ std::shared_ptr<Schema> activeFusionPrunedSchema;
+ std::shared_ptr<Schema> newActiveFusionPrunedSchema;
{
LockGuard lock(_state_lock);
activeFusionSchema = _activeFusionSchema;
@@ -276,10 +280,10 @@ IndexMaintainer::updateActiveFusionPrunedSchema(const Schema &schema)
if (!activeFusionSchema)
return; // No active fusion
if (!activeFusionPrunedSchema) {
- Schema::UP newSchema = Schema::intersect(*activeFusionSchema, schema);
+ auto newSchema = Schema::intersect(*activeFusionSchema, schema);
newActiveFusionPrunedSchema = std::move(newSchema);
} else {
- Schema::UP newSchema = Schema::intersect(*activeFusionPrunedSchema, schema);
+ auto newSchema = Schema::intersect(*activeFusionPrunedSchema, schema);
newActiveFusionPrunedSchema = std::move(newSchema);
}
{
@@ -302,7 +306,7 @@ IndexMaintainer::deactivateDiskIndexes(vespalib::string indexDir)
removeOldDiskIndexes();
}
-IDiskIndex::SP
+std::shared_ptr<IDiskIndex>
IndexMaintainer::loadDiskIndex(const string &indexDir)
{
// Called by a flush worker thread OR CTOR (in document db init executor thread)
@@ -323,7 +327,7 @@ IndexMaintainer::loadDiskIndex(const string &indexDir)
return retval;
}
-IDiskIndex::SP
+std::shared_ptr<IDiskIndex>
IndexMaintainer::reloadDiskIndex(const IDiskIndex &oldIndex)
{
// Called by a flush worker thread OR document db executor thread
@@ -346,7 +350,7 @@ IndexMaintainer::reloadDiskIndex(const IDiskIndex &oldIndex)
return retval;
}
-IDiskIndex::SP
+std::shared_ptr<IDiskIndex>
IndexMaintainer::flushMemoryIndex(IMemoryIndex &memoryIndex,
uint32_t indexId,
uint32_t docIdLimit,
@@ -356,7 +360,7 @@ IndexMaintainer::flushMemoryIndex(IMemoryIndex &memoryIndex,
// Called by a flush worker thread
const string flushDir = getFlushDir(indexId);
memoryIndex.flushToDisk(flushDir, docIdLimit, serialNum);
- Schema::SP prunedSchema(memoryIndex.getPrunedSchema());
+ auto prunedSchema(memoryIndex.getPrunedSchema());
if (prunedSchema) {
updateDiskIndexSchema(flushDir, *prunedSchema, noSerialNumHigh);
}
@@ -366,8 +370,8 @@ IndexMaintainer::flushMemoryIndex(IMemoryIndex &memoryIndex,
return loadDiskIndex(flushDir);
}
-ISearchableIndexCollection::UP
-IndexMaintainer::loadDiskIndexes(const FusionSpec &spec, ISearchableIndexCollection::UP sourceList)
+std::unique_ptr<ISearchableIndexCollection>
+IndexMaintainer::loadDiskIndexes(const FusionSpec &spec, std::unique_ptr<ISearchableIndexCollection> sourceList)
{
// Called by CTOR (in document db init executor thread)
uint32_t fusion_id = spec.last_fusion_id;
@@ -386,8 +390,8 @@ namespace {
using LockGuard = std::lock_guard<std::mutex>;
-ISearchableIndexCollection::SP
-getLeaf(const LockGuard &newSearchLock, const ISearchableIndexCollection::SP & is, bool warn=false)
+std::shared_ptr<ISearchableIndexCollection>
+getLeaf(const LockGuard &newSearchLock, const std::shared_ptr<ISearchableIndexCollection>& is, bool warn=false)
{
if (dynamic_cast<const WarmupIndexCollection *>(is.get()) != nullptr) {
if (warn) {
@@ -408,11 +412,11 @@ getLeaf(const LockGuard &newSearchLock, const ISearchableIndexCollection::SP & i
* Caller must hold _state_lock (SL).
*/
void
-IndexMaintainer::replaceSource(uint32_t sourceId, const IndexSearchable::SP &source)
+IndexMaintainer::replaceSource(uint32_t sourceId, const std::shared_ptr<IndexSearchable>& source)
{
assert(_ctx.getThreadingService().master().isCurrentThread());
LockGuard lock(_new_search_lock);
- ISearchableIndexCollection::UP indexes = createNewSourceCollection(lock);
+ auto indexes = createNewSourceCollection(lock);
indexes->replace(sourceId, source);
swapInNewIndex(lock, std::move(indexes), *source);
}
@@ -423,7 +427,7 @@ IndexMaintainer::replaceSource(uint32_t sourceId, const IndexSearchable::SP &sou
*/
void
IndexMaintainer::swapInNewIndex(LockGuard & guard,
- ISearchableIndexCollection::SP indexes,
+ std::shared_ptr<ISearchableIndexCollection> indexes,
IndexSearchable & source)
{
assert(indexes->valid());
@@ -448,19 +452,19 @@ IndexMaintainer::swapInNewIndex(LockGuard & guard,
* Caller must hold _state_lock (SL).
*/
void
-IndexMaintainer::appendSource(uint32_t sourceId, const IndexSearchable::SP &source)
+IndexMaintainer::appendSource(uint32_t sourceId, const std::shared_ptr<IndexSearchable>& source)
{
assert(_ctx.getThreadingService().master().isCurrentThread());
LockGuard lock(_new_search_lock);
- ISearchableIndexCollection::UP indexes = createNewSourceCollection(lock);
+ auto indexes = createNewSourceCollection(lock);
indexes->append(sourceId, source);
swapInNewIndex(lock, std::move(indexes), *source);
}
-ISearchableIndexCollection::UP
+std::unique_ptr<ISearchableIndexCollection>
IndexMaintainer::createNewSourceCollection(const LockGuard &newSearchLock)
{
- ISearchableIndexCollection::SP currentLeaf(getLeaf(newSearchLock, _source_list));
+ auto currentLeaf(getLeaf(newSearchLock, _source_list));
return std::make_unique<IndexCollection>(_selector, *currentLeaf);
}
@@ -482,13 +486,13 @@ IndexMaintainer::FlushArgs::FlushArgs(FlushArgs &&) = default;
IndexMaintainer::FlushArgs & IndexMaintainer::FlushArgs::operator=(FlushArgs &&) = default;
bool
-IndexMaintainer::doneInitFlush(FlushArgs *args, IMemoryIndex::SP *new_index)
+IndexMaintainer::doneInitFlush(FlushArgs *args, std::shared_ptr<IMemoryIndex>* new_index)
{
// Called by initFlush via reconfigurer
assert(_ctx.getThreadingService().master().isCurrentThread());
LockGuard state_lock(_state_lock);
args->old_index = _current_index;
- args->old_absolute_id = _current_index_id + _last_fusion_id;
+ args->old_absolute_id = get_absolute_id();
args->old_source_list = _source_list;
string selector_name = IndexDiskLayout::getSelectorFileName(getFlushDir(args->old_absolute_id));
args->flush_serial_num = current_serial_num();
@@ -513,9 +517,7 @@ IndexMaintainer::doneInitFlush(FlushArgs *args, IMemoryIndex::SP *new_index)
if (!args->_skippedEmptyLast) {
// Keep on using same source selector with extended valid range
args->save_info = getSourceSelector().extractSaveInfo(selector_name);
- // XXX: Overflow issue in source selector
- _current_index_id = getNewAbsoluteId() - _last_fusion_id;
- assert(_current_index_id < ISourceSelector::SOURCE_LIMIT);
+ set_id_for_new_memory_index();
_selector->setDefaultSource(_current_index_id);
_source_selector_changes = 0;
}
@@ -604,10 +606,10 @@ IndexMaintainer::flushMemoryIndex(FlushArgs &args,
// Called by a flush worker thread
ChangeGens changeGens = getChangeGens();
IMemoryIndex &memoryIndex = *args.old_index;
- Schema::SP prunedSchema = memoryIndex.getPrunedSchema();
- IDiskIndex::SP diskIndex = flushMemoryIndex(memoryIndex, args.old_absolute_id,
- docIdLimit, args.flush_serial_num,
- saveInfo);
+ auto prunedSchema = memoryIndex.getPrunedSchema();
+ auto diskIndex = flushMemoryIndex(memoryIndex, args.old_absolute_id,
+ docIdLimit, args.flush_serial_num,
+ saveInfo);
// Post processing after memory index has been written to disk and
// opened as disk index.
args._changeGens = changeGens;
@@ -619,7 +621,7 @@ IndexMaintainer::flushMemoryIndex(FlushArgs &args,
void
-IndexMaintainer::reconfigureAfterFlush(FlushArgs &args, IDiskIndex::SP &diskIndex)
+IndexMaintainer::reconfigureAfterFlush(FlushArgs &args, std::shared_ptr<IDiskIndex>& diskIndex)
{
// Called by a flush worker thread
for (;;) {
@@ -631,12 +633,12 @@ IndexMaintainer::reconfigureAfterFlush(FlushArgs &args, IDiskIndex::SP &diskInde
return;
}
ChangeGens changeGens = getChangeGens();
- Schema::SP prunedSchema = args.old_index->getPrunedSchema();
+ auto prunedSchema = args.old_index->getPrunedSchema();
const string indexDir = getFlushDir(args.old_absolute_id);
if (prunedSchema) {
updateDiskIndexSchema(indexDir, *prunedSchema, noSerialNumHigh);
}
- IDiskIndex::SP reloadedDiskIndex = reloadDiskIndex(*diskIndex);
+ auto reloadedDiskIndex = reloadDiskIndex(*diskIndex);
diskIndex = reloadedDiskIndex;
args._changeGens = changeGens;
args._prunedSchema = prunedSchema;
@@ -645,7 +647,7 @@ IndexMaintainer::reconfigureAfterFlush(FlushArgs &args, IDiskIndex::SP &diskInde
bool
-IndexMaintainer::doneFlush(FlushArgs *args, IDiskIndex::SP *disk_index) {
+IndexMaintainer::doneFlush(FlushArgs *args, std::shared_ptr<IDiskIndex> *disk_index) {
// Called by doFlush via reconfigurer
assert(_ctx.getThreadingService().master().isCurrentThread());
LockGuard state_lock(_state_lock);
@@ -683,7 +685,7 @@ IndexMaintainer::canRunFusion(const FusionSpec &spec) const
}
bool
-IndexMaintainer::doneFusion(FusionArgs *args, IDiskIndex::SP *new_index)
+IndexMaintainer::doneFusion(FusionArgs *args, std::shared_ptr<IDiskIndex>* new_index)
{
// Called by runFusion via reconfigurer
assert(_ctx.getThreadingService().master().isCurrentThread());
@@ -711,13 +713,12 @@ IndexMaintainer::doneFusion(FusionArgs *args, IDiskIndex::SP *new_index)
_activeFusionPrunedSchema.reset();
}
- ISearchableIndexCollection::SP currentLeaf;
+ std::shared_ptr<ISearchableIndexCollection> currentLeaf;
{
LockGuard lock(_new_search_lock);
currentLeaf = getLeaf(lock, _source_list);
}
- ISearchableIndexCollection::UP fsc =
- IndexCollection::replaceAndRenumber(_selector, *currentLeaf, id_diff, *new_index);
+ auto fsc = IndexCollection::replaceAndRenumber(_selector, *currentLeaf, id_diff, *new_index);
fsc->setCurrentIndex(_current_index_id);
{
@@ -732,7 +733,7 @@ IndexMaintainer::makeSureAllRemainingWarmupIsDone(std::shared_ptr<WarmupIndexCol
{
// called by warmupDone via reconfigurer, warmupDone() doesn't wait for us
assert(_ctx.getThreadingService().master().isCurrentThread());
- ISearchableIndexCollection::SP warmIndex;
+ std::shared_ptr<ISearchableIndexCollection> warmIndex;
{
LockGuard state_lock(_state_lock);
if (keepAlive == _source_list) {
@@ -767,26 +768,8 @@ IndexMaintainer::warmupDone(std::shared_ptr<WarmupIndexCollection> current)
}
}
-namespace {
-
-bool
-has_matching_interleaved_features(const Schema& old_schema, const Schema& new_schema)
-{
- for (SchemaUtil::IndexIterator itr(new_schema); itr.isValid(); ++itr) {
- if (itr.hasMatchingOldFields(old_schema) &&
- !itr.has_matching_use_interleaved_features(old_schema))
- {
- return false;
- }
- }
- return true;
-}
-
-}
-
-
void
-IndexMaintainer::doneSetSchema(SetSchemaArgs &args, IMemoryIndex::SP &newIndex)
+IndexMaintainer::doneSetSchema(SetSchemaArgs &args, std::shared_ptr<IMemoryIndex>& newIndex, SerialNum serial_num)
{
assert(_ctx.getThreadingService().master().isCurrentThread()); // with idle index executor
LockGuard state_lock(_state_lock);
@@ -794,11 +777,11 @@ IndexMaintainer::doneSetSchema(SetSchemaArgs &args, IMemoryIndex::SP &newIndex)
args._oldSchema = _schema; // Delay destruction
args._oldIndex = _current_index; // Delay destruction
args._oldSourceList = _source_list; // Delay destruction
- uint32_t oldAbsoluteId = _current_index_id + _last_fusion_id;
+ uint32_t oldAbsoluteId = get_absolute_id();
string selectorName = IndexDiskLayout::getSelectorFileName(getFlushDir(oldAbsoluteId));
SerialNum freezeSerialNum = current_serial_num();
bool dropEmptyLast = false;
- SaveInfo::UP saveInfo;
+ std::unique_ptr<SaveInfo> saveInfo;
LOG(info, "Making new schema. Id = %u. Serial num = %llu", oldAbsoluteId, (unsigned long long) freezeSerialNum);
{
@@ -811,20 +794,18 @@ IndexMaintainer::doneSetSchema(SetSchemaArgs &args, IMemoryIndex::SP &newIndex)
if (!dropEmptyLast) {
// Keep on using same source selector with extended valid range
saveInfo = getSourceSelector().extractSaveInfo(selectorName);
- // XXX: Overflow issue in source selector
- _current_index_id = getNewAbsoluteId() - _last_fusion_id;
- assert(_current_index_id < ISourceSelector::SOURCE_LIMIT);
+ set_id_for_new_memory_index();
_selector->setDefaultSource(_current_index_id);
// Extra index to flush next time flushing is performed
_frozenMemoryIndexes.emplace_back(args._oldIndex, freezeSerialNum, std::move(saveInfo), oldAbsoluteId);
}
_current_index = newIndex;
- // Non-matching interleaved features in schemas means that we need to
- // reconstruct or drop interleaved features in posting lists.
- // If so, we must flush the new index to disk even if it is empty.
- // This ensures that 2x triggerFlush will run fusion
- // to reconstruct or drop interleaved features in the posting lists.
- _flush_empty_current_index = !has_matching_interleaved_features(args._oldSchema, args._newSchema);
+ if (serial_num > flush_serial_num() && get_absolute_id() > 1) {
+ consider_urgent_flush(args._oldSchema, args._newSchema, get_absolute_id());
+ }
+ // If schema changes triggered a need for urgent flush then we must
+ // be able to flush the new index to disk even if it is empty.
+ _flush_empty_current_index = (_urgent_flush_id == get_absolute_id());
}
if (dropEmptyLast) {
replaceSource(_current_index_id, _current_index);
@@ -842,7 +823,7 @@ IndexMaintainer::getSchema(void) const
return _schema;
}
-Schema::SP
+std::shared_ptr<Schema>
IndexMaintainer::getActiveFusionPrunedSchema(void) const
{
LockGuard lock(_index_update_lock);
@@ -888,6 +869,7 @@ IndexMaintainer::IndexMaintainer(const IndexMaintainerConfig &config,
_last_fusion_id(),
_next_id(),
_current_index_id(),
+ _urgent_flush_id(),
_current_index(),
_flush_empty_current_index(false),
_current_serial_num(0),
@@ -938,8 +920,7 @@ IndexMaintainer::IndexMaintainer(const IndexMaintainerConfig &config,
_selector = getSourceSelector().cloneAndSubtract(ost.str(), id_diff);
assert(_last_fusion_id == _selector->getBaseId());
}
- _current_index_id = getNewAbsoluteId() - _last_fusion_id;
- assert(_current_index_id < ISourceSelector::SOURCE_LIMIT);
+ set_id_for_new_memory_index();
_selector->setDefaultSource(_current_index_id);
auto sourceList = loadDiskIndexes(spec, std::make_unique<IndexCollection>(_selector));
_current_index = operations.createMemoryIndex(_schema, *sourceList, current_serial_num());
@@ -952,6 +933,7 @@ IndexMaintainer::IndexMaintainer(const IndexMaintainerConfig &config,
pruneRemovedFields(_schema, config.getSerialNum());
}));
_ctx.getThreadingService().master().sync();
+ consider_initial_urgent_flush();
}
IndexMaintainer::~IndexMaintainer()
@@ -961,7 +943,7 @@ IndexMaintainer::~IndexMaintainer()
_selector.reset();
}
-FlushTask::UP
+std::unique_ptr<FlushTask>
IndexMaintainer::initFlush(SerialNum serialNum, searchcorespi::FlushStats * stats)
{
assert(_ctx.getThreadingService().master().isCurrentThread()); // while flush engine scheduler thread waits
@@ -970,7 +952,7 @@ IndexMaintainer::initFlush(SerialNum serialNum, searchcorespi::FlushStats * stat
set_current_serial_num(std::max(current_serial_num(), serialNum));
}
- IMemoryIndex::SP new_index(_operations.createMemoryIndex(getSchema(), *_current_index, current_serial_num()));
+ auto new_index(_operations.createMemoryIndex(getSchema(), *_current_index, current_serial_num()));
FlushArgs args;
args.stats = stats;
// Ensure that all index thread tasks accessing memory index have completed.
@@ -990,7 +972,7 @@ IndexMaintainer::initFlush(SerialNum serialNum, searchcorespi::FlushStats * stat
LOG(debug, "No memory index to flush. Update serial number and flush time to current: "
"flushSerialNum(%" PRIu64 "), lastFlushTime(%f)",
flush_serial_num(), vespalib::to_s(_lastFlushTime.time_since_epoch()));
- return FlushTask::UP();
+ return {};
}
SerialNum realSerialNum = args.flush_serial_num;
return makeLambdaFlushTask([this, myargs=std::move(args)]() mutable { doFlush(std::move(myargs)); }, realSerialNum);
@@ -1115,12 +1097,12 @@ IndexMaintainer::runFusion(const FusionSpec &fusion_spec, std::shared_ptr<search
}
const string new_fusion_dir = getFusionDir(new_fusion_id);
- Schema::SP prunedSchema = getActiveFusionPrunedSchema();
+ auto prunedSchema = getActiveFusionPrunedSchema();
if (prunedSchema) {
updateDiskIndexSchema(new_fusion_dir, *prunedSchema, noSerialNumHigh);
}
ChangeGens changeGens = getChangeGens();
- IDiskIndex::SP new_index(loadDiskIndex(new_fusion_dir));
+ auto new_index(loadDiskIndex(new_fusion_dir));
remove_fusion_index_guard.reset();
// Post processing after fusion operation has completed and new disk
@@ -1142,7 +1124,7 @@ IndexMaintainer::runFusion(const FusionSpec &fusion_spec, std::shared_ptr<search
if (prunedSchema) {
updateDiskIndexSchema(new_fusion_dir, *prunedSchema, noSerialNumHigh);
}
- IDiskIndex::SP diskIndex2;
+ std::shared_ptr<IDiskIndex> diskIndex2;
diskIndex2 = reloadDiskIndex(*new_index);
new_index = diskIndex2;
args._changeGens = changeGens;
@@ -1196,7 +1178,7 @@ IndexMaintainer::getFusionStats() const
{
// Called by flush engine scheduler thread (from getFlushTargets())
FusionStats stats;
- IndexSearchable::SP source_list;
+ std::shared_ptr<IndexSearchable> source_list;
{
LockGuard lock(_new_search_lock);
@@ -1315,14 +1297,14 @@ IndexMaintainer::setSchema(const Schema & schema, SerialNum serialNum)
{
assert(_ctx.getThreadingService().master().isCurrentThread());
pruneRemovedFields(schema, serialNum);
- IMemoryIndex::SP new_index(_operations.createMemoryIndex(schema, *_current_index, current_serial_num()));
+ auto new_index(_operations.createMemoryIndex(schema, *_current_index, current_serial_num()));
SetSchemaArgs args;
args._newSchema = schema;
// Ensure that all index thread tasks accessing memory index have completed.
commit_and_wait();
// Everything should be quiet now.
- doneSetSchema(args, new_index);
+ doneSetSchema(args, new_index, serialNum);
// Source collection has now changed, caller must reconfigure further
// as appropriate.
}
@@ -1331,8 +1313,8 @@ void
IndexMaintainer::pruneRemovedFields(const Schema &schema, SerialNum serialNum)
{
assert(_ctx.getThreadingService().master().isCurrentThread());
- ISearchableIndexCollection::SP new_source_list;
- IIndexCollection::SP coll = getSourceCollection();
+ std::shared_ptr<ISearchableIndexCollection> new_source_list;
+ auto coll = getSourceCollection();
updateIndexSchemas(*coll, schema, serialNum);
updateActiveFusionPrunedSchema(schema);
{
@@ -1360,4 +1342,78 @@ IndexMaintainer::setMaxFlushed(uint32_t maxFlushed)
_maxFlushed = maxFlushed;
}
+void
+IndexMaintainer::consider_urgent_flush(const Schema& old_schema, const Schema& new_schema, uint32_t flush_id)
+{
+ // Non-matching interleaved features in schemas means that we need to
+ // reconstruct or drop interleaved features in posting lists. Schedule
+ // urgent flush until all indexes are in sync.
+ for (SchemaUtil::IndexIterator itr(new_schema); itr.isValid(); ++itr) {
+ if (itr.hasMatchingOldFields(old_schema) &&
+ !itr.has_matching_use_interleaved_features(old_schema))
+ {
+ _urgent_flush_id = flush_id;
+ break;
+ }
+ }
+}
+
+void
+IndexMaintainer::consider_initial_urgent_flush()
+{
+ const Schema *prev_schema = nullptr;
+ auto coll = getSourceCollection();
+ uint32_t count = coll->getSourceCount();
+ for (uint32_t i = 0; i < count; ++i) {
+ IndexSearchable &is = coll->getSearchable(i);
+ const auto *const d = dynamic_cast<const DiskIndexWithDestructorCallback *>(&is);
+ if (d != nullptr) {
+ auto schema = &d->getSchema();
+ if (prev_schema != nullptr) {
+ consider_urgent_flush(*prev_schema, *schema, _last_fusion_id + coll->getSourceId(i));
+ }
+ prev_schema = schema;
+ }
+ }
+}
+
+uint32_t
+IndexMaintainer::get_urgent_flush_id() const
+{
+ LockGuard lock(_index_update_lock);
+ return _urgent_flush_id;
+}
+
+bool
+IndexMaintainer::urgent_memory_index_flush() const
+{
+ LockGuard lock(_index_update_lock);
+ for (auto& frozen : _frozenMemoryIndexes) {
+ if (frozen._absoluteId == _urgent_flush_id) {
+ return true;
+ }
+ }
+ if (get_absolute_id() == _urgent_flush_id) {
+ return true;
+ }
+ return false;
+}
+
+bool
+IndexMaintainer::urgent_disk_index_fusion() const
+{
+ uint32_t urgent_flush_id = get_urgent_flush_id();
+ LockGuard lock(_fusion_lock);
+ auto& flush_ids = _fusion_spec.flush_ids;
+ return std::find(flush_ids.begin(), flush_ids.end(), urgent_flush_id) != std::end(flush_ids);
+}
+
+bool
+IndexMaintainer::has_pending_urgent_flush() const
+{
+ uint32_t urgent_flush_id = get_urgent_flush_id();
+ LockGuard lock(_fusion_lock);
+ return urgent_flush_id > _fusion_spec.last_fusion_id;
+}
+
}