aboutsummaryrefslogtreecommitdiffstats
path: root/searchlib/src/vespa/searchlib/docstore
diff options
context:
space:
mode:
Diffstat (limited to 'searchlib/src/vespa/searchlib/docstore')
-rw-r--r--searchlib/src/vespa/searchlib/docstore/CMakeLists.txt2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/chunk.cpp11
-rw-r--r--searchlib/src/vespa/searchlib/docstore/chunk.h7
-rw-r--r--searchlib/src/vespa/searchlib/docstore/chunkformat.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/chunkformat.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/chunkformats.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/chunkformats.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/compacter.cpp149
-rw-r--r--searchlib/src/vespa/searchlib/docstore/compacter.h69
-rw-r--r--searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_stats.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/data_store_storage_stats.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/documentstore.cpp6
-rw-r--r--searchlib/src/vespa/searchlib/docstore/documentstore.h4
-rw-r--r--searchlib/src/vespa/searchlib/docstore/filechunk.cpp192
-rw-r--r--searchlib/src/vespa/searchlib/docstore/filechunk.h59
-rw-r--r--searchlib/src/vespa/searchlib/docstore/ibucketizer.h6
-rw-r--r--searchlib/src/vespa/searchlib/docstore/idatastore.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/idatastore.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/idocumentstore.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/idocumentstore.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/lid_info.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/lid_info.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/liddatastore.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/logdatastore.cpp46
-rw-r--r--searchlib/src/vespa/searchlib/docstore/logdatastore.h9
-rw-r--r--searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/logdocumentstore.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/randread.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/randreaders.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/randreaders.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/storebybucket.cpp51
-rw-r--r--searchlib/src/vespa/searchlib/docstore/storebybucket.h76
-rw-r--r--searchlib/src/vespa/searchlib/docstore/summaryexceptions.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/summaryexceptions.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/value.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/value.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/visitcache.cpp2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/visitcache.h2
-rw-r--r--searchlib/src/vespa/searchlib/docstore/writeablefilechunk.cpp28
-rw-r--r--searchlib/src/vespa/searchlib/docstore/writeablefilechunk.h8
44 files changed, 424 insertions, 355 deletions
diff --git a/searchlib/src/vespa/searchlib/docstore/CMakeLists.txt b/searchlib/src/vespa/searchlib/docstore/CMakeLists.txt
index 89804f79a13..1e2b6aa58fb 100644
--- a/searchlib/src/vespa/searchlib/docstore/CMakeLists.txt
+++ b/searchlib/src/vespa/searchlib/docstore/CMakeLists.txt
@@ -1,4 +1,4 @@
-# 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.
vespa_add_library(searchlib_docstore OBJECT
SOURCES
chunk.cpp
diff --git a/searchlib/src/vespa/searchlib/docstore/chunk.cpp b/searchlib/src/vespa/searchlib/docstore/chunk.cpp
index 60255af3521..d7a93f3a31a 100644
--- a/searchlib/src/vespa/searchlib/docstore/chunk.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/chunk.cpp
@@ -1,4 +1,4 @@
-// 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 "chunk.h"
#include "chunkformats.h"
@@ -8,15 +8,16 @@
namespace search {
LidMeta
-Chunk::append(uint32_t lid, const void * buffer, size_t len)
+Chunk::append(uint32_t lid, ConstBufferRef data)
{
vespalib::nbostream & os = getData();
size_t oldSz(os.size());
+ uint32_t len = data.size();
std::lock_guard guard(_lock);
- os << lid << static_cast<uint32_t>(len);
- os.write(buffer, len);
+ os << lid << len;
+ os.write(data.c_str(), len);
_lids.emplace_back(lid, len, oldSz);
- return LidMeta(lid, len);
+ return {lid, len};
}
ssize_t
diff --git a/searchlib/src/vespa/searchlib/docstore/chunk.h b/searchlib/src/vespa/searchlib/docstore/chunk.h
index 211165934e1..725f9739655 100644
--- a/searchlib/src/vespa/searchlib/docstore/chunk.h
+++ b/searchlib/src/vespa/searchlib/docstore/chunk.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
@@ -59,6 +59,7 @@ class Chunk {
public:
using UP = std::unique_ptr<Chunk>;
using CompressionConfig = vespalib::compression::CompressionConfig;
+ using ConstBufferRef = vespalib::ConstBufferRef;
class Config {
public:
Config(size_t maxBytes) noexcept : _maxBytes(maxBytes) { }
@@ -84,7 +85,7 @@ public:
Chunk(uint32_t id, const Config & config);
Chunk(uint32_t id, const void * buffer, size_t len);
~Chunk();
- LidMeta append(uint32_t lid, const void * buffer, size_t len);
+ LidMeta append(uint32_t lid, ConstBufferRef data);
ssize_t read(uint32_t lid, vespalib::DataBuffer & buffer) const;
std::pair<size_t, vespalib::alloc::Alloc> read(uint32_t lid) const;
size_t count() const { return _lids.size(); }
@@ -96,7 +97,7 @@ public:
void pack(uint64_t lastSerial, vespalib::DataBuffer & buffer, CompressionConfig compression);
uint64_t getLastSerial() const { return _lastSerial; }
uint32_t getId() const { return _id; }
- vespalib::ConstBufferRef getLid(uint32_t lid) const;
+ ConstBufferRef getLid(uint32_t lid) const;
const vespalib::nbostream & getData() const;
bool hasRoom(size_t len) const;
vespalib::MemoryUsage getMemoryUsage() const;
diff --git a/searchlib/src/vespa/searchlib/docstore/chunkformat.cpp b/searchlib/src/vespa/searchlib/docstore/chunkformat.cpp
index 2f622cfa78c..b359dace09c 100644
--- a/searchlib/src/vespa/searchlib/docstore/chunkformat.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/chunkformat.cpp
@@ -1,4 +1,4 @@
-// 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 "chunkformats.h"
#include <vespa/vespalib/util/compressor.h>
diff --git a/searchlib/src/vespa/searchlib/docstore/chunkformat.h b/searchlib/src/vespa/searchlib/docstore/chunkformat.h
index 6c76840ea21..5a22c887a23 100644
--- a/searchlib/src/vespa/searchlib/docstore/chunkformat.h
+++ b/searchlib/src/vespa/searchlib/docstore/chunkformat.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/chunkformats.cpp b/searchlib/src/vespa/searchlib/docstore/chunkformats.cpp
index c895fceb6bf..fa576a3a513 100644
--- a/searchlib/src/vespa/searchlib/docstore/chunkformats.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/chunkformats.cpp
@@ -1,4 +1,4 @@
-// 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 "chunkformats.h"
#include <vespa/vespalib/util/crc.h>
diff --git a/searchlib/src/vespa/searchlib/docstore/chunkformats.h b/searchlib/src/vespa/searchlib/docstore/chunkformats.h
index 47ee9ea8a6c..b607e0fae30 100644
--- a/searchlib/src/vespa/searchlib/docstore/chunkformats.h
+++ b/searchlib/src/vespa/searchlib/docstore/chunkformats.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/compacter.cpp b/searchlib/src/vespa/searchlib/docstore/compacter.cpp
index c886e52659f..91faafc2a4e 100644
--- a/searchlib/src/vespa/searchlib/docstore/compacter.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/compacter.cpp
@@ -1,10 +1,10 @@
-// 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 "compacter.h"
#include "logdatastore.h"
#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/util/array.hpp>
-#include <cinttypes>
+#include <cassert>
#include <vespa/log/log.h>
LOG_SETUP(".searchlib.docstore.compacter");
@@ -14,83 +14,124 @@ namespace search::docstore {
using vespalib::alloc::Alloc;
namespace {
- static constexpr size_t INITIAL_BACKING_BUFFER_SIZE = 64_Mi;
+ constexpr size_t INITIAL_BACKING_BUFFER_SIZE = 64_Mi;
}
void
-Compacter::write(LockGuard guard, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz) {
+Compacter::write(LockGuard guard, uint32_t chunkId, uint32_t lid, ConstBufferRef data) {
(void) chunkId;
- FileChunk::FileId fileId= _ds.getActiveFileId(guard);
- _ds.write(std::move(guard), fileId, lid, buffer, sz);
+ FileChunk::FileId fileId = _ds.getActiveFileId(guard);
+ _ds.write(std::move(guard), fileId, lid, data);
+}
+
+BucketIndexStore::BucketIndexStore(size_t maxSignificantBucketBits, uint32_t numPartitions) noexcept
+ : _inSignificantBucketBits((maxSignificantBucketBits > 8) ? (maxSignificantBucketBits - 8) : 0),
+ _where(),
+ _numPartitions(numPartitions),
+ _readyForIterate(true)
+{}
+BucketIndexStore::~BucketIndexStore() = default;
+
+void
+BucketIndexStore::prepareForIterate() {
+ std::sort(_where.begin(), _where.end());
+ _readyForIterate = true;
+}
+
+void
+BucketIndexStore::store(const StoreByBucket::Index & index) {
+ _where.push_back(index);
+ _readyForIterate = false;
+}
+
+size_t
+BucketIndexStore::getBucketCount() const noexcept {
+ if (_where.empty()) return 0;
+
+ size_t count = 0;
+ document::BucketId prev = _where.front()._bucketId;
+ for (const auto & lid : _where) {
+ if (lid._bucketId != prev) {
+ count++;
+ prev = lid._bucketId;
+ }
+ }
+ return count + 1;
+}
+
+std::unique_ptr<StoreByBucket::IndexIterator>
+BucketIndexStore::createIterator(uint32_t partitionId) const {
+ assert(_readyForIterate);
+ return std::make_unique<LidIterator>(*this, partitionId);
+}
+
+BucketIndexStore::LidIterator::LidIterator(const BucketIndexStore & store, size_t partitionId)
+ : _store(store),
+ _partitionId(partitionId),
+ _current(_store._where.begin())
+{}
+
+bool
+BucketIndexStore::LidIterator::has_next() noexcept {
+ for (;(_current != _store._where.end()) && (_store.toPartitionId(_current->_bucketId) != _partitionId); _current++);
+ return (_current != _store._where.end()) && (_store.toPartitionId(_current->_bucketId) == _partitionId);
+}
+
+StoreByBucket::Index
+BucketIndexStore::LidIterator::next() noexcept {
+ return *_current++;
}
BucketCompacter::BucketCompacter(size_t maxSignificantBucketBits, CompressionConfig compression, LogDataStore & ds,
- Executor & executor, const IBucketizer & bucketizer, FileId source, FileId destination) :
- _unSignificantBucketBits((maxSignificantBucketBits > 8) ? (maxSignificantBucketBits - 8) : 0),
- _sourceFileId(source),
- _destinationFileId(destination),
- _ds(ds),
- _bucketizer(bucketizer),
- _writeCount(0),
- _maxBucketGuardDuration(vespalib::duration::zero()),
- _lastSample(vespalib::steady_clock::now()),
- _lock(),
- _backingMemory(Alloc::alloc(INITIAL_BACKING_BUFFER_SIZE), &_lock),
- _tmpStore(),
- _lidGuard(ds.getLidReadGuard()),
- _bucketizerGuard(),
- _stat()
+ Executor & executor, const IBucketizer & bucketizer, FileId source, FileId destination)
+ : _sourceFileId(source),
+ _destinationFileId(destination),
+ _ds(ds),
+ _bucketizer(bucketizer),
+ _lock(),
+ _backingMemory(Alloc::alloc(INITIAL_BACKING_BUFFER_SIZE), &_lock),
+ _bucketIndexStore(maxSignificantBucketBits, NUM_PARTITIONS),
+ _tmpStore(),
+ _lidGuard(ds.getLidReadGuard()),
+ _stat()
{
- _tmpStore.reserve(256);
- for (size_t i(0); i < 256; i++) {
- _tmpStore.emplace_back(_backingMemory, executor, compression);
+ for (auto & partition : _tmpStore) {
+ partition = std::make_unique<StoreByBucket>(_bucketIndexStore, _backingMemory, executor, compression);
}
}
+BucketCompacter::~BucketCompacter() = default;
+
FileChunk::FileId
BucketCompacter::getDestinationId(const LockGuard & guard) const {
return (_destinationFileId.isActive()) ? _ds.getActiveFileId(guard) : _destinationFileId;
}
void
-BucketCompacter::write(LockGuard guard, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz)
+BucketCompacter::write(LockGuard guard, uint32_t chunkId, uint32_t lid, ConstBufferRef data)
{
- if (_writeCount++ == 0) {
- _bucketizerGuard = _bucketizer.getGuard();
- _lastSample = vespalib::steady_clock::now();
- }
guard.unlock();
- BucketId bucketId = (sz > 0) ? _bucketizer.getBucketOf(_bucketizerGuard, lid) : BucketId();
- uint64_t sortableBucketId = bucketId.toKey();
- _tmpStore[(sortableBucketId >> _unSignificantBucketBits) % _tmpStore.size()].add(bucketId, chunkId, lid, buffer, sz);
- if ((_writeCount % 1000) == 0) {
- _bucketizerGuard = _bucketizer.getGuard();
- vespalib::steady_time now = vespalib::steady_clock::now();
- _maxBucketGuardDuration = std::max(_maxBucketGuardDuration, now - _lastSample);
- _lastSample = now;
- }
+ BucketId bucketId = (data.size() > 0) ? _bucketizer.getBucketOf(_bucketizer.getGuard(), lid) : BucketId();
+ _tmpStore[_bucketIndexStore.toPartitionId(bucketId)]->add(bucketId, chunkId, lid, data);
}
void
BucketCompacter::close()
{
- _bucketizerGuard = GenerationHandler::Guard();
- vespalib::duration lastBucketGuardDuration = vespalib::steady_clock::now() - _lastSample;
- size_t lidCount1(0);
- size_t bucketCount(0);
size_t chunkCount(0);
- for (const StoreByBucket & store : _tmpStore) {
- lidCount1 += store.getLidCount();
- bucketCount += store.getBucketCount();
- chunkCount += store.getChunkCount();
+ for (const auto & store : _tmpStore) {
+ store->close();
+ chunkCount += store->getChunkCount();
}
- LOG(info, "Have read %ld lids and placed them in %ld buckets. Temporary compressed in %ld chunks."
- " Max bucket guard held for %" PRId64 " us, and last before close for %" PRId64 " us",
- lidCount1, bucketCount, chunkCount, vespalib::count_us(_maxBucketGuardDuration), vespalib::count_us(lastBucketGuardDuration));
+ _bucketIndexStore.prepareForIterate();
+ LOG(info, "Have read %ld lids and placed them in %ld buckets. Temporary compressed in %ld chunks.",
+ _bucketIndexStore.getLidCount(), _bucketIndexStore.getBucketCount(), chunkCount);
- for (StoreByBucket & store : _tmpStore) {
- store.drain(*this);
+ for (size_t partId(0); partId < _tmpStore.size(); partId++) {
+ auto partIterator = _bucketIndexStore.createIterator(partId);
+ _tmpStore[partId]->drain(*this, *partIterator);
}
+ // All partitions using _backingMemory should be destructed before clearing.
_backingMemory.clear();
size_t lidCount(0);
@@ -101,14 +142,14 @@ BucketCompacter::close()
}
void
-BucketCompacter::write(BucketId bucketId, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz)
+BucketCompacter::write(BucketId bucketId, uint32_t chunkId, uint32_t lid, ConstBufferRef data)
{
_stat[bucketId.getId()]++;
LockGuard guard(_ds.getLidGuard(lid));
- LidInfo lidInfo(_sourceFileId.getId(), chunkId, sz);
+ LidInfo lidInfo(_sourceFileId.getId(), chunkId, data.size());
if (_ds.getLid(_lidGuard, lid) == lidInfo) {
FileId fileId = getDestinationId(guard);
- _ds.write(std::move(guard), fileId, lid, buffer, sz);
+ _ds.write(std::move(guard), fileId, lid, data);
}
}
diff --git a/searchlib/src/vespa/searchlib/docstore/compacter.h b/searchlib/src/vespa/searchlib/docstore/compacter.h
index 0d7633b1699..f44d7c341d2 100644
--- a/searchlib/src/vespa/searchlib/docstore/compacter.h
+++ b/searchlib/src/vespa/searchlib/docstore/compacter.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
@@ -16,20 +16,52 @@ namespace search::docstore {
class Compacter : public IWriteData
{
public:
- Compacter(LogDataStore & ds) : _ds(ds) { }
- void write(LockGuard guard, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz) override;
+ explicit Compacter(LogDataStore & ds) : _ds(ds) { }
+ void write(LockGuard guard, uint32_t chunkId, uint32_t lid, ConstBufferRef data) override;
void close() override { }
private:
LogDataStore & _ds;
};
+class BucketIndexStore : public StoreByBucket::StoreIndex {
+public:
+ BucketIndexStore(size_t maxSignificantBucketBits, uint32_t numPartitions) noexcept;
+ ~BucketIndexStore() override;
+ size_t toPartitionId(document::BucketId bucketId) const noexcept {
+ uint64_t sortableBucketId = bucketId.toKey();
+ return (sortableBucketId >> _inSignificantBucketBits) % _numPartitions;
+ }
+ void store(const StoreByBucket::Index & index) override;
+ size_t getBucketCount() const noexcept;
+ size_t getLidCount() const noexcept { return _where.size(); }
+ void prepareForIterate();
+ std::unique_ptr<StoreByBucket::IndexIterator> createIterator(uint32_t partitionId) const;
+private:
+ using IndexVector = std::vector<StoreByBucket::Index, vespalib::allocator_large<StoreByBucket::Index>>;
+ class LidIterator : public StoreByBucket::IndexIterator {
+ public:
+ LidIterator(const BucketIndexStore & bc, size_t partitionId);
+ bool has_next() noexcept override;
+ StoreByBucket::Index next() noexcept override;
+ private:
+ const BucketIndexStore & _store;
+ size_t _partitionId;
+ IndexVector::const_iterator _current;
+ };
+ size_t _inSignificantBucketBits;
+ IndexVector _where;
+ uint32_t _numPartitions;
+ bool _readyForIterate;
+};
+
/**
* This will split the incoming data into buckets.
* The buckets data will then be written out in bucket order.
* The buckets will be ordered, and the objects inside the buckets will be further ordered.
* All data are kept compressed to minimize memory usage.
**/
-class BucketCompacter : public IWriteData, public StoreByBucket::IWrite
+class BucketCompacter : public IWriteData,
+ public StoreByBucket::IWrite
{
using CompressionConfig = vespalib::compression::CompressionConfig;
using Executor = vespalib::Executor;
@@ -37,25 +69,24 @@ public:
using FileId = FileChunk::FileId;
BucketCompacter(size_t maxSignificantBucketBits, CompressionConfig compression, LogDataStore & ds,
Executor & executor, const IBucketizer & bucketizer, FileId source, FileId destination);
- void write(LockGuard guard, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz) override ;
- void write(BucketId bucketId, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz) override;
+ ~BucketCompacter() override;
+ void write(LockGuard guard, uint32_t chunkId, uint32_t lid, ConstBufferRef data) override;
+ void write(BucketId bucketId, uint32_t chunkId, uint32_t lid, ConstBufferRef data) override;
void close() override;
private:
+ static constexpr size_t NUM_PARTITIONS = 256;
using GenerationHandler = vespalib::GenerationHandler;
+ using Partitions = std::array<std::unique_ptr<StoreByBucket>, NUM_PARTITIONS>;
FileId getDestinationId(const LockGuard & guard) const;
- size_t _unSignificantBucketBits;
- FileId _sourceFileId;
- FileId _destinationFileId;
- LogDataStore & _ds;
- const IBucketizer & _bucketizer;
- uint64_t _writeCount;
- vespalib::duration _maxBucketGuardDuration;
- vespalib::steady_time _lastSample;
- std::mutex _lock;
- vespalib::MemoryDataStore _backingMemory;
- std::vector<StoreByBucket> _tmpStore;
- GenerationHandler::Guard _lidGuard;
- GenerationHandler::Guard _bucketizerGuard;
+ FileId _sourceFileId;
+ FileId _destinationFileId;
+ LogDataStore & _ds;
+ const IBucketizer & _bucketizer;
+ std::mutex _lock;
+ vespalib::MemoryDataStore _backingMemory;
+ BucketIndexStore _bucketIndexStore;
+ Partitions _tmpStore;
+ GenerationHandler::Guard _lidGuard;
vespalib::hash_map<uint64_t, uint32_t> _stat;
};
diff --git a/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.cpp b/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.cpp
index 42938ad343a..74ad7d10912 100644
--- a/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.cpp
@@ -1,4 +1,4 @@
-// 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 "data_store_file_chunk_id.h"
#include "filechunk.h"
diff --git a/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.h b/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.h
index 39de89eeb2e..86e10633a68 100644
--- a/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.h
+++ b/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_id.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_stats.h b/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_stats.h
index 231e34b6c64..26abcd08e71 100644
--- a/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_stats.h
+++ b/searchlib/src/vespa/searchlib/docstore/data_store_file_chunk_stats.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/data_store_storage_stats.h b/searchlib/src/vespa/searchlib/docstore/data_store_storage_stats.h
index 5f0a7c493f9..1cb15fcb411 100644
--- a/searchlib/src/vespa/searchlib/docstore/data_store_storage_stats.h
+++ b/searchlib/src/vespa/searchlib/docstore/data_store_storage_stats.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.cpp b/searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.cpp
index 0a83eb613e1..8afa8872c85 100644
--- a/searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.cpp
@@ -1,4 +1,4 @@
-// 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 "document_store_visitor_progress.h"
diff --git a/searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.h b/searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.h
index 49b9634cc50..211d738607c 100644
--- a/searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.h
+++ b/searchlib/src/vespa/searchlib/docstore/document_store_visitor_progress.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
#include "idocumentstore.h"
diff --git a/searchlib/src/vespa/searchlib/docstore/documentstore.cpp b/searchlib/src/vespa/searchlib/docstore/documentstore.cpp
index 7d585007d76..bb0dcbe699f 100644
--- a/searchlib/src/vespa/searchlib/docstore/documentstore.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/documentstore.cpp
@@ -1,4 +1,4 @@
-// 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 "documentstore.h"
#include "visitcache.h"
@@ -121,6 +121,10 @@ DocumentStore::Config::operator == (const Config &rhs) const {
(_compression == rhs._compression);
}
+size_t
+DocumentStore::getCacheCapacity() const {
+ return _cache->capacityBytes();
+}
DocumentStore::DocumentStore(const Config & config, IDataStore & store)
: IDocumentStore(),
diff --git a/searchlib/src/vespa/searchlib/docstore/documentstore.h b/searchlib/src/vespa/searchlib/docstore/documentstore.h
index bb62b09123f..aa849371b6d 100644
--- a/searchlib/src/vespa/searchlib/docstore/documentstore.h
+++ b/searchlib/src/vespa/searchlib/docstore/documentstore.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
@@ -37,6 +37,7 @@ public:
{ }
CompressionConfig getCompression() const { return _compression; }
size_t getMaxCacheBytes() const { return _maxCacheBytes; }
+ Config & disableCache() { _maxCacheBytes = 0; return *this; }
Config & updateStrategy(UpdateStrategy strategy) { _updateStrategy = strategy; return *this; }
UpdateStrategy updateStrategy() const { return _updateStrategy; }
bool operator == (const Config &) const;
@@ -84,6 +85,7 @@ public:
DataStoreStorageStats getStorageStats() const override;
vespalib::MemoryUsage getMemoryUsage() const override;
std::vector<DataStoreFileChunkStats> getFileChunkStats() const override;
+ size_t getCacheCapacity() const;
/**
* Implements common::ICompactableLidSpace
diff --git a/searchlib/src/vespa/searchlib/docstore/filechunk.cpp b/searchlib/src/vespa/searchlib/docstore/filechunk.cpp
index 71dfed86fdb..c57650bb16f 100644
--- a/searchlib/src/vespa/searchlib/docstore/filechunk.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/filechunk.cpp
@@ -1,4 +1,4 @@
-// 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 "filechunk.h"
#include "data_store_file_chunk_stats.h"
@@ -6,14 +6,12 @@
#include "randreaders.h"
#include <vespa/searchlib/util/filekit.h>
#include <vespa/vespalib/util/lambdatask.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/vespalib/data/fileheader.h>
#include <vespa/vespalib/data/databuffer.h>
#include <vespa/vespalib/stllike/asciistream.h>
#include <vespa/vespalib/objects/nbostream.h>
#include <vespa/vespalib/util/executor.h>
#include <vespa/vespalib/util/arrayqueue.hpp>
-#include <vespa/vespalib/util/array.hpp>
#include <vespa/fastos/file.h>
#include <filesystem>
#include <future>
@@ -117,33 +115,14 @@ FileChunk::addNumBuckets(size_t numBucketsInChunk)
}
}
-class TmpChunkMeta : public ChunkMeta,
- public std::vector<LidMeta>
-{
-public:
- void fill(vespalib::nbostream & is) {
- resize(getNumEntries());
- for (LidMeta & lm : *this) {
- lm.deserialize(is);
- }
- }
-};
-
-using TmpChunkMetaV = std::vector<TmpChunkMeta>;
-
-namespace {
-
void
-verifyOrAssert(const TmpChunkMetaV & v)
-{
- for (auto prev(v.begin()), it(prev); it != v.end(); ++it) {
- assert(prev->getLastSerial() <= it->getLastSerial());
- prev = it;
+FileChunk::TmpChunkMeta::fill(vespalib::nbostream & is) {
+ resize(getNumEntries());
+ for (LidMeta & lm : *this) {
+ lm.deserialize(is);
}
}
-}
-
void
FileChunk::erase()
{
@@ -152,98 +131,92 @@ FileChunk::erase()
std::filesystem::remove(std::filesystem::path(_dataFileName));
}
-size_t
+void
FileChunk::updateLidMap(const unique_lock &guard, ISetLid &ds, uint64_t serialNum, uint32_t docIdLimit)
{
- size_t sz(0);
assert(_chunkInfo.empty());
FastOS_File idxFile(_idxFileName.c_str());
idxFile.enableMemoryMap(0);
- if (idxFile.OpenReadOnly()) {
- if (idxFile.IsMemoryMapped()) {
- const int64_t fileSize = idxFile.getSize();
- if (_idxHeaderLen == 0) {
- _idxHeaderLen = readIdxHeader(idxFile, _docIdLimit);
+ if ( ! idxFile.OpenReadOnly()) {
+ LOG_ABORT("should not reach here");
+ }
+ if ( ! idxFile.IsMemoryMapped()) {
+ assert(idxFile.getSize() == 0);
+ return;
+ }
+ const int64_t fileSize = idxFile.getSize();
+ if (_idxHeaderLen == 0) {
+ _idxHeaderLen = readIdxHeader(idxFile, _docIdLimit);
+ }
+ BucketDensityComputer globalBucketMap(_bucketizer);
+ // Guard comes from the same bucketizer so the same guard can be used
+ // for both local and global BucketDensityComputer
+ vespalib::GenerationHandler::Guard bucketizerGuard = globalBucketMap.getGuard();
+ vespalib::nbostream is(static_cast<const char *>(idxFile.MemoryMapPtr(0)) + _idxHeaderLen,
+ fileSize - _idxHeaderLen);
+ for (size_t count=0; ! is.empty() && is.good(); count++) {
+ const int64_t lastKnownGoodPos = _idxHeaderLen + is.rp();
+ TmpChunkMeta chunkMeta;
+ try {
+ chunkMeta.deserialize(is);
+ chunkMeta.fill(is);
+ if ((count == 0) && (chunkMeta.getLastSerial() < serialNum)) {
+ LOG(warning, "last serial num(%" PRIu64 ") from previous file is bigger than my first(%" PRIu64
+ "). That is odd.Current filename is '%s'",
+ serialNum, chunkMeta.getLastSerial(), _idxFileName.c_str());
+ serialNum = chunkMeta.getLastSerial();
}
- vespalib::nbostream is(static_cast<const char *>(idxFile.MemoryMapPtr(0)) + _idxHeaderLen,
- fileSize - _idxHeaderLen);
- TmpChunkMetaV tempVector;
- tempVector.reserve(fileSize/(sizeof(ChunkMeta)+sizeof(LidMeta)));
- while ( ! is.empty() && is.good()) {
- const int64_t lastKnownGoodPos = _idxHeaderLen + is.rp();
- tempVector.emplace_back();
- TmpChunkMeta & chunkMeta(tempVector.back());
- try {
- chunkMeta.deserialize(is);
- chunkMeta.fill(is);
- } catch (const vespalib::IllegalStateException & e) {
- LOG(warning, "Exception deserializing idx file : %s", e.what());
- LOG(warning, "File '%s' seems to be partially truncated. Will truncate from size=%" PRId64 " to %" PRId64,
- _idxFileName.c_str(), fileSize, lastKnownGoodPos);
- FastOS_File toTruncate(_idxFileName.c_str());
- if ( toTruncate.OpenReadWrite()) {
- if (toTruncate.SetSize(lastKnownGoodPos)) {
- tempVector.resize(tempVector.size() - 1);
- } else {
- throw SummaryException("SetSize() failed.", toTruncate, VESPA_STRLOC);
- }
- } else {
- throw SummaryException("Open for truncation failed.", toTruncate, VESPA_STRLOC);
- }
- break;
+ assert(serialNum <= chunkMeta.getLastSerial());
+ serialNum = handleChunk(guard, ds, docIdLimit, bucketizerGuard, globalBucketMap, chunkMeta);
+ assert(serialNum >= _lastPersistedSerialNum.load(std::memory_order_relaxed));
+ _lastPersistedSerialNum.store(serialNum, std::memory_order_relaxed);
+ } catch (const vespalib::IllegalStateException & e) {
+ LOG(warning, "Exception deserializing idx file : %s", e.what());
+ LOG(warning, "File '%s' seems to be partially truncated. Will truncate from size=%" PRId64 " to %" PRId64,
+ _idxFileName.c_str(), fileSize, lastKnownGoodPos);
+ FastOS_File toTruncate(_idxFileName.c_str());
+ if ( toTruncate.OpenReadWrite()) {
+ if (toTruncate.SetSize(lastKnownGoodPos)) {
+ } else {
+ throw SummaryException("SetSize() failed.", toTruncate, VESPA_STRLOC);
}
+ } else {
+ throw SummaryException("Open for truncation failed.", toTruncate, VESPA_STRLOC);
}
- if ( ! tempVector.empty()) {
- verifyOrAssert(tempVector);
- if (tempVector[0].getLastSerial() < serialNum) {
- LOG(warning,
- "last serial num(%" PRIu64 ") from previous file is "
- "bigger than my first(%" PRIu64 "). That is odd."
- "Current filename is '%s'",
- serialNum, tempVector[0].getLastSerial(),
- _idxFileName.c_str());
- serialNum = tempVector[0].getLastSerial();
- }
- BucketDensityComputer globalBucketMap(_bucketizer);
- // Guard comes from the same bucketizer so the same guard can be used
- // for both local and global BucketDensityComputer
- vespalib::GenerationHandler::Guard bucketizerGuard = globalBucketMap.getGuard();
- for (const TmpChunkMeta & chunkMeta : tempVector) {
- assert(serialNum <= chunkMeta.getLastSerial());
- BucketDensityComputer bucketMap(_bucketizer);
- for (size_t i(0), m(chunkMeta.getNumEntries()); i < m; i++) {
- const LidMeta & lidMeta(chunkMeta[i]);
- if (lidMeta.getLid() < docIdLimit) {
- if (_bucketizer && (lidMeta.size() > 0)) {
- document::BucketId bucketId = _bucketizer->getBucketOf(bucketizerGuard, lidMeta.getLid());
- bucketMap.recordLid(bucketId);
- globalBucketMap.recordLid(bucketId);
- }
- ds.setLid(guard, lidMeta.getLid(), LidInfo(getFileId().getId(), _chunkInfo.size(), lidMeta.size()));
- _numLids++;
- } else {
- remove(lidMeta.getLid(), lidMeta.size());
- }
- _addedBytes += adjustSize(lidMeta.size());
- }
- serialNum = chunkMeta.getLastSerial();
- addNumBuckets(bucketMap.getNumBuckets());
- _chunkInfo.emplace_back(chunkMeta.getOffset(), chunkMeta.getSize(), chunkMeta.getLastSerial());
- assert(serialNum >= _lastPersistedSerialNum.load(std::memory_order_relaxed));
- _lastPersistedSerialNum.store(serialNum, std::memory_order_relaxed);
- }
- _numUniqueBuckets = globalBucketMap.getNumBuckets();
+ break;
+ }
+ }
+ _numUniqueBuckets = globalBucketMap.getNumBuckets();
+}
+
+uint64_t
+FileChunk::handleChunk(const unique_lock &guard, ISetLid &ds, uint32_t docIdLimit,
+ const vespalib::GenerationHandler::Guard & bucketizerGuard, BucketDensityComputer &globalBucketMap,
+ const TmpChunkMeta & chunkMeta) {
+ BucketDensityComputer bucketMap(_bucketizer);
+ for (size_t i(0), m(chunkMeta.getNumEntries()); i < m; i++) {
+ const LidMeta & lidMeta(chunkMeta[i]);
+ if (lidMeta.getLid() < docIdLimit) {
+ if (_bucketizer && (lidMeta.size() > 0)) {
+ document::BucketId bucketId = _bucketizer->getBucketOf(bucketizerGuard, lidMeta.getLid());
+ bucketMap.recordLid(bucketId);
+ globalBucketMap.recordLid(bucketId);
}
+ ds.setLid(guard, lidMeta.getLid(), LidInfo(getFileId().getId(), _chunkInfo.size(), lidMeta.size()));
+ _numLids++;
} else {
- assert(idxFile.getSize() == 0);
+ remove(lidMeta.getLid(), lidMeta.size());
}
- } else {
- LOG_ABORT("should not reach here");
+ _addedBytes.store(getAddedBytes() + adjustSize(lidMeta.size()), std::memory_order_relaxed);
}
- return sz;
+ uint64_t serialNum = chunkMeta.getLastSerial();
+ addNumBuckets(bucketMap.getNumBuckets());
+ _chunkInfo.emplace_back(chunkMeta.getOffset(), chunkMeta.getSize(), chunkMeta.getLastSerial());
+ return serialNum;
}
+
void
FileChunk::enableRead()
{
@@ -277,8 +250,8 @@ void
FileChunk::remove(uint32_t lid, uint32_t size)
{
(void) lid;
- _erasedCount++;
- _erasedBytes += adjustSize(size);
+ _erasedCount.store(getErasedCount() + 1, std::memory_order_relaxed);
+ _erasedBytes.store(getErasedBytes() + adjustSize(size), std::memory_order_relaxed);
}
uint64_t
@@ -312,9 +285,9 @@ appendChunks(FixedParams * args, Chunk::UP chunk)
if (args->db.getLid(args->lidReadGuard, e.getLid()) == lidInfo) {
auto guard(args->db.getLidGuard(e.getLid()));
if (args->db.getLid(args->lidReadGuard, e.getLid()) == lidInfo) {
- // I am still in use so I need to taken care of.
+ // I am still in use, so I need to be taken care of.
vespalib::ConstBufferRef data(chunk->getLid(e.getLid()));
- args->dest.write(std::move(guard), chunk->getId(), e.getLid(), data.c_str(), data.size());
+ args->dest.write(std::move(guard), chunk->getId(), e.getLid(), data);
}
}
}
@@ -478,7 +451,7 @@ FileChunk::verify(bool reportOnly) const
(void) reportOnly;
LOG(info,
"Verifying file '%s' with fileid '%u'. erased-count='%zu' and erased-bytes='%zu'. diskFootprint='%zu'",
- _name.c_str(), _fileId.getId(), _erasedCount, _erasedBytes, _diskFootprint.load(std::memory_order_relaxed));
+ _name.c_str(), _fileId.getId(), getErasedCount(), getErasedBytes(), _diskFootprint.load(std::memory_order_relaxed));
uint64_t lastSerial(0);
size_t chunkId(0);
bool errorInPrev(false);
@@ -581,8 +554,7 @@ FileChunk::getStats() const
uint64_t serialNum = getLastPersistedSerialNum();
uint32_t docIdLimit = getDocIdLimit();
uint64_t nameId = getNameId().getId();
- return DataStoreFileChunkStats(diskFootprint, diskBloat, bucketSpread,
- serialNum, serialNum, docIdLimit, nameId);
+ return {diskFootprint, diskBloat, bucketSpread, serialNum, serialNum, docIdLimit, nameId};
}
} // namespace search
diff --git a/searchlib/src/vespa/searchlib/docstore/filechunk.h b/searchlib/src/vespa/searchlib/docstore/filechunk.h
index 3664da3dfd9..1668d030141 100644
--- a/searchlib/src/vespa/searchlib/docstore/filechunk.h
+++ b/searchlib/src/vespa/searchlib/docstore/filechunk.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
@@ -28,12 +28,12 @@ class DataStoreFileChunkStats;
class IWriteData
{
public:
- using UP = std::unique_ptr<IWriteData>;
using LockGuard = std::unique_lock<std::mutex>;
+ using ConstBufferRef = vespalib::ConstBufferRef;
virtual ~IWriteData() = default;
- virtual void write(LockGuard guard, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz) = 0;
+ virtual void write(LockGuard guard, uint32_t chunkId, uint32_t lid, ConstBufferRef data) = 0;
virtual void close() = 0;
};
@@ -47,7 +47,7 @@ public:
class BucketDensityComputer
{
public:
- BucketDensityComputer(const IBucketizer * bucketizer) : _bucketizer(bucketizer), _count(0) { }
+ explicit BucketDensityComputer(const IBucketizer * bucketizer) : _bucketizer(bucketizer), _count(0) { }
void recordLid(const vespalib::GenerationHandler::Guard & guard, uint32_t lid, uint32_t dataSize) {
if (_bucketizer && (dataSize > 0)) {
recordLid(_bucketizer->getBucketOf(guard, lid));
@@ -109,7 +109,7 @@ public:
const IBucketizer *bucketizer);
virtual ~FileChunk();
- virtual size_t updateLidMap(const unique_lock &guard, ISetLid &lidMap, uint64_t serialNum, uint32_t docIdLimit);
+ virtual void updateLidMap(const unique_lock &guard, ISetLid &lidMap, uint64_t serialNum, uint32_t docIdLimit);
virtual ssize_t read(uint32_t lid, SubChunkId chunk, vespalib::DataBuffer & buffer) const;
virtual void read(LidInfoWithLidV::const_iterator begin, size_t count, IBufferVisitor & visitor) const;
void remove(uint32_t lid, uint32_t size);
@@ -118,11 +118,12 @@ public:
virtual size_t getMemoryMetaFootprint() const;
virtual vespalib::MemoryUsage getMemoryUsage() const;
- virtual size_t getDiskHeaderFootprint(void) const { return _dataHeaderLen + _idxHeaderLen; }
+ virtual size_t getDiskHeaderFootprint() const { return _dataHeaderLen + _idxHeaderLen; }
size_t getDiskBloat() const {
- return (_addedBytes == 0)
+ size_t addedBytes = getAddedBytes();
+ return (addedBytes == 0)
? getDiskFootprint()
- : size_t(getDiskFootprint() * double(_erasedBytes)/_addedBytes);
+ : size_t(getDiskFootprint() * double(getErasedBytes())/addedBytes);
}
/**
* Get a metric for unorder of data in the file relative to when
@@ -154,9 +155,9 @@ public:
FileId getFileId() const { return _fileId; }
NameId getNameId() const { return _nameId; }
uint32_t getNumLids() const { return _numLids; }
- size_t getBloatCount() const { return _erasedCount; }
- size_t getAddedBytes() const { return _addedBytes; }
- size_t getErasedBytes() const { return _erasedBytes; }
+ size_t getErasedCount() const { return _erasedCount.load(std::memory_order_relaxed); }
+ size_t getAddedBytes() const { return _addedBytes.load(std::memory_order_relaxed); }
+ size_t getErasedBytes() const { return _erasedBytes.load(std::memory_order_relaxed); }
uint64_t getLastPersistedSerialNum() const;
uint32_t getDocIdLimit() const { return _docIdLimit; }
virtual vespalib::system_time getModificationTime() const;
@@ -199,12 +200,22 @@ public:
static vespalib::string createIdxFileName(const vespalib::string & name);
static vespalib::string createDatFileName(const vespalib::string & name);
private:
+ class TmpChunkMeta : public ChunkMeta,
+ public std::vector<LidMeta>
+ {
+ public:
+ void fill(vespalib::nbostream & is);
+ };
+ using BucketizerGuard = vespalib::GenerationHandler::Guard;
+ uint64_t handleChunk(const unique_lock &guard, ISetLid &lidMap, uint32_t docIdLimit,
+ const BucketizerGuard & bucketizerGuard, BucketDensityComputer & global,
+ const TmpChunkMeta & chunkMeta);
using File = std::unique_ptr<FileRandRead>;
const FileId _fileId;
const NameId _nameId;
const vespalib::string _name;
- size_t _erasedCount;
- size_t _erasedBytes;
+ std::atomic<size_t> _erasedCount;
+ std::atomic<size_t> _erasedBytes;
std::atomic<size_t> _diskFootprint;
size_t _sumNumBuckets;
size_t _numChunksWithBuckets;
@@ -237,17 +248,17 @@ protected:
static void writeDocIdLimit(vespalib::GenericHeader &header, uint32_t docIdLimit);
using ChunkInfoVector = std::vector<ChunkInfo, vespalib::allocator_large<ChunkInfo>>;
- const IBucketizer * _bucketizer;
- size_t _addedBytes;
- TuneFileSummary _tune;
- vespalib::string _dataFileName;
- vespalib::string _idxFileName;
- ChunkInfoVector _chunkInfo;
- std::atomic<uint64_t> _lastPersistedSerialNum;
- uint32_t _dataHeaderLen;
- uint32_t _idxHeaderLen;
- uint32_t _numLids;
- uint32_t _docIdLimit; // Limit when the file was created. Stored in idx file header.
+ const IBucketizer * _bucketizer;
+ std::atomic<size_t> _addedBytes;
+ TuneFileSummary _tune;
+ vespalib::string _dataFileName;
+ vespalib::string _idxFileName;
+ ChunkInfoVector _chunkInfo;
+ std::atomic<uint64_t> _lastPersistedSerialNum;
+ uint32_t _dataHeaderLen;
+ uint32_t _idxHeaderLen;
+ uint32_t _numLids;
+ uint32_t _docIdLimit; // Limit when the file was created. Stored in idx file header.
vespalib::system_time _modificationTime;
};
diff --git a/searchlib/src/vespa/searchlib/docstore/ibucketizer.h b/searchlib/src/vespa/searchlib/docstore/ibucketizer.h
index b7b55974978..7c3e6c10e9d 100644
--- a/searchlib/src/vespa/searchlib/docstore/ibucketizer.h
+++ b/searchlib/src/vespa/searchlib/docstore/ibucketizer.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
@@ -12,14 +12,14 @@ class IBucketizer
{
public:
using SP = std::shared_ptr<IBucketizer>;
- virtual ~IBucketizer() { }
+ virtual ~IBucketizer() = default;
virtual document::BucketId getBucketOf(const vespalib::GenerationHandler::Guard & guard, uint32_t lid) const = 0;
virtual vespalib::GenerationHandler::Guard getGuard() const = 0;
};
class IBufferVisitor {
public:
- virtual ~IBufferVisitor() { }
+ virtual ~IBufferVisitor() = default;
virtual void visit(uint32_t lid, vespalib::ConstBufferRef buffer) = 0;
};
diff --git a/searchlib/src/vespa/searchlib/docstore/idatastore.cpp b/searchlib/src/vespa/searchlib/docstore/idatastore.cpp
index 86800322516..e14e56703a9 100644
--- a/searchlib/src/vespa/searchlib/docstore/idatastore.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/idatastore.cpp
@@ -1,4 +1,4 @@
-// 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 "idatastore.h"
diff --git a/searchlib/src/vespa/searchlib/docstore/idatastore.h b/searchlib/src/vespa/searchlib/docstore/idatastore.h
index 863ee3e1268..208797d7cb1 100644
--- a/searchlib/src/vespa/searchlib/docstore/idatastore.h
+++ b/searchlib/src/vespa/searchlib/docstore/idatastore.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/idocumentstore.cpp b/searchlib/src/vespa/searchlib/docstore/idocumentstore.cpp
index 4f9b91f3e15..f0c849adcdd 100644
--- a/searchlib/src/vespa/searchlib/docstore/idocumentstore.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/idocumentstore.cpp
@@ -1,4 +1,4 @@
-// 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 "idocumentstore.h"
#include <vespa/document/fieldvalue/document.h>
diff --git a/searchlib/src/vespa/searchlib/docstore/idocumentstore.h b/searchlib/src/vespa/searchlib/docstore/idocumentstore.h
index f99302538e6..bdb815d6acf 100644
--- a/searchlib/src/vespa/searchlib/docstore/idocumentstore.h
+++ b/searchlib/src/vespa/searchlib/docstore/idocumentstore.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/lid_info.cpp b/searchlib/src/vespa/searchlib/docstore/lid_info.cpp
index cb5c9499ecd..64ea1062de6 100644
--- a/searchlib/src/vespa/searchlib/docstore/lid_info.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/lid_info.cpp
@@ -1,4 +1,4 @@
-// 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 "lid_info.h"
#include <vespa/vespalib/util/stringfmt.h>
diff --git a/searchlib/src/vespa/searchlib/docstore/lid_info.h b/searchlib/src/vespa/searchlib/docstore/lid_info.h
index 72da441bbee..68541e15509 100644
--- a/searchlib/src/vespa/searchlib/docstore/lid_info.h
+++ b/searchlib/src/vespa/searchlib/docstore/lid_info.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/liddatastore.h b/searchlib/src/vespa/searchlib/docstore/liddatastore.h
index e431f33af05..b87987c95c2 100644
--- a/searchlib/src/vespa/searchlib/docstore/liddatastore.h
+++ b/searchlib/src/vespa/searchlib/docstore/liddatastore.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/logdatastore.cpp b/searchlib/src/vespa/searchlib/docstore/logdatastore.cpp
index a187e690158..45028e124f1 100644
--- a/searchlib/src/vespa/searchlib/docstore/logdatastore.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/logdatastore.cpp
@@ -1,4 +1,4 @@
-// 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 "logdatastore.h"
#include "storebybucket.h"
@@ -22,8 +22,8 @@ namespace fs = std::filesystem;
namespace search {
namespace {
- constexpr size_t DEFAULT_MAX_FILESIZE = 1000000000ul;
- constexpr uint32_t DEFAULT_MAX_LIDS_PER_FILE = 32_Mi;
+ constexpr size_t DEFAULT_MAX_FILESIZE = 256_Mi;
+ constexpr uint32_t DEFAULT_MAX_LIDS_PER_FILE = 1_Mi;
}
using common::FileHeaderContext;
@@ -187,22 +187,22 @@ LogDataStore::write(uint64_t serialNum, uint32_t lid, const void * buffer, size_
{
std::unique_lock guard(_updateLock);
WriteableFileChunk & active = getActive(guard);
- write(std::move(guard), active, serialNum, lid, buffer, len, CpuCategory::WRITE);
+ write(std::move(guard), active, serialNum, lid, {buffer, len}, CpuCategory::WRITE);
}
void
-LogDataStore::write(MonitorGuard guard, FileId destinationFileId, uint32_t lid, const void * buffer, size_t len)
+LogDataStore::write(MonitorGuard guard, FileId destinationFileId, uint32_t lid, ConstBufferRef data)
{
auto & destination = static_cast<WriteableFileChunk &>(*_fileChunks[destinationFileId.getId()]);
- write(std::move(guard), destination, destination.getSerialNum(), lid, buffer, len, CpuCategory::COMPACT);
+ write(std::move(guard), destination, destination.getSerialNum(), lid, data, CpuCategory::COMPACT);
}
void
LogDataStore::write(MonitorGuard guard, WriteableFileChunk & destination,
- uint64_t serialNum, uint32_t lid, const void * buffer, size_t len,
+ uint64_t serialNum, uint32_t lid, ConstBufferRef data,
CpuUsage::Category cpu_category)
{
- LidInfo lm = destination.append(serialNum, lid, buffer, len, cpu_category);
+ LidInfo lm = destination.append(serialNum, lid, data, cpu_category);
setLid(guard, lid, lm);
if (destination.getFileId() == getActiveFileId(guard)) {
requireSpace(std::move(guard), destination, cpu_category);
@@ -263,7 +263,7 @@ vespalib::system_time
LogDataStore::getLastFlushTime() const
{
if (lastSyncToken() == 0) {
- return vespalib::system_time();
+ return {};
}
MonitorGuard guard(_updateLock);
vespalib::system_time timeStamp(getActive(guard).getModificationTime());
@@ -286,7 +286,7 @@ LogDataStore::remove(uint64_t serialNum, uint32_t lid)
if (lm.valid()) {
_fileChunks[lm.getFileId()]->remove(lid, lm.size());
}
- lm = getActive(guard).append(serialNum, lid, nullptr, 0, CpuCategory::WRITE);
+ lm = getActive(guard).append(serialNum, lid, {}, CpuCategory::WRITE);
assert( lm.empty() );
vespalib::atomic::store_ref_release(_lidInfo[lid], lm);
}
@@ -450,20 +450,24 @@ void LogDataStore::compactFile(FileId fileId)
NameId compactedNameId = fc->getNameId();
LOG(info, "Compacting file '%s' which has bloat '%2.2f' and bucket-spread '%1.4f",
fc->getName().c_str(), 100*fc->getDiskBloat()/double(fc->getDiskFootprint()), fc->getBucketSpread());
- IWriteData::UP compacter;
+ std::unique_ptr<IWriteData> compacter;
FileId destinationFileId = FileId::active();
if (_bucketizer) {
- size_t disk_footprint = fc->getDiskFootprint();
- size_t disk_bloat = fc->getDiskBloat();
- size_t compacted_size = (disk_footprint <= disk_bloat) ? 0u : (disk_footprint - disk_bloat);
+ size_t compacted_size;
+ {
+ MonitorGuard guard(_updateLock);
+ size_t disk_footprint = fc->getDiskFootprint();
+ size_t disk_bloat = fc->getDiskBloat();
+ compacted_size = (disk_footprint <= disk_bloat) ? 0u : (disk_footprint - disk_bloat);
+ }
if ( ! shouldCompactToActiveFile(compacted_size)) {
MonitorGuard guard(_updateLock);
destinationFileId = allocateFileId(guard);
setNewFileChunk(guard, createWritableFile(destinationFileId, fc->getLastPersistedSerialNum(), fc->getNameId().next()));
}
size_t numSignificantBucketBits = computeNumberOfSignificantBucketIdBits(*_bucketizer, fc->getFileId());
- compacter = std::make_unique<BucketCompacter>(numSignificantBucketBits, _config.compactCompression(), *this, _executor,
- *_bucketizer, fc->getFileId(), destinationFileId);
+ compacter = std::make_unique<BucketCompacter>(numSignificantBucketBits, _config.compactCompression(), *this,
+ _executor, *_bucketizer, fc->getFileId(), destinationFileId);
} else {
compacter = std::make_unique<docstore::Compacter>(*this);
}
@@ -645,7 +649,7 @@ LogDataStore::createWritableFile(FileId fileId, SerialNum serialNum, NameId name
if (fc && (fc->getNameId() == nameId)) {
LOG(error, "We already have a file registered with internal fileId=%u, and external nameId=%" PRIu64,
fileId.getId(), nameId.getId());
- return FileChunk::UP();
+ return {};
}
}
uint32_t docIdLimit = (getDocIdLimit() != 0) ? getDocIdLimit() : std::numeric_limits<uint32_t>::max();
@@ -721,7 +725,7 @@ LogDataStore::verifyModificationTime(const NameIdSet & partList)
vespalib::string datName(createDatFileName(nameId));
vespalib::string idxName(createIdxFileName(nameId));
vespalib::file_time prevDatTime = fs::last_write_time(fs::path(datName));
- vespalib::file_time prevIdxTime = fs::last_write_time(fs::path(idxName));;
+ vespalib::file_time prevIdxTime = fs::last_write_time(fs::path(idxName));
for (auto it(++partList.begin()), mt(partList.end()); it != mt; ++it) {
vespalib::string prevDatNam(datName);
vespalib::string prevIdxNam(idxName);
@@ -729,7 +733,7 @@ LogDataStore::verifyModificationTime(const NameIdSet & partList)
datName = createDatFileName(nameId);
idxName = createIdxFileName(nameId);
vespalib::file_time datTime = fs::last_write_time(fs::path(datName));
- vespalib::file_time idxTime = fs::last_write_time(fs::path(idxName));;
+ vespalib::file_time idxTime = fs::last_write_time(fs::path(idxName));
ns_log::Logger::LogLevel logLevel = ns_log::Logger::debug;
if ((datTime < prevDatTime) && hasNonHeaderData(datName)) {
VLOG(logLevel, "Older file '%s' is newer (%s) than file '%s' (%s)\nDirectory =\n%s",
@@ -992,10 +996,10 @@ class LogDataStore::WrapVisitor : public IWriteData
IDataStoreVisitor &_visitor;
public:
- void write(MonitorGuard guard, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz) override {
+ void write(MonitorGuard guard, uint32_t chunkId, uint32_t lid, ConstBufferRef data) override {
(void) chunkId;
guard.unlock();
- _visitor.visit(lid, buffer, sz);
+ _visitor.visit(lid, data.c_str(), data.size());
}
WrapVisitor(IDataStoreVisitor &visitor) : _visitor(visitor) { }
diff --git a/searchlib/src/vespa/searchlib/docstore/logdatastore.h b/searchlib/src/vespa/searchlib/docstore/logdatastore.h
index 877446e510e..97050d72cd7 100644
--- a/searchlib/src/vespa/searchlib/docstore/logdatastore.h
+++ b/searchlib/src/vespa/searchlib/docstore/logdatastore.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
@@ -71,6 +71,7 @@ public:
WriteableFileChunk::Config _fileConfig;
};
public:
+ using ConstBufferRef = vespalib::ConstBufferRef;
/**
* Construct a log based data store.
* All files are stored in base directory.
@@ -117,8 +118,8 @@ public:
Config & getConfig() { return _config; }
void write(MonitorGuard guard, WriteableFileChunk & destination, uint64_t serialNum, uint32_t lid,
- const void * buffer, size_t len, vespalib::CpuUsage::Category cpu_category);
- void write(MonitorGuard guard, FileId destinationFileId, uint32_t lid, const void * buffer, size_t len);
+ ConstBufferRef data, vespalib::CpuUsage::Category cpu_category);
+ void write(MonitorGuard guard, FileId destinationFileId, uint32_t lid, ConstBufferRef data);
/**
* This will spinn through the data and verify the content of both
@@ -155,7 +156,7 @@ public:
if (lid < getDocIdLimit()) {
return vespalib::atomic::load_ref_acquire(_lidInfo.acquire_elem_ref(lid));
} else {
- return LidInfo();
+ return {};
}
}
FileId getActiveFileId(const MonitorGuard & guard) const;
diff --git a/searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp b/searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp
index d18726dbecb..e3f1f9e4b34 100644
--- a/searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/logdocumentstore.cpp
@@ -1,4 +1,4 @@
-// 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 "logdocumentstore.h"
diff --git a/searchlib/src/vespa/searchlib/docstore/logdocumentstore.h b/searchlib/src/vespa/searchlib/docstore/logdocumentstore.h
index f2b8130fa95..c53cdea0eb2 100644
--- a/searchlib/src/vespa/searchlib/docstore/logdocumentstore.h
+++ b/searchlib/src/vespa/searchlib/docstore/logdocumentstore.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/randread.h b/searchlib/src/vespa/searchlib/docstore/randread.h
index 6f47e625e48..dee229114d1 100644
--- a/searchlib/src/vespa/searchlib/docstore/randread.h
+++ b/searchlib/src/vespa/searchlib/docstore/randread.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/randreaders.cpp b/searchlib/src/vespa/searchlib/docstore/randreaders.cpp
index 337ae1f95d1..b68d27e731d 100644
--- a/searchlib/src/vespa/searchlib/docstore/randreaders.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/randreaders.cpp
@@ -1,4 +1,4 @@
-// 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 "randreaders.h"
#include "summaryexceptions.h"
diff --git a/searchlib/src/vespa/searchlib/docstore/randreaders.h b/searchlib/src/vespa/searchlib/docstore/randreaders.h
index d7fe655dc7a..69895ba7e01 100644
--- a/searchlib/src/vespa/searchlib/docstore/randreaders.h
+++ b/searchlib/src/vespa/searchlib/docstore/randreaders.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/storebybucket.cpp b/searchlib/src/vespa/searchlib/docstore/storebybucket.cpp
index 6d3c39a51dc..f26afc595f3 100644
--- a/searchlib/src/vespa/searchlib/docstore/storebybucket.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/storebybucket.cpp
@@ -1,4 +1,4 @@
-// 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 "storebybucket.h"
#include <vespa/vespalib/data/databuffer.h>
@@ -13,14 +13,14 @@ using document::BucketId;
using vespalib::CpuUsage;
using vespalib::makeLambdaTask;
-StoreByBucket::StoreByBucket(MemoryDataStore & backingMemory, Executor & executor, CompressionConfig compression) noexcept
+StoreByBucket::StoreByBucket(StoreIndex & storeIndex, MemoryDataStore & backingMemory, Executor & executor, CompressionConfig compression) noexcept
: _chunkSerial(0),
_current(),
- _where(),
+ _storeIndex(storeIndex),
_backingMemory(backingMemory),
_executor(executor),
- _lock(std::make_unique<std::mutex>()),
- _cond(std::make_unique<std::condition_variable>()),
+ _lock(),
+ _cond(),
_numChunksPosted(0),
_chunks(),
_compression(compression)
@@ -31,9 +31,9 @@ StoreByBucket::StoreByBucket(MemoryDataStore & backingMemory, Executor & executo
StoreByBucket::~StoreByBucket() = default;
void
-StoreByBucket::add(BucketId bucketId, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz)
+StoreByBucket::add(BucketId bucketId, uint32_t chunkId, uint32_t lid, ConstBufferRef data)
{
- if ( ! _current->hasRoom(sz)) {
+ if ( ! _current->hasRoom(data.size())) {
Chunk::UP tmpChunk = createChunk();
_current.swap(tmpChunk);
incChunksPosted();
@@ -42,9 +42,8 @@ StoreByBucket::add(BucketId bucketId, uint32_t chunkId, uint32_t lid, const void
});
_executor.execute(CpuUsage::wrap(std::move(task), CpuUsage::Category::COMPACT));
}
- Index idx(bucketId, _current->getId(), chunkId, lid);
- _current->append(lid, buffer, sz);
- _where[bucketId.toKey()].push_back(idx);
+ _current->append(lid, data);
+ _storeIndex.store(Index(bucketId, _current->getId(), chunkId, lid));
}
Chunk::UP
@@ -54,8 +53,8 @@ StoreByBucket::createChunk()
}
size_t
-StoreByBucket::getChunkCount() const {
- std::lock_guard guard(*_lock);
+StoreByBucket::getChunkCount() const noexcept {
+ std::lock_guard guard(_lock);
return _chunks.size();
}
@@ -66,36 +65,40 @@ StoreByBucket::closeChunk(Chunk::UP chunk)
chunk->pack(1, buffer, _compression);
buffer.shrink(buffer.getDataLen());
ConstBufferRef bufferRef(_backingMemory.push_back(buffer.getData(), buffer.getDataLen()).data(), buffer.getDataLen());
- std::lock_guard guard(*_lock);
+ std::lock_guard guard(_lock);
_chunks[chunk->getId()] = bufferRef;
if (_numChunksPosted == _chunks.size()) {
- _cond->notify_one();
+ _cond.notify_one();
}
}
void
StoreByBucket::incChunksPosted() {
- std::lock_guard guard(*_lock);
+ std::lock_guard guard(_lock);
_numChunksPosted++;
}
void
StoreByBucket::waitAllProcessed() {
- std::unique_lock guard(*_lock);
+ std::unique_lock guard(_lock);
while (_numChunksPosted != _chunks.size()) {
- _cond->wait(guard);
+ _cond.wait(guard);
}
}
void
-StoreByBucket::drain(IWrite & drainer)
-{
+StoreByBucket::close() {
incChunksPosted();
auto task = makeLambdaTask([this, chunk=std::move(_current)]() mutable {
closeChunk(std::move(chunk));
});
_executor.execute(CpuUsage::wrap(std::move(task), CpuUsage::Category::COMPACT));
waitAllProcessed();
+}
+
+void
+StoreByBucket::drain(IWrite & drainer, IndexIterator & indexIterator)
+{
std::vector<Chunk::UP> chunks;
chunks.resize(_chunks.size());
for (const auto & it : _chunks) {
@@ -103,12 +106,10 @@ StoreByBucket::drain(IWrite & drainer)
chunks[it.first] = std::make_unique<Chunk>(it.first, buf.data(), buf.size());
}
_chunks.clear();
- for (auto & it : _where) {
- std::sort(it.second.begin(), it.second.end());
- for (Index idx : it.second) {
- vespalib::ConstBufferRef data(chunks[idx._id]->getLid(idx._lid));
- drainer.write(idx._bucketId, idx._chunkId, idx._lid, data.c_str(), data.size());
- }
+ while (indexIterator.has_next()) {
+ Index idx = indexIterator.next();
+ vespalib::ConstBufferRef data(chunks[idx._localChunkId]->getLid(idx._lid));
+ drainer.write(idx._bucketId, idx._chunkId, idx._lid, data);
}
}
diff --git a/searchlib/src/vespa/searchlib/docstore/storebybucket.h b/searchlib/src/vespa/searchlib/docstore/storebybucket.h
index dfe6199aa2e..ea1c6e766e0 100644
--- a/searchlib/src/vespa/searchlib/docstore/storebybucket.h
+++ b/searchlib/src/vespa/searchlib/docstore/storebybucket.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
@@ -7,7 +7,6 @@
#include <vespa/vespalib/data/memorydatastore.h>
#include <vespa/vespalib/util/executor.h>
#include <vespa/vespalib/stllike/hash_map.h>
-#include <map>
#include <condition_variable>
namespace search::docstore {
@@ -24,56 +23,59 @@ class StoreByBucket
using ConstBufferRef = vespalib::ConstBufferRef;
using CompressionConfig = vespalib::compression::CompressionConfig;
public:
- StoreByBucket(MemoryDataStore & backingMemory, Executor & executor, CompressionConfig compression) noexcept;
- //TODO Putting the below move constructor into cpp file fails for some unknown reason. Needs to be resolved.
- StoreByBucket(StoreByBucket &&) noexcept = default;
- StoreByBucket(const StoreByBucket &) = delete;
- StoreByBucket & operator=(StoreByBucket &&) noexcept = delete;
- StoreByBucket & operator = (const StoreByBucket &) = delete;
- ~StoreByBucket();
- class IWrite {
- public:
- using BucketId=document::BucketId;
- virtual ~IWrite() { }
- virtual void write(BucketId bucketId, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz) = 0;
- };
- void add(document::BucketId bucketId, uint32_t chunkId, uint32_t lid, const void *buffer, size_t sz);
- void drain(IWrite & drain);
- size_t getChunkCount() const;
- size_t getBucketCount() const { return _where.size(); }
- size_t getLidCount() const {
- size_t lidCount(0);
- for (const auto & it : _where) {
- lidCount += it.second.size();
- }
- return lidCount;
- }
-private:
- void incChunksPosted();
- void waitAllProcessed();
- Chunk::UP createChunk();
- void closeChunk(Chunk::UP chunk);
struct Index {
using BucketId=document::BucketId;
- Index(BucketId bucketId, uint32_t id, uint32_t chunkId, uint32_t entry) noexcept :
- _bucketId(bucketId), _id(id), _chunkId(chunkId), _lid(entry)
+ Index(BucketId bucketId, uint32_t localChunkId, uint32_t chunkId, uint32_t entry) noexcept :
+ _bucketId(bucketId), _localChunkId(localChunkId), _chunkId(chunkId), _lid(entry)
{ }
bool operator < (const Index & b) const noexcept {
return BucketId::bucketIdToKey(_bucketId.getRawId()) < BucketId::bucketIdToKey(b._bucketId.getRawId());
}
BucketId _bucketId;
- uint32_t _id;
+ uint32_t _localChunkId;
uint32_t _chunkId;
uint32_t _lid;
};
using IndexVector = std::vector<Index, vespalib::allocator_large<Index>>;
+ struct IWrite {
+ using BucketId=document::BucketId;
+ virtual ~IWrite() = default;
+ virtual void write(BucketId bucketId, uint32_t chunkId, uint32_t lid, ConstBufferRef data) = 0;
+ };
+ struct IndexIterator {
+ virtual ~IndexIterator() = default;
+ virtual bool has_next() noexcept = 0;
+ virtual Index next() noexcept = 0;
+ };
+ struct StoreIndex {
+ virtual ~StoreIndex() = default;
+ virtual void store(const Index & index) = 0;
+ };
+ StoreByBucket(StoreIndex & storeIndex, MemoryDataStore & backingMemory,
+ Executor & executor, CompressionConfig compression) noexcept;
+ //TODO Putting the below move constructor into cpp file fails for some unknown reason. Needs to be resolved.
+ StoreByBucket(StoreByBucket &&) noexcept = delete;
+ StoreByBucket(const StoreByBucket &) = delete;
+ StoreByBucket & operator=(StoreByBucket &&) noexcept = delete;
+ StoreByBucket & operator = (const StoreByBucket &) = delete;
+ ~StoreByBucket();
+ void add(document::BucketId bucketId, uint32_t chunkId, uint32_t lid, ConstBufferRef data);
+ void close();
+ /// close() must have been called prior to calling getBucketCount() or drain()
+ void drain(IWrite & drain, IndexIterator & iterator);
+ size_t getChunkCount() const noexcept;
+private:
+ void incChunksPosted();
+ void waitAllProcessed();
+ Chunk::UP createChunk();
+ void closeChunk(Chunk::UP chunk);
uint64_t _chunkSerial;
Chunk::UP _current;
- std::map<uint64_t, IndexVector> _where;
+ StoreIndex & _storeIndex;
MemoryDataStore & _backingMemory;
Executor & _executor;
- std::unique_ptr<std::mutex> _lock;
- std::unique_ptr<std::condition_variable> _cond;
+ mutable std::mutex _lock;
+ std::condition_variable _cond;
size_t _numChunksPosted;
vespalib::hash_map<uint64_t, ConstBufferRef> _chunks;
CompressionConfig _compression;
diff --git a/searchlib/src/vespa/searchlib/docstore/summaryexceptions.cpp b/searchlib/src/vespa/searchlib/docstore/summaryexceptions.cpp
index 175b5b7a3ce..e3e8bfe8ccb 100644
--- a/searchlib/src/vespa/searchlib/docstore/summaryexceptions.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/summaryexceptions.cpp
@@ -1,4 +1,4 @@
-// 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 "summaryexceptions.h"
#include <vespa/fastos/file.h>
diff --git a/searchlib/src/vespa/searchlib/docstore/summaryexceptions.h b/searchlib/src/vespa/searchlib/docstore/summaryexceptions.h
index c0f518d7db2..11d477d44cd 100644
--- a/searchlib/src/vespa/searchlib/docstore/summaryexceptions.h
+++ b/searchlib/src/vespa/searchlib/docstore/summaryexceptions.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/value.cpp b/searchlib/src/vespa/searchlib/docstore/value.cpp
index 8ac43f7a2de..57dc24e3614 100644
--- a/searchlib/src/vespa/searchlib/docstore/value.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/value.cpp
@@ -1,4 +1,4 @@
-// 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 "value.h"
#include <vespa/vespalib/util/compressor.h>
diff --git a/searchlib/src/vespa/searchlib/docstore/value.h b/searchlib/src/vespa/searchlib/docstore/value.h
index 9e98d4d4122..d511f2c419d 100644
--- a/searchlib/src/vespa/searchlib/docstore/value.h
+++ b/searchlib/src/vespa/searchlib/docstore/value.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/visitcache.cpp b/searchlib/src/vespa/searchlib/docstore/visitcache.cpp
index 322d0eb341b..2f9b75261f3 100644
--- a/searchlib/src/vespa/searchlib/docstore/visitcache.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/visitcache.cpp
@@ -1,4 +1,4 @@
-// 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 "visitcache.h"
#include "ibucketizer.h"
diff --git a/searchlib/src/vespa/searchlib/docstore/visitcache.h b/searchlib/src/vespa/searchlib/docstore/visitcache.h
index 2f312ec3011..55c68a549da 100644
--- a/searchlib/src/vespa/searchlib/docstore/visitcache.h
+++ b/searchlib/src/vespa/searchlib/docstore/visitcache.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
diff --git a/searchlib/src/vespa/searchlib/docstore/writeablefilechunk.cpp b/searchlib/src/vespa/searchlib/docstore/writeablefilechunk.cpp
index 7102b80d7d0..297b8f66099 100644
--- a/searchlib/src/vespa/searchlib/docstore/writeablefilechunk.cpp
+++ b/searchlib/src/vespa/searchlib/docstore/writeablefilechunk.cpp
@@ -1,4 +1,4 @@
-// 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 "writeablefilechunk.h"
#include "data_store_file_chunk_stats.h"
@@ -11,7 +11,6 @@
#include <vespa/vespalib/stllike/hash_map.hpp>
#include <vespa/vespalib/util/cpu_usage.h>
#include <vespa/vespalib/util/lambdatask.h>
-#include <vespa/vespalib/util/size_literals.h>
#include <vespa/log/log.h>
LOG_SETUP(".search.writeablefilechunk");
@@ -174,16 +173,15 @@ WriteableFileChunk::~WriteableFileChunk()
}
}
-size_t
+void
WriteableFileChunk::updateLidMap(const unique_lock &guard, ISetLid &ds, uint64_t serialNum, uint32_t docIdLimit)
{
- size_t sz = FileChunk::updateLidMap(guard, ds, serialNum, docIdLimit);
+ FileChunk::updateLidMap(guard, ds, serialNum, docIdLimit);
_nextChunkId = _chunkInfo.size();
_active = std::make_unique<Chunk>(_nextChunkId++, Chunk::Config(_config.getMaxChunkBytes()));
_serialNum = getLastPersistedSerialNum();
_firstChunkIdToBeWritten = _active->getId();
setDiskFootprint(0);
- return sz;
}
void
@@ -419,7 +417,7 @@ WriteableFileChunk::computeChunkMeta(const unique_lock & guard,
assert((size_t(tmp.getBuf().getData())%_alignment) == 0);
assert((dataLen%_alignment) == 0);
auto pcsp = std::make_shared<PendingChunk>(active.getLastSerial(), offset, dataLen);
- PendingChunk &pc(*pcsp.get());
+ PendingChunk &pc(*pcsp);
nbostream &os(pc.getSerializedIdx());
cmeta.serialize(os);
BucketDensityComputer bucketMap(_bucketizer);
@@ -723,21 +721,21 @@ WriteableFileChunk::waitForAllChunksFlushedToDisk() const
}
LidInfo
-WriteableFileChunk::append(uint64_t serialNum, uint32_t lid, const void * buffer, size_t len,
+WriteableFileChunk::append(uint64_t serialNum, uint32_t lid, vespalib::ConstBufferRef data,
CpuUsage::Category cpu_category)
{
assert( !frozen() );
- if ( ! _active->hasRoom(len)) {
+ if ( ! _active->hasRoom(data.size())) {
flush(false, _serialNum, cpu_category);
}
assert(serialNum >= _serialNum);
_serialNum = serialNum;
- _addedBytes += adjustSize(len);
+ _addedBytes.store(getAddedBytes() + adjustSize(data.size()), std::memory_order_relaxed);
_numLids++;
size_t oldSz(_active->size());
- LidMeta lm = _active->append(lid, buffer, len);
+ LidMeta lm = _active->append(lid, data);
setDiskFootprint(FileChunk::getDiskFootprint() - oldSz + _active->size());
- return LidInfo(getFileId().getId(), _active->getId(), lm.size());
+ return {getFileId().getId(), _active->getId(), lm.size()};
}
@@ -897,7 +895,7 @@ WriteableFileChunk::unconditionallyFlushPendingChunks(const unique_lock &flushGu
break;
std::shared_ptr<PendingChunk> pcsp = std::move(_pendingChunks.front());
_pendingChunks.pop_front();
- const PendingChunk &pc(*pcsp.get());
+ const PendingChunk &pc(*pcsp);
assert(_pendingIdx >= pc.getIdxLen());
assert(_pendingDat >= pc.getDataLen());
assert(datFileLen >= pc.getDataOffset() + pc.getDataLen());
@@ -933,9 +931,9 @@ WriteableFileChunk::getStats() const
{
DataStoreFileChunkStats stats = FileChunk::getStats();
uint64_t serialNum = getSerialNum();
- return DataStoreFileChunkStats(stats.diskUsage(), stats.diskBloat(), stats.maxBucketSpread(),
- serialNum, stats.lastFlushedSerialNum(), stats.docIdLimit(), stats.nameId());
-};
+ return {stats.diskUsage(), stats.diskBloat(), stats.maxBucketSpread(),
+ serialNum, stats.lastFlushedSerialNum(), stats.docIdLimit(), stats.nameId()};
+}
PendingChunk::PendingChunk(uint64_t lastSerial, uint64_t dataOffset, uint32_t dataLen)
: _idx(),
diff --git a/searchlib/src/vespa/searchlib/docstore/writeablefilechunk.h b/searchlib/src/vespa/searchlib/docstore/writeablefilechunk.h
index b5a52dc83f7..49cdf6ae3ff 100644
--- a/searchlib/src/vespa/searchlib/docstore/writeablefilechunk.h
+++ b/searchlib/src/vespa/searchlib/docstore/writeablefilechunk.h
@@ -1,4 +1,4 @@
-// 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.
#pragma once
@@ -52,7 +52,7 @@ public:
ssize_t read(uint32_t lid, SubChunkId chunk, vespalib::DataBuffer & buffer) const override;
void read(LidInfoWithLidV::const_iterator begin, size_t count, IBufferVisitor & visitor) const override;
- LidInfo append(uint64_t serialNum, uint32_t lid, const void * buffer, size_t len,
+ LidInfo append(uint64_t serialNum, uint32_t lid, vespalib::ConstBufferRef data,
vespalib::CpuUsage::Category cpu_category);
void flush(bool block, uint64_t syncToken, vespalib::CpuUsage::Category cpu_category);
uint64_t getSerialNum() const { return _serialNum; }
@@ -64,7 +64,7 @@ public:
size_t getMemoryFootprint() const override;
size_t getMemoryMetaFootprint() const override;
vespalib::MemoryUsage getMemoryUsage() const override;
- size_t updateLidMap(const unique_lock &guard, ISetLid &lidMap, uint64_t serialNum, uint32_t docIdLimit) override;
+ void updateLidMap(const unique_lock &guard, ISetLid &lidMap, uint64_t serialNum, uint32_t docIdLimit) override;
void waitForDiskToCatchUpToNow() const;
void flushPendingChunks(uint64_t serialNum);
DataStoreFileChunkStats getStats() const override;
@@ -79,7 +79,7 @@ private:
bool frozen() const override { return _frozen.load(std::memory_order_acquire); }
void waitForChunkFlushedToDisk(uint32_t chunkId) const;
void waitForAllChunksFlushedToDisk() const;
- void fileWriter(const uint32_t firstChunkId);
+ void fileWriter(uint32_t firstChunkId);
void internalFlush(uint32_t, uint64_t serialNum, vespalib::CpuUsage::Category cpu_category);
void enque(ProcessedChunkUP, vespalib::CpuUsage::Category cpu_category);
int32_t flushLastIfNonEmpty(bool force);