summaryrefslogtreecommitdiffstats
path: root/memfilepersistence
diff options
context:
space:
mode:
authorHaavard <havardpe@yahoo-inc.com>2017-03-20 13:04:33 +0000
committerHaavard <havardpe@yahoo-inc.com>2017-03-27 09:53:26 +0000
commit23abed1a0bc4f4c5ea47b43fc7ea0645e63a26e6 (patch)
tree6d943bbe31738f7e9b84979e4fd63dfd76eef580 /memfilepersistence
parent8844ccb7297e8a5120dd903c85e923f2f93aa693 (diff)
remove most usage of LinkedPtr from vespa
Diffstat (limited to 'memfilepersistence')
-rw-r--r--memfilepersistence/src/tests/device/devicemanagertest.cpp4
-rw-r--r--memfilepersistence/src/tests/device/devicestest.cpp10
-rw-r--r--memfilepersistence/src/tests/device/mountpointlisttest.cpp11
-rw-r--r--memfilepersistence/src/tests/init/filescannertest.cpp2
-rw-r--r--memfilepersistence/src/tests/spi/iteratorhandlertest.cpp38
-rw-r--r--memfilepersistence/src/tests/spi/simplememfileiobuffertest.cpp4
-rw-r--r--memfilepersistence/src/tests/tools/vdsdisktooltest.cpp5
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/common/environment.cpp4
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/common/types.h1
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.cpp36
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.h20
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/device/directory.h4
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/device/disk.h3
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.cpp22
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.h7
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/device/partition.h6
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/device/partitionmonitor.h3
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/init/filescanner.cpp2
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/mapper/buffer.h3
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/mapper/memfile_v1_serializer.h2
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.cpp16
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.h4
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.cpp20
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.h20
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/mapper/versionserializer.h3
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.cpp6
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.h8
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/memfile/memfileptr.h8
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/memfile/memslot.h2
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.cpp22
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.h14
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/tools/dumpslotfile.cpp2
-rw-r--r--memfilepersistence/src/vespa/memfilepersistence/tools/vdsdisktool.cpp4
33 files changed, 146 insertions, 170 deletions
diff --git a/memfilepersistence/src/tests/device/devicemanagertest.cpp b/memfilepersistence/src/tests/device/devicemanagertest.cpp
index eeb5007f452..7e26543349b 100644
--- a/memfilepersistence/src/tests/device/devicemanagertest.cpp
+++ b/memfilepersistence/src/tests/device/devicemanagertest.cpp
@@ -94,7 +94,7 @@ void DeviceManagerTest::testEventSending()
DeviceManager manager(DeviceMapper::UP(new SimpleDeviceMapper), _clock);
Listener l;
manager.addIOEventListener(l);
- Directory::LP dir(manager.getDirectory("/home/foo/var", 0));
+ Directory::SP dir(manager.getDirectory("/home/foo/var", 0));
// IO failures are disk events. Will mark all partitions and
// directories on that disk bad
dir->addEvent(IOEvent::createEventFromErrno(1, EIO, "/home/foo/var/foo"));
@@ -117,7 +117,7 @@ void DeviceManagerTest::testEventSending()
void DeviceManagerTest::testXml()
{
DeviceManager manager(DeviceMapper::UP(new SimpleDeviceMapper), _clock);
- Directory::LP dir(manager.getDirectory("/home/", 0));
+ Directory::SP dir(manager.getDirectory("/home/", 0));
dir->getPartition().initializeMonitor();
std::string xml = manager.toXml(" ");
CPPUNIT_ASSERT_MSG(xml,
diff --git a/memfilepersistence/src/tests/device/devicestest.cpp b/memfilepersistence/src/tests/device/devicestest.cpp
index bd6898cb7ac..8bcc93e5fc9 100644
--- a/memfilepersistence/src/tests/device/devicestest.cpp
+++ b/memfilepersistence/src/tests/device/devicestest.cpp
@@ -31,11 +31,11 @@ CPPUNIT_TEST_SUITE_REGISTRATION(DevicesTest);
void DevicesTest::testDisk()
{
DeviceManager manager(DeviceMapper::UP(new SimpleDeviceMapper), _clock);
- Disk::LP disk1(manager.getDisk("/something/on/disk"));
- Disk::LP disk2(manager.getDisk("/something/on/disk"));
+ Disk::SP disk1(manager.getDisk("/something/on/disk"));
+ Disk::SP disk2(manager.getDisk("/something/on/disk"));
CPPUNIT_ASSERT_EQUAL(disk1->getId(), disk2->getId());
CPPUNIT_ASSERT_EQUAL(disk1.get(), disk2.get());
- Disk::LP disk3(manager.getDisk("/something/on/disk2"));
+ Disk::SP disk3(manager.getDisk("/something/on/disk2"));
CPPUNIT_ASSERT(disk2->getId() != disk3->getId());
disk3->toString(); // Add code coverage
}
@@ -43,7 +43,7 @@ void DevicesTest::testDisk()
void DevicesTest::testPartition()
{
DeviceManager manager(DeviceMapper::UP(new SimpleDeviceMapper), _clock);
- Partition::LP part(manager.getPartition("/etc"));
+ Partition::SP part(manager.getPartition("/etc"));
CPPUNIT_ASSERT_EQUAL(std::string("/etc"), part->getMountPoint());
part->toString(); // Add code coverage
}
@@ -51,7 +51,7 @@ void DevicesTest::testPartition()
void DevicesTest::testDirectory()
{
DeviceManager manager(DeviceMapper::UP(new SimpleDeviceMapper), _clock);
- Directory::LP dir1(manager.getDirectory("/on/disk", 0));
+ Directory::SP dir1(manager.getDirectory("/on/disk", 0));
CPPUNIT_ASSERT_EQUAL(std::string("/on/disk"), dir1->getPath());
CPPUNIT_ASSERT(dir1->getLastEvent() == 0);
CPPUNIT_ASSERT_EQUAL(Device::OK, dir1->getState());
diff --git a/memfilepersistence/src/tests/device/mountpointlisttest.cpp b/memfilepersistence/src/tests/device/mountpointlisttest.cpp
index 4cb5822ceb7..aaeff981af8 100644
--- a/memfilepersistence/src/tests/device/mountpointlisttest.cpp
+++ b/memfilepersistence/src/tests/device/mountpointlisttest.cpp
@@ -7,7 +7,6 @@
#include <vespa/vespalib/io/fileutil.h>
#include <vespa/storageframework/defaultimplementation/clock/fakeclock.h>
-using vespalib::LinkedPtr;
using vespalib::fileExists;
using vespalib::isDirectory;
using vespalib::isSymLink;
@@ -37,8 +36,8 @@ public:
framework::defaultimplementation::FakeClock _clock;
private:
- LinkedPtr<DeviceManager> newDeviceManager() {
- return LinkedPtr<DeviceManager>(
+ DeviceManager::UP newDeviceManager() {
+ return DeviceManager::UP(
new DeviceManager(
DeviceMapper::UP(new SimpleDeviceMapper),
_clock));
@@ -88,7 +87,7 @@ void MountPointList_Test::testScanning()
init();
MountPointList list(_prefix,
std::vector<vespalib::string>(),
- vespalib::LinkedPtr<DeviceManager>(
+ DeviceManager::UP(
new DeviceManager(
DeviceMapper::UP(new SimpleDeviceMapper),
_clock)));
@@ -133,7 +132,7 @@ void MountPointList_Test::testStatusFile()
{
MountPointList list(_prefix,
std::vector<vespalib::string>(),
- vespalib::LinkedPtr<DeviceManager>(
+ DeviceManager::UP(
new DeviceManager(
DeviceMapper::UP(new SimpleDeviceMapper),
_clock)));
@@ -195,7 +194,7 @@ void MountPointList_Test::testStatusFile()
{
MountPointList list(_prefix,
std::vector<vespalib::string>(),
- vespalib::LinkedPtr<DeviceManager>(
+ DeviceManager::UP(
new DeviceManager(
DeviceMapper::UP(new SimpleDeviceMapper),
_clock)));
diff --git a/memfilepersistence/src/tests/init/filescannertest.cpp b/memfilepersistence/src/tests/init/filescannertest.cpp
index 8b49a21dad0..2a69804a295 100644
--- a/memfilepersistence/src/tests/init/filescannertest.cpp
+++ b/memfilepersistence/src/tests/init/filescannertest.cpp
@@ -375,7 +375,7 @@ FileScannerTest::runTest(const TestParameters& params)
MountPointList mountPoints("./vdsroot",
std::vector<vespalib::string>(),
- vespalib::LinkedPtr<DeviceManager>(
+ DeviceManager::UP(
new DeviceManager(
DeviceMapper::UP(new SimpleDeviceMapper),
clock)));
diff --git a/memfilepersistence/src/tests/spi/iteratorhandlertest.cpp b/memfilepersistence/src/tests/spi/iteratorhandlertest.cpp
index 5b622987e73..6fb2a8393be 100644
--- a/memfilepersistence/src/tests/spi/iteratorhandlertest.cpp
+++ b/memfilepersistence/src/tests/spi/iteratorhandlertest.cpp
@@ -63,7 +63,7 @@ public:
struct Chunk
{
- std::vector<spi::DocEntry::LP> _entries;
+ std::vector<spi::DocEntry::UP> _entries;
};
private:
@@ -144,18 +144,12 @@ IteratorHandlerTest::doIterate(spi::IteratorId id,
std::vector<Chunk> chunks;
while (true) {
- std::vector<spi::DocEntry::LP> entries;
-
spi::IterateResult result(getPersistenceProvider().iterate(
id, maxByteSize, context));
CPPUNIT_ASSERT_EQUAL(spi::Result::NONE, result.getErrorCode());
CPPUNIT_ASSERT(result.getEntries().size() > 0 || allowEmptyResult);
- for (size_t i = 0; i < result.getEntries().size(); ++i) {
- entries.push_back(result.getEntries()[i]);
- }
- chunks.push_back(Chunk());
- chunks.back()._entries.swap(entries);
+ chunks.push_back(Chunk{std::move(result.steal_entries())});
if (result.isCompleted()
|| (maxChunks != 0 && chunks.size() >= maxChunks))
{
@@ -178,7 +172,7 @@ getDocCount(const std::vector<IteratorHandlerTest::Chunk>& chunks)
}
size_t
-getRemoveEntryCount(const std::vector<spi::DocEntry::LP>& entries)
+getRemoveEntryCount(const std::vector<spi::DocEntry::UP>& entries)
{
size_t ret = 0;
for (size_t i = 0; i < entries.size(); ++i) {
@@ -191,20 +185,20 @@ getRemoveEntryCount(const std::vector<spi::DocEntry::LP>& entries)
struct DocEntryIndirectTimestampComparator
{
- bool operator()(const spi::DocEntry::LP& e1,
- const spi::DocEntry::LP& e2) const
+ bool operator()(const spi::DocEntry::UP& e1,
+ const spi::DocEntry::UP& e2) const
{
return e1->getTimestamp() < e2->getTimestamp();
}
};
-std::vector<spi::DocEntry::LP>
+std::vector<spi::DocEntry::UP>
getEntriesFromChunks(const std::vector<IteratorHandlerTest::Chunk>& chunks)
{
- std::vector<spi::DocEntry::LP> ret;
+ std::vector<spi::DocEntry::UP> ret;
for (size_t chunk = 0; chunk < chunks.size(); ++chunk) {
for (size_t i = 0; i < chunks[chunk]._entries.size(); ++i) {
- ret.push_back(chunks[chunk]._entries[i]);
+ ret.push_back(spi::DocEntry::UP(chunks[chunk]._entries[i]->clone()));
}
}
std::sort(ret.begin(),
@@ -233,7 +227,7 @@ IteratorHandlerTest::verifyDocs(const std::vector<DocAndTimestamp>& wanted,
const std::vector<IteratorHandlerTest::Chunk>& chunks,
const std::set<vespalib::string>& removes) const
{
- std::vector<spi::DocEntry::LP> retrieved(
+ std::vector<spi::DocEntry::UP> retrieved(
getEntriesFromChunks(chunks));
size_t removeCount = getRemoveEntryCount(retrieved);
// Ensure that we've got the correct number of puts and removes
@@ -321,9 +315,7 @@ IteratorHandlerTest::testSomeSlotsRemovedBetweenInvocations()
std::vector<Chunk> chunks2 = doIterate(iter.getIteratorId(), 10000);
CPPUNIT_ASSERT_EQUAL(size_t(24), chunks2.size());
- std::copy(chunks2.begin(),
- chunks2.end(),
- std::back_insert_iterator<std::vector<Chunk> >(chunks));
+ std::move(chunks2.begin(), chunks2.end(), std::back_inserter(chunks));
verifyDocs(docs, chunks);
@@ -393,7 +385,7 @@ IteratorHandlerTest::testIterateMetadataOnly()
create(b, sel, spi::NEWEST_DOCUMENT_OR_REMOVE, document::NoFields()));
std::vector<Chunk> chunks = doIterate(iter.getIteratorId(), 4096);
- std::vector<spi::DocEntry::LP> entries = getEntriesFromChunks(chunks);
+ std::vector<spi::DocEntry::UP> entries = getEntriesFromChunks(chunks);
CPPUNIT_ASSERT_EQUAL(docs.size(), entries.size());
std::vector<DocAndTimestamp>::const_iterator docIter(
docs.begin());
@@ -495,9 +487,7 @@ IteratorHandlerTest::testDocumentsRemovedBetweenInvocations()
std::vector<Chunk> chunks2 = doIterate(iter.getIteratorId(), 1);
CPPUNIT_ASSERT_EQUAL(size_t(75), chunks2.size());
- std::copy(chunks2.begin(),
- chunks2.end(),
- std::back_insert_iterator<std::vector<Chunk> >(chunks));
+ std::move(chunks2.begin(), chunks2.end(), std::back_inserter(chunks));
verifyDocs(docs, chunks);
@@ -539,7 +529,7 @@ IteratorHandlerTest::doTestUnrevertableRemoveBetweenInvocations(bool includeRemo
flush(b.getBucketId());
std::vector<Chunk> chunks2 = doIterate(iter.getIteratorId(), 1);
- std::vector<spi::DocEntry::LP> entries = getEntriesFromChunks(chunks2);
+ std::vector<spi::DocEntry::UP> entries = getEntriesFromChunks(chunks2);
if (!includeRemoves) {
CPPUNIT_ASSERT_EQUAL(nonRemovedDocs.size(), chunks2.size());
verifyDocs(nonRemovedDocs, chunks2);
@@ -764,7 +754,7 @@ IteratorHandlerTest::testFieldSetFiltering()
spi::CreateIteratorResult iter(
create(b, sel, spi::NEWEST_DOCUMENT_ONLY,
*repo.parse(*getTypeRepo(), "testdoctype1:hstringval,content")));
- std::vector<spi::DocEntry::LP> entries(
+ std::vector<spi::DocEntry::UP> entries(
getEntriesFromChunks(doIterate(iter.getIteratorId(), 4096)));
CPPUNIT_ASSERT_EQUAL(size_t(1), entries.size());
CPPUNIT_ASSERT_EQUAL(std::string("content: fancy content\n"
diff --git a/memfilepersistence/src/tests/spi/simplememfileiobuffertest.cpp b/memfilepersistence/src/tests/spi/simplememfileiobuffertest.cpp
index d7dc3354316..3eac7b57e81 100644
--- a/memfilepersistence/src/tests/spi/simplememfileiobuffertest.cpp
+++ b/memfilepersistence/src/tests/spi/simplememfileiobuffertest.cpp
@@ -31,13 +31,13 @@ class SimpleMemFileIOBufferTest : public SingleDiskMemFileTestUtils
CPPUNIT_TEST_SUITE_END();
using BufferType = SimpleMemFileIOBuffer::BufferType;
- using BufferLP = BufferType::LP;
+ using BufferSP = BufferType::SP;
using BufferAllocation = SimpleMemFileIOBuffer::BufferAllocation;
using HeaderChunkEncoder = SimpleMemFileIOBuffer::HeaderChunkEncoder;
using SimpleMemFileIOBufferUP = std::unique_ptr<SimpleMemFileIOBuffer>;
BufferAllocation allocateBuffer(size_t sz) {
- return BufferAllocation(BufferLP(new BufferType(sz)), 0, sz);
+ return BufferAllocation(BufferSP(new BufferType(sz)), 0, sz);
}
/**
diff --git a/memfilepersistence/src/tests/tools/vdsdisktooltest.cpp b/memfilepersistence/src/tests/tools/vdsdisktooltest.cpp
index 29e780bc900..08bfa3bcab3 100644
--- a/memfilepersistence/src/tests/tools/vdsdisktooltest.cpp
+++ b/memfilepersistence/src/tests/tools/vdsdisktooltest.cpp
@@ -14,7 +14,6 @@ namespace memfile {
struct VdsDiskToolTest : public SingleDiskMemFileTestUtils
{
framework::defaultimplementation::FakeClock _clock;
- DeviceManager::LP _deviceManager;
void setUp();
void setupRoot();
@@ -55,8 +54,6 @@ void
VdsDiskToolTest::setUp()
{
system("rm -rf vdsroot");
- _deviceManager.reset(new DeviceManager(
- DeviceMapper::UP(new SimpleDeviceMapper), _clock));
}
void
@@ -94,7 +91,7 @@ VdsDiskToolTest::testSimple()
createDisk(1);
MountPointList mountPoints("vdsroot/mycluster/storage/3",
std::vector<vespalib::string>(),
- _deviceManager);
+ std::make_unique<DeviceManager>(std::make_unique<SimpleDeviceMapper>(), _clock));
mountPoints.scanForDisks();
CPPUNIT_ASSERT_EQUAL(2u, mountPoints.getSize());
mountPoints[1].addEvent(Device::IO_FAILURE, "Bad", "Found in test");
diff --git a/memfilepersistence/src/vespa/memfilepersistence/common/environment.cpp b/memfilepersistence/src/vespa/memfilepersistence/common/environment.cpp
index ff4c91026f2..11577542573 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/common/environment.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/common/environment.cpp
@@ -50,7 +50,7 @@ Environment::Environment(const config::ConfigUri & configUri,
_devicesConfig(resolveConfig<DevicesConfig>(configUri)),
_options(std::make_shared<Options>(*_config, *_persistenceConfig))
{
- DeviceManager::LP manager(
+ DeviceManager::UP manager(
new DeviceManager(DeviceMapper::UP(new SimpleDeviceMapper()),
_clock));
@@ -58,7 +58,7 @@ Environment::Environment(const config::ConfigUri & configUri,
_devicesConfig->statfsPolicy, _devicesConfig->statfsPeriod);
_mountPoints.reset(new MountPointList(_devicesConfig->rootFolder,
_devicesConfig->diskPath,
- manager));
+ std::move(manager)));
if (!ignoreDisks) {
_mountPoints->init(0);
diff --git a/memfilepersistence/src/vespa/memfilepersistence/common/types.h b/memfilepersistence/src/vespa/memfilepersistence/common/types.h
index 283eb1153f4..58643275503 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/common/types.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/common/types.h
@@ -76,7 +76,6 @@ std::ostream& operator<<(std::ostream&, const DataLocation&);
struct Types {
typedef document::BucketId BucketId;
typedef document::Document Document;
- typedef vespalib::LinkedPtr<Document> DocLP;
typedef document::DocumentId DocumentId;
typedef document::GlobalId GlobalId;
typedef framework::MicroSecTime Timestamp;
diff --git a/memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.cpp b/memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.cpp
index d088f1dab46..9d77b84d4c6 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.cpp
@@ -28,7 +28,7 @@ DeviceManager::setPartitionMonitorPolicy(
{
_statPolicy = policy;
_statPeriod = period;
- for (std::map<std::string, Partition::LP>::iterator it
+ for (std::map<std::string, Partition::SP>::iterator it
= _partitions.begin(); it != _partitions.end(); ++it)
{
Partition& p(*it->second);
@@ -80,26 +80,26 @@ DeviceManager::removeIOEventListener(IOEventListener& listener)
_eventListeners.erase(&listener);
}
-Directory::LP
+Directory::SP
DeviceManager::getDirectory(const std::string& dir, uint16_t index)
{
- std::map<std::string, Directory::LP>::iterator it =
+ std::map<std::string, Directory::SP>::iterator it =
_directories.find(dir);
if (it != _directories.end()) {
return it->second;
}
- Directory::LP d(new Directory(*this, index, dir));
+ Directory::SP d(new Directory(*this, index, dir));
_directories[dir] = d;
return d;
}
-Directory::LP
+Directory::SP
DeviceManager::deserializeDirectory(const std::string& serialized)
{
// Deserialize object
- Directory::LP d(new Directory(serialized, *this));
+ Directory::SP d(new Directory(serialized, *this));
// If not existing, just add it.
- std::map<std::string, Directory::LP>::iterator it =
+ std::map<std::string, Directory::SP>::iterator it =
_directories.find(d->getPath());
if (it == _directories.end()) {
_directories[d->getPath()] = d;
@@ -110,18 +110,18 @@ DeviceManager::deserializeDirectory(const std::string& serialized)
return it->second;
}
-Partition::LP
+Partition::SP
DeviceManager::getPartition(const std::string& path)
{
try{
std::string mountPoint(_deviceMapper->getMountPoint(path));
uint64_t id = _deviceMapper->getPartitionId(mountPoint);
- std::map<std::string, Partition::LP>::iterator it(
+ std::map<std::string, Partition::SP>::iterator it(
_partitions.find(mountPoint));
if (it != _partitions.end()) {
return it->second;
}
- Partition::LP part(new Partition(*this, id, mountPoint));
+ Partition::SP part(new Partition(*this, id, mountPoint));
if (part->getMonitor() != 0) {
part->getMonitor()->setPolicy(_statPolicy, _statPeriod);
}
@@ -131,7 +131,7 @@ DeviceManager::getPartition(const std::string& path)
// If we fail to create partition, due to having IO troubles getting
// partition id or mount point, create a partition that doesn't
// correspond to a physical device containing the error found.
- Partition::LP part(new Partition(*this, -1, path));
+ Partition::SP part(new Partition(*this, -1, path));
part->addEvent(IOEvent::createEventFromIoException(
e,
_clock.getTimeInSeconds().getTime()));
@@ -140,16 +140,16 @@ DeviceManager::getPartition(const std::string& path)
}
}
-Disk::LP
+Disk::SP
DeviceManager::getDisk(const std::string& path)
{
try{
int devnr = _deviceMapper->getDeviceId(path);
- std::map<int, Disk::LP>::iterator it = _disks.find(devnr);
+ std::map<int, Disk::SP>::iterator it = _disks.find(devnr);
if (it != _disks.end()) {
return it->second;
}
- Disk::LP disk(new Disk(*this, devnr));
+ Disk::SP disk(new Disk(*this, devnr));
_disks[devnr] = disk;
return disk;
} catch (vespalib::IoException& e) {
@@ -160,7 +160,7 @@ DeviceManager::getDisk(const std::string& path)
// If we fail to create partition, due to having IO troubles getting
// partition id or mount point, create a partition that doesn't
// correspond to a physical device containing the error found.
- Disk::LP disk(new Disk(*this, devnr));
+ Disk::SP disk(new Disk(*this, devnr));
disk->addEvent(IOEvent::createEventFromIoException(
e,
_clock.getTimeInSeconds().getTime()));
@@ -177,11 +177,11 @@ DeviceManager::printXml(vespalib::XmlOutputStream& xos) const
xos << XmlTag("mapper") << XmlAttribute("type", _deviceMapper->getName())
<< XmlEndTag();
xos << XmlTag("devices");
- for (std::map<int, Disk::LP>::const_iterator diskIt = _disks.begin();
+ for (std::map<int, Disk::SP>::const_iterator diskIt = _disks.begin();
diskIt != _disks.end(); ++diskIt)
{
xos << XmlTag("disk") << XmlAttribute("deviceId", diskIt->first);
- for (std::map<std::string, Partition::LP>::const_iterator partIt
+ for (std::map<std::string, Partition::SP>::const_iterator partIt
= _partitions.begin(); partIt != _partitions.end(); ++partIt)
{
if (partIt->second->getDisk() != *diskIt->second) continue;
@@ -191,7 +191,7 @@ DeviceManager::printXml(vespalib::XmlOutputStream& xos) const
if (partIt->second->getMonitor() != 0) {
xos << *partIt->second->getMonitor();
}
- for (std::map<std::string, Directory::LP>::const_iterator dirIt
+ for (std::map<std::string, Directory::SP>::const_iterator dirIt
= _directories.begin(); dirIt != _directories.end();
++dirIt)
{
diff --git a/memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.h b/memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.h
index dc1c6fdd68d..d4528ad10b6 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/device/devicemanager.h
@@ -25,9 +25,9 @@ namespace memfile {
class DeviceManager : public vespalib::XmlSerializable {
DeviceMapper::UP _deviceMapper;
- std::map<int, Disk::LP> _disks;
- std::map<std::string, Partition::LP> _partitions;
- std::map<std::string, Directory::LP> _directories;
+ std::map<int, Disk::SP> _disks;
+ std::map<std::string, Partition::SP> _partitions;
+ std::map<std::string, Directory::SP> _directories;
std::set<IOEventListener*> _eventListeners;
vespa::config::storage::StorDevicesConfig::StatfsPolicy _statPolicy;
uint32_t _statPeriod;
@@ -39,7 +39,7 @@ class DeviceManager : public vespalib::XmlSerializable {
void setFindDeviceFunction();
public:
- typedef vespalib::LinkedPtr<DeviceManager> LP;
+ using UP = std::unique_ptr<DeviceManager>;
DeviceManager(DeviceMapper::UP mapper,
const framework::Clock& clock);
@@ -54,13 +54,13 @@ public:
void addIOEventListener(IOEventListener& listener);
void removeIOEventListener(IOEventListener& listener);
- Directory::LP getDirectory(const std::string& dir, uint16_t index);
- Directory::LP deserializeDirectory(const std::string& serialized);
- Partition::LP getPartition(const std::string& path);
- Disk::LP getDisk(const std::string& path);
+ Directory::SP getDirectory(const std::string& dir, uint16_t index);
+ Directory::SP deserializeDirectory(const std::string& serialized);
+ Partition::SP getPartition(const std::string& path);
+ Disk::SP getDisk(const std::string& path);
- std::vector<Directory::LP> getDirectories(const Disk& disk) const;
- std::vector<Directory::LP> getDirectories(const Partition& part) const;
+ std::vector<Directory::SP> getDirectories(const Disk& disk) const;
+ std::vector<Directory::SP> getDirectories(const Partition& part) const;
vespa::config::storage::StorDevicesConfig::StatfsPolicy getStatPolicy() const
{ return _statPolicy; }
diff --git a/memfilepersistence/src/vespa/memfilepersistence/device/directory.h b/memfilepersistence/src/vespa/memfilepersistence/device/directory.h
index 7bd2f7dcd53..b02d12a4552 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/device/directory.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/device/directory.h
@@ -19,7 +19,7 @@ namespace memfile {
class Directory : public Device {
uint16_t _index;
std::string _path;
- Partition::LP _partition;
+ Partition::SP _partition;
// Only DeviceManager can create these objects, so we only need
// to cope with these constructors being so similar there.
@@ -31,7 +31,7 @@ class Directory : public Device {
friend class DeviceManager;
public:
- typedef vespalib::LinkedPtr<Directory> LP;
+ using SP = std::shared_ptr<Directory>;
void setIndex(uint16_t index) { _index = index; } // Used when deserializing
uint16_t getIndex() const { return _index; }
diff --git a/memfilepersistence/src/vespa/memfilepersistence/device/disk.h b/memfilepersistence/src/vespa/memfilepersistence/device/disk.h
index 77549a12470..9e63b7b7c68 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/device/disk.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/device/disk.h
@@ -15,7 +15,6 @@
#pragma once
#include <vespa/memfilepersistence/device/device.h>
-#include <vespa/vespalib/util/linkedptr.h>
namespace storage {
@@ -29,7 +28,7 @@ class Disk : public Device {
friend class DeviceManager;
public:
- typedef vespalib::LinkedPtr<Disk> LP;
+ using SP = std::shared_ptr<Disk>;
uint64_t getId() const { return _id; }
diff --git a/memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.cpp b/memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.cpp
index 0f5dbb288f1..63d3251fead 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.cpp
@@ -28,9 +28,9 @@ using vespalib::DirPointer;
MountPointList::MountPointList(const std::string& vdsRoot,
const std::vector<vespalib::string>& diskPath,
- DeviceManager::LP manager)
+ DeviceManager::UP manager)
: framework::XmlStatusReporter("mountpointlist", "Disk directories"),
- _deviceManager(manager),
+ _deviceManager(std::move(manager)),
_vdsRoot(vdsRoot),
_diskPath(diskPath),
_mountPoints(0)
@@ -120,7 +120,7 @@ void
MountPointList::scanForDisks()
{
_mountPoints.clear();
- std::vector<Directory::LP> entries;
+ std::vector<Directory::SP> entries;
DirPointer dir(opendir((_vdsRoot + "/disks").c_str()));
struct dirent* entry;
if (dir) while ((entry = readdir(dir))) {
@@ -245,7 +245,7 @@ namespace {
void
MountPointList::readFromFile()
{
- std::vector<Directory::LP> entries;
+ std::vector<Directory::SP> entries;
// Read entries from disk
std::ifstream is;
// Throw exception if failing to read file
@@ -254,7 +254,7 @@ MountPointList::readFromFile()
std::string line("EOF");
while (std::getline(is, line)) {
if (line == "EOF") { break; }
- Directory::LP dir = _deviceManager->deserializeDirectory(line);
+ Directory::SP dir = _deviceManager->deserializeDirectory(line);
int diskNr = getDiskNr(dir->getPath());
if (diskNr == -1) {
LOG(warning, "Found illegal disk entry '%s' in vds disk file %s.",
@@ -297,7 +297,7 @@ MountPointList::writeToFile() const
"disks.status file.", filename.c_str());
return;
}
- for (std::vector<Directory::LP>::const_iterator it
+ for (std::vector<Directory::SP>::const_iterator it
= _mountPoints.begin(); it != _mountPoints.end(); ++it)
{
if (it->get() &&
@@ -453,10 +453,10 @@ MountPointList::verifyHealthyDisks(int mountPointCount)
{
WriteStatusFileIfFailing statusWriter(*this);
int usable = 0, empty = 0;
- std::map<uint32_t, Directory::LP> lackingChunkDef;
+ std::map<uint32_t, Directory::SP> lackingChunkDef;
// Test disks and get chunkinfo
for (uint32_t i=0, n=_mountPoints.size(); i<n; ++i) {
- Directory::LP dir(_mountPoints[i]);
+ Directory::SP dir(_mountPoints[i]);
// Insert NOT_FOUND disk if not found, such that operator[]
// can return only valid pointers
if (!dir.get()) {
@@ -567,10 +567,10 @@ MountPointList::verifyHealthyDisks(int mountPointCount)
}
}
// Write chunkdef files where these are missing
- for (std::map<uint32_t, Directory::LP>::const_iterator it
+ for (std::map<uint32_t, Directory::SP>::const_iterator it
= lackingChunkDef.begin(); it != lackingChunkDef.end(); ++it)
{
- const Directory::LP& dir = it->second;
+ const Directory::SP &dir = it->second;
Chunk c;
c.nr = it->first;
c.total = mountPointCount;
@@ -610,7 +610,7 @@ MountPointList::verifyHealthyDisks(int mountPointCount)
for (int i = _mountPoints.size(); i < mountPointCount; ++i) {
std::ostringstream ost;
ost << _vdsRoot + "/disks/d" << i;
- Directory::LP dir(_deviceManager->getDirectory(ost.str(), i));
+ Directory::SP dir(_deviceManager->getDirectory(ost.str(), i));
dir->addEvent(Device::NOT_FOUND,
"Disk not found during scanning of disks directory",
VESPA_STRLOC);
diff --git a/memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.h b/memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.h
index 33a9574682a..0bd035097f1 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/device/mountpointlist.h
@@ -21,7 +21,6 @@
#include <vespa/memfilepersistence/device/directory.h>
#include <vespa/storageframework/storageframework.h>
#include <vector>
-#include <vespa/vespalib/util/linkedptr.h>
#include <vespa/persistence/spi/persistenceprovider.h>
namespace storage {
@@ -37,7 +36,7 @@ struct MountPointList : public framework::XmlStatusReporter {
/** Create a mount point list. */
MountPointList(const std::string& vdsRoot,
const std::vector<vespalib::string>& diskPath,
- vespalib::LinkedPtr<DeviceManager>);
+ std::unique_ptr<DeviceManager>);
DeviceManager& getDeviceManager() { return *_deviceManager; }
@@ -122,10 +121,10 @@ struct MountPointList : public framework::XmlStatusReporter {
spi::PartitionStateList getPartitionStates() const;
private:
- vespalib::LinkedPtr<DeviceManager> _deviceManager;
+ std::unique_ptr<DeviceManager> _deviceManager;
std::string _vdsRoot;
std::vector<vespalib::string> _diskPath;
- std::vector<Directory::LP> _mountPoints;
+ std::vector<Directory::SP> _mountPoints;
/** Get the name used for the disk status file. */
std::string getDiskStatusFileName() const;
diff --git a/memfilepersistence/src/vespa/memfilepersistence/device/partition.h b/memfilepersistence/src/vespa/memfilepersistence/device/partition.h
index eeedafb7a49..1dc9fdacd41 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/device/partition.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/device/partition.h
@@ -21,8 +21,8 @@ namespace memfile {
class Partition : public Device {
uint64_t _id;
std::string _mountPoint;
- Disk::LP _disk;
- PartitionMonitor::LP _monitor;
+ Disk::SP _disk;
+ PartitionMonitor::UP _monitor;
Partition(DeviceManager& manager, uint64_t id,
const std::string& mountPoint);
@@ -30,7 +30,7 @@ class Partition : public Device {
friend class DeviceManager;
public:
- typedef vespalib::LinkedPtr<Partition> LP;
+ using SP = std::shared_ptr<Partition>;
void initializeMonitor();
diff --git a/memfilepersistence/src/vespa/memfilepersistence/device/partitionmonitor.h b/memfilepersistence/src/vespa/memfilepersistence/device/partitionmonitor.h
index 401a070389e..8e513b0f42f 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/device/partitionmonitor.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/device/partitionmonitor.h
@@ -14,7 +14,6 @@
#include <vespa/vespalib/util/printable.h>
#include <sys/statvfs.h>
#include <vespa/config-stor-devices.h>
-#include <vespa/vespalib/util/linkedptr.h>
#include <vespa/vespalib/util/sync.h>
#include <vespa/vespalib/util/xmlserializable.h>
@@ -28,7 +27,7 @@ class PartitionMonitor : public vespalib::Printable,
public vespalib::XmlSerializable
{
public:
- typedef vespalib::LinkedPtr<PartitionMonitor> LP;
+ using UP = std::unique_ptr<PartitionMonitor>;
/**
* Use an object to stat through, such that unit tests can fake stat
diff --git a/memfilepersistence/src/vespa/memfilepersistence/init/filescanner.cpp b/memfilepersistence/src/vespa/memfilepersistence/init/filescanner.cpp
index e921101aa17..8084d529d79 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/init/filescanner.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/init/filescanner.cpp
@@ -71,7 +71,7 @@ FileScanner::buildBucketList(document::BucketId::List & list,
}
// Grab lock and update metrics
vespalib::LockGuard lock(_globalLock);
- std::vector<metrics::Metric::LP> newMetrics;
+ std::vector<metrics::Metric::UP> newMetrics;
context._metrics.addToSnapshot(_globalMetrics, newMetrics);
assert(newMetrics.empty());
}
diff --git a/memfilepersistence/src/vespa/memfilepersistence/mapper/buffer.h b/memfilepersistence/src/vespa/memfilepersistence/mapper/buffer.h
index 26f4a644d0c..c3409b47656 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/mapper/buffer.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/mapper/buffer.h
@@ -12,7 +12,6 @@
#pragma once
#include <vespa/vespalib/util/alloc.h>
-#include <vespa/vespalib/util/linkedptr.h>
namespace storage {
namespace memfile {
@@ -24,7 +23,7 @@ class Buffer
size_t _size;
public:
- typedef vespalib::LinkedPtr<Buffer> LP;
+ using UP = std::unique_ptr<Buffer>;
Buffer(const Buffer &) = delete;
Buffer & operator = (const Buffer &) = delete;
diff --git a/memfilepersistence/src/vespa/memfilepersistence/mapper/memfile_v1_serializer.h b/memfilepersistence/src/vespa/memfilepersistence/mapper/memfile_v1_serializer.h
index bc1bdd902b9..04d62074f05 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/mapper/memfile_v1_serializer.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/mapper/memfile_v1_serializer.h
@@ -18,7 +18,7 @@ class MemFileV1Serializer : public VersionSerializer
return _metricProvider.getMetrics();
}
public:
- typedef vespalib::LinkedPtr<MemFileV1Serializer> LP;
+ using UP = std::unique_ptr<MemFileV1Serializer>;
MemFileV1Serializer(ThreadMetricProvider&);
diff --git a/memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.cpp b/memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.cpp
index 686ff05cf0e..90b261ceccc 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.cpp
@@ -69,7 +69,7 @@ MemFileMapper::sendNotifyBucketCommand(const MemFile&,
}
void
-MemFileMapper::addVersionSerializer(VersionSerializer::LP serializer)
+MemFileMapper::addVersionSerializer(VersionSerializer::UP serializer)
{
FileVersion version = serializer->getFileVersion();
if (_serializers.find(version) != _serializers.end()) {
@@ -78,13 +78,13 @@ MemFileMapper::addVersionSerializer(VersionSerializer::LP serializer)
<< " is already registered.";
throw vespalib::IllegalStateException(error.str(), VESPA_STRLOC);
}
- _serializers[version] = serializer;
+ _serializers[version] = std::move(serializer);
}
VersionSerializer&
MemFileMapper::getVersionSerializer(const MemFile& file)
{
- std::map<FileVersion, VersionSerializer::LP>::iterator it(
+ std::map<FileVersion, VersionSerializer::UP>::iterator it(
_serializers.find(file.getCurrentVersion()));
if (it == _serializers.end()) {
std::ostringstream ost;
@@ -99,7 +99,7 @@ MemFileMapper::getVersionSerializer(const MemFile& file)
MemFileMapper::MemFileMapper(ThreadMetricProvider& metricProvider)
: _metricProvider(metricProvider)
{
- addVersionSerializer(VersionSerializer::LP(new MemFileV1Serializer(metricProvider)));
+ addVersionSerializer(VersionSerializer::UP(new MemFileV1Serializer(metricProvider)));
}
void
@@ -107,7 +107,7 @@ MemFileMapper::setDefaultMemFileIO(MemFile& file,
vespalib::LazyFile::UP lf,
const Environment& env)
{
- std::map<FileVersion, VersionSerializer::LP>::iterator serializer(
+ std::map<FileVersion, VersionSerializer::UP>::iterator serializer(
_serializers.find(file.getFile().getWantedFileVersion()));
assert(serializer != _serializers.end());
@@ -162,7 +162,7 @@ MemFileMapper::loadFileImpl(MemFile& file, Environment& env)
FileVersion version = static_cast<FileVersion>(
*reinterpret_cast<uint32_t*>(buffer.getBuffer()));
- std::map<FileVersion, VersionSerializer::LP>::iterator serializer(
+ std::map<FileVersion, VersionSerializer::UP>::iterator serializer(
_serializers.find(version));
file.setCurrentVersion(version);
if (serializer == _serializers.end()) {
@@ -237,7 +237,7 @@ MemFileMapper::flush(MemFile& f, Environment& env, bool autoRepair)
}
// If we get here we failed to write updates only and will rewrite
- std::map<FileVersion, VersionSerializer::LP>::iterator serializer(
+ std::map<FileVersion, VersionSerializer::UP>::iterator serializer(
_serializers.find(f.getFile().getWantedFileVersion()));
assert(serializer != _serializers.end());
@@ -250,7 +250,7 @@ MemFileMapper::verify(MemFile& file, Environment& env,
uint16_t fileVerifyFlags)
{
if (file.fileExists()) {
- std::map<FileVersion, VersionSerializer::LP>::iterator serializer(
+ std::map<FileVersion, VersionSerializer::UP>::iterator serializer(
_serializers.find(file.getCurrentVersion()));
if (serializer != _serializers.end()) {
bool wasOk = serializer->second->verify(
diff --git a/memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.h b/memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.h
index 30f483fc582..b82bc59b732 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/mapper/memfilemapper.h
@@ -25,7 +25,7 @@ namespace memfile {
class MemFileMapper : private Types {
private:
- std::map<FileVersion, VersionSerializer::LP> _serializers;
+ std::map<FileVersion, VersionSerializer::UP> _serializers;
ThreadMetricProvider& _metricProvider;
void setDefaultMemFileIO(MemFile& file,
vespalib::LazyFile::UP lf,
@@ -84,7 +84,7 @@ public:
void removeAllSlotsExcept(MemFile& file, std::vector<Timestamp>& keep);
private:
- void addVersionSerializer(VersionSerializer::LP);
+ void addVersionSerializer(VersionSerializer::UP);
VersionSerializer& getVersionSerializer(const MemFile& file);
void loadFileImpl(MemFile&, Environment&);
diff --git a/memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.cpp b/memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.cpp
index 3c10efdf0c9..da3b180201e 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.cpp
@@ -292,24 +292,24 @@ SimpleMemFileIOBuffer::allocateBuffer(DocumentPart part,
// If the requested size is greater than or equal to our working buffer
// size, simply allocate a separate buffer for it.
if (sz >= WORKING_BUFFER_SIZE) {
- return BufferAllocation(SharedBuffer::LP(new SharedBuffer(sz)), 0, sz);
+ return BufferAllocation(SharedBuffer::SP(new SharedBuffer(sz)), 0, sz);
}
- SharedBuffer::LP& bufLP(_workingBuffers[part]);
+ SharedBuffer::SP &bufSP(_workingBuffers[part]);
bool requireNewBlock = false;
- if (!bufLP.get()) {
+ if (!bufSP.get()) {
requireNewBlock = true;
- } else if (!bufLP->hasRoomFor(sz, align)) {
+ } else if (!bufSP->hasRoomFor(sz, align)) {
requireNewBlock = true;
}
if (!requireNewBlock) {
- return BufferAllocation(bufLP,
- static_cast<uint32_t>(bufLP->allocate(sz, align)),
+ return BufferAllocation(bufSP,
+ static_cast<uint32_t>(bufSP->allocate(sz, align)),
sz);
} else {
- SharedBuffer::LP newBuf(new SharedBuffer(WORKING_BUFFER_SIZE));
- bufLP = newBuf;
+ SharedBuffer::SP newBuf(new SharedBuffer(WORKING_BUFFER_SIZE));
+ bufSP = newBuf;
return BufferAllocation(newBuf,
static_cast<uint32_t>(newBuf->allocate(sz, align)),
sz);
@@ -384,7 +384,7 @@ SimpleMemFileIOBuffer::copyCache(const MemFileIOInterface& source,
void
SimpleMemFileIOBuffer::cacheLocation(DocumentPart part,
DataLocation loc,
- BufferType::LP& buf,
+ BufferType::SP buf,
uint32_t bufferPos)
{
LOG(spam,
@@ -396,7 +396,7 @@ SimpleMemFileIOBuffer::cacheLocation(DocumentPart part,
loc._size,
buf.get(),
bufferPos);
- _data[part][loc] = Data(buf, bufferPos, true);
+ _data[part][loc] = Data(std::move(buf), bufferPos, true);
}
bool
diff --git a/memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.h b/memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.h
index 3e91916ff0b..40c5ffb4865 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/mapper/simplememfileiobuffer.h
@@ -74,7 +74,7 @@ public:
ALIGN_512_BYTES
};
- typedef vespalib::LinkedPtr<SharedBuffer> LP;
+ using SP = std::shared_ptr<SharedBuffer>;
explicit SharedBuffer(size_t totalSize)
: _buf(vespalib::alloc::Alloc::allocMMap(totalSize)),
_totalSize(totalSize),
@@ -123,8 +123,8 @@ public:
{
BufferAllocation() : pos(0), size(0) {}
- BufferAllocation(const SharedBuffer::LP& b, uint32_t p, uint32_t sz)
- : buf(b), pos(p), size(sz) { }
+ BufferAllocation(SharedBuffer::SP b, uint32_t p, uint32_t sz)
+ : buf(std::move(b)), pos(p), size(sz) { }
/**
* Get buffer area available to this specific allocation
@@ -136,11 +136,11 @@ public:
* Get buffer that is (potentially) shared between many individual
* allocations.
*/
- SharedBuffer::LP& getSharedBuffer() { return buf; }
+ SharedBuffer::SP getSharedBuffer() { return buf; }
uint32_t getBufferPosition() const { return pos; }
uint32_t getSize() const { return size; }
- SharedBuffer::LP buf;
+ SharedBuffer::SP buf;
uint32_t pos;
uint32_t size;
};
@@ -250,7 +250,7 @@ public:
*/
void cacheLocation(DocumentPart part,
DataLocation loc,
- BufferType::LP& buf,
+ BufferType::SP buf,
uint32_t bufferPos);
/**
@@ -319,10 +319,10 @@ private:
struct Data {
Data() : pos(0), persisted(false) {}
- Data(const BufferType::LP& b, uint32_t p, bool isPersisted)
- : buf(b), pos(p), persisted(isPersisted) {}
+ Data(BufferType::SP b, uint32_t p, bool isPersisted)
+ : buf(std::move(b)), pos(p), persisted(isPersisted) {}
- BufferType::LP buf;
+ BufferType::SP buf;
uint32_t pos;
bool persisted;
};
@@ -331,7 +331,7 @@ private:
VersionSerializer& _reader;
std::vector<DataMap> _data;
- std::vector<SharedBuffer::LP> _workingBuffers;
+ std::vector<SharedBuffer::SP> _workingBuffers;
vespalib::LazyFile::UP _file;
FileInfo::UP _fileInfo;
FileSpecification _fileSpec;
diff --git a/memfilepersistence/src/vespa/memfilepersistence/mapper/versionserializer.h b/memfilepersistence/src/vespa/memfilepersistence/mapper/versionserializer.h
index b57734c2b24..7cc45451126 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/mapper/versionserializer.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/mapper/versionserializer.h
@@ -15,7 +15,6 @@
#include <vespa/memfilepersistence/mapper/buffer.h>
#include <vespa/memfilepersistence/mapper/mapperslotoperation.h>
#include <vespa/memfilepersistence/memfile/memfile.h>
-#include <vespa/vespalib/util/linkedptr.h>
#include <vespa/memfilepersistence/memfile/memfileiointerface.h>
namespace storage {
@@ -26,7 +25,7 @@ class MemFileEnvironment;
class Options;
struct VersionSerializer : protected Types {
- typedef vespalib::LinkedPtr<VersionSerializer> LP;
+ using UP = std::unique_ptr<VersionSerializer>;
virtual ~VersionSerializer() {}
diff --git a/memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.cpp b/memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.cpp
index 722fef80103..409789eadcc 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.cpp
@@ -203,7 +203,7 @@ MemFileCache::get(const BucketId& id, Environment& env, Directory& dir,
// in the common case that there's a bucket file on the disk. The
// content layer shall guarantee that no concurrent operations happen
// for a single bucket, so this should be fully thread safe.
- Entry::LP entry(new Entry(file, env, createIfNotExisting));
+ Entry::SP entry(new Entry(file, env, createIfNotExisting));
vespalib::LockGuard reLock(_cacheLock);
std::pair<LRUCache::iterator, bool> inserted(
@@ -211,7 +211,7 @@ MemFileCache::get(const BucketId& id, Environment& env, Directory& dir,
assert(inserted.second);
_metrics.misses.inc();
- return MemFilePtr(MemFilePtr::EntryGuard::LP(
+ return MemFilePtr(MemFilePtr::EntryGuard::SP(
new CacheEntryGuard(*this, env, *entry)));
} else {
if (it->_ptr->isInUse()) {
@@ -234,7 +234,7 @@ MemFileCache::get(const BucketId& id, Environment& env, Directory& dir,
it->_ptr->_cacheSize.toString().c_str(),
_memoryUsage.toString().c_str());
- return MemFilePtr(MemFilePtr::EntryGuard::LP(
+ return MemFilePtr(MemFilePtr::EntryGuard::SP(
new CacheEntryGuard(*this, env, *it->_ptr)));
}
diff --git a/memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.h b/memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.h
index e4417fb6e0f..112c5e6c399 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/memfile/memfilecache.h
@@ -53,7 +53,7 @@ public:
private:
class Entry {
public:
- typedef vespalib::LinkedPtr<Entry> LP;
+ using SP = std::shared_ptr<Entry>;
MemFile _file;
MemoryUsage _cacheSize;
@@ -78,10 +78,10 @@ private:
struct EntryWrapper {
EntryWrapper(
- Entry::LP ptr,
+ Entry::SP ptr,
uint64_t lastUsed,
const document::BucketId& bid)
- : _ptr(ptr), _lastUsed(lastUsed), _bid(bid) {}
+ : _ptr(std::move(ptr)), _lastUsed(lastUsed), _bid(bid) {}
const Entry* operator->() const {
return _ptr.get();
@@ -91,7 +91,7 @@ private:
return _ptr.get();
};
- Entry::LP _ptr;
+ Entry::SP _ptr;
uint64_t _lastUsed;
document::BucketId _bid;
};
diff --git a/memfilepersistence/src/vespa/memfilepersistence/memfile/memfileptr.h b/memfilepersistence/src/vespa/memfilepersistence/memfile/memfileptr.h
index 545686e5f2f..32d86bd415c 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/memfile/memfileptr.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/memfile/memfileptr.h
@@ -18,8 +18,6 @@
#pragma once
-#include <vespa/vespalib/util/linkedptr.h>
-
namespace storage {
namespace memfile {
@@ -33,7 +31,7 @@ public:
* doing it, to prevent cyclic dependency with cache.
*/
struct EntryGuard {
- typedef vespalib::LinkedPtr<EntryGuard> LP;
+ using SP = std::shared_ptr<EntryGuard>;
MemFile* _file;
@@ -46,11 +44,11 @@ public:
};
private:
- EntryGuard::LP _entry;
+ EntryGuard::SP _entry;
public:
MemFilePtr() {};
- MemFilePtr(EntryGuard::LP entry) : _entry(entry) {}
+ MemFilePtr(EntryGuard::SP entry) : _entry(std::move(entry)) {}
// Behave like pointer to MemFile for ease of use.
MemFile* operator->() { return _entry->_file; }
diff --git a/memfilepersistence/src/vespa/memfilepersistence/memfile/memslot.h b/memfilepersistence/src/vespa/memfilepersistence/memfile/memslot.h
index 32f7f0ebe39..31a0b7b2ea6 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/memfile/memslot.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/memfile/memslot.h
@@ -81,7 +81,7 @@ public:
std::string toString() const;
};
- typedef vespalib::LinkedPtr<MemSlot> LP;
+ using UP = std::unique_ptr<MemSlot>;
MemSlot(const MemSlot&);
/** Constructor used by mappers reading from file. */
diff --git a/memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.cpp b/memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.cpp
index 3985459a5be..7a733e8cc1b 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.cpp
@@ -60,7 +60,7 @@ IteratorHandler::createIterator(const spi::Bucket& bucket,
// By default, no explicit prefetching is required.
CachePrefetchRequirements prefetcher;
- vespalib::LinkedPtr<document::select::Node> docSelection;
+ std::unique_ptr<document::select::Node> docSelection;
if (!sel.getDocumentSelection().getDocumentSelection().empty()) {
docSelection.reset(
parseDocumentSelection(
@@ -96,7 +96,7 @@ IteratorHandler::createIterator(const spi::Bucket& bucket,
sel,
document::FieldSet::UP(fields.clone()),
versions,
- docSelection,
+ std::move(docSelection),
prefetcher))));
assert(inserted.second); // Should never have duplicates
@@ -250,11 +250,11 @@ IteratorHandler::addMetaDataEntry(spi::IterateResult::List& result,
totalSize += entrySize;
int metaFlags = (slot.deleted() || slot.deletedInPlace()) ? spi::REMOVE_ENTRY : 0;
- spi::DocEntry::LP docEntry(
+ spi::DocEntry::UP docEntry(
new spi::DocEntry(
spi::Timestamp(slot.getTimestamp().getTime()),
metaFlags));
- result.push_back(docEntry);
+ result.push_back(std::move(docEntry));
return true;
}
@@ -274,12 +274,12 @@ IteratorHandler::addRemoveEntry(spi::IterateResult::List& results,
}
totalSize += entrySize;
- spi::DocEntry::LP docEntry(
+ spi::DocEntry::UP docEntry(
new spi::DocEntry(
spi::Timestamp(slot.getTimestamp().getTime()),
spi::REMOVE_ENTRY,
did));
- results.push_back(docEntry);
+ results.push_back(std::move(docEntry));
return true;
}
@@ -307,12 +307,12 @@ IteratorHandler::addPutEntry(spi::IterateResult::List& results,
{
document::FieldSet::stripFields(*doc, fieldsToKeep);
}
- spi::DocEntry::LP docEntry(
+ spi::DocEntry::UP docEntry(
new spi::DocEntry(spi::Timestamp(slot.getTimestamp().getTime()),
0,
std::move(doc),
docSize));
- results.push_back(docEntry);
+ results.push_back(std::move(docEntry));
return true;
}
@@ -338,7 +338,7 @@ IteratorHandler::iterate(spi::IteratorId id, uint64_t maxByteSize)
assert(!iter->second.isActive());
state = &iter->second;
if (state->isCompleted()) {
- return spi::IterateResult(results, true);
+ return spi::IterateResult(std::move(results), true);
}
state->setActive(true);
}
@@ -422,10 +422,10 @@ IteratorHandler::iterate(spi::IteratorId id, uint64_t maxByteSize)
if (remaining.empty()) {
state->setCompleted();
- return spi::IterateResult(results, true);
+ return spi::IterateResult(std::move(results), true);
}
- return spi::IterateResult(results, false);
+ return spi::IterateResult(std::move(results), false);
}
}
diff --git a/memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.h b/memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.h
index 7b3ee9627e5..4e15ef5d137 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.h
+++ b/memfilepersistence/src/vespa/memfilepersistence/spi/iteratorhandler.h
@@ -11,14 +11,12 @@
#include <vespa/memfilepersistence/spi/operationhandler.h>
#include <vespa/persistence/spi/persistenceprovider.h>
#include <vespa/document/fieldset/fieldsetrepo.h>
+#include <vespa/document/select/node.h>
namespace document {
class FieldSet;
-namespace select {
-class Node;
-}
}
namespace storage {
@@ -75,8 +73,8 @@ class IteratorState
{
spi::Bucket _bucket;
spi::Selection _selection;
- vespalib::LinkedPtr<document::FieldSet> _fieldSet;
- vespalib::LinkedPtr<document::select::Node> _documentSelection;
+ std::unique_ptr<document::FieldSet> _fieldSet;
+ std::unique_ptr<document::select::Node> _documentSelection;
std::vector<Types::Timestamp> _remaining;
spi::IncludedVersions _versions;
CachePrefetchRequirements _prefetchRequirements;
@@ -89,12 +87,12 @@ public:
const spi::Selection& sel,
document::FieldSet::UP fieldSet,
spi::IncludedVersions versions,
- vespalib::LinkedPtr<document::select::Node> docSel,
+ std::unique_ptr<document::select::Node> docSel,
const CachePrefetchRequirements& prefetchRequirements)
: _bucket(bucket),
_selection(sel),
- _fieldSet(vespalib::LinkedPtr<document::FieldSet>(fieldSet.release())),
- _documentSelection(docSel),
+ _fieldSet(std::move(fieldSet)),
+ _documentSelection(std::move(docSel)),
_remaining(),
_versions(versions),
_prefetchRequirements(prefetchRequirements),
diff --git a/memfilepersistence/src/vespa/memfilepersistence/tools/dumpslotfile.cpp b/memfilepersistence/src/vespa/memfilepersistence/tools/dumpslotfile.cpp
index f3eb1429207..030bac0752e 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/tools/dumpslotfile.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/tools/dumpslotfile.cpp
@@ -257,7 +257,7 @@ int SlotFileDumper::dump(int argc, const char * const * argv,
EnvironmentImpl env(config, o.toXml ? o.documentManConfigId.c_str() : "");
document::BucketId bucket(extractBucketId(o.filename));
- Directory::LP dir(env._deviceManager.getDirectory(o.filename, 0));
+ Directory::SP dir(env._deviceManager.getDirectory(o.filename, 0));
FileSpecification fileSpec(bucket, *dir, o.filename);
MemFile::LoadOptions opts;
diff --git a/memfilepersistence/src/vespa/memfilepersistence/tools/vdsdisktool.cpp b/memfilepersistence/src/vespa/memfilepersistence/tools/vdsdisktool.cpp
index ad9b08f4994..04d300e83b0 100644
--- a/memfilepersistence/src/vespa/memfilepersistence/tools/vdsdisktool.cpp
+++ b/memfilepersistence/src/vespa/memfilepersistence/tools/vdsdisktool.cpp
@@ -379,12 +379,12 @@ VdsDiskTool::run(int argc, const char * const * argv,
}
framework::defaultimplementation::RealClock clock;
// Read the disk status file.
- DeviceManager::LP devMan(new DeviceManager(
+ DeviceManager::UP devMan(new DeviceManager(
DeviceMapper::UP(new SimpleDeviceMapper),
clock));
MountPointList mountPointList(options.getNodePath(nodeIndex),
vector<vespalib::string>(),
- devMan);
+ std::move(devMan));
mountPointList.scanForDisks();
if (options._mode == "enable" || options._mode == "disable") {
if (mountPointList.getSize() <= options._diskIndex