summaryrefslogtreecommitdiffstats
path: root/src/os
diff options
context:
space:
mode:
Diffstat (limited to 'src/os')
-rw-r--r--src/os/DBObjectMap.cc5
-rw-r--r--src/os/DBObjectMap.h2
-rw-r--r--src/os/ObjectStore.h52
-rw-r--r--src/os/bluestore/BlueFS.cc11
-rw-r--r--src/os/bluestore/BlueStore.cc220
-rw-r--r--src/os/bluestore/BlueStore.h36
-rw-r--r--src/os/bluestore/bluefs_types.cc4
-rw-r--r--src/os/bluestore/bluefs_types.h1
-rw-r--r--src/os/bluestore/bluestore_tool.cc2
-rw-r--r--src/os/fs/btrfs_ioctl.h201
-rw-r--r--src/os/kstore/KStore.cc72
-rw-r--r--src/os/kstore/KStore.h8
-rw-r--r--src/os/memstore/MemStore.cc70
-rw-r--r--src/os/memstore/MemStore.h15
14 files changed, 391 insertions, 308 deletions
diff --git a/src/os/DBObjectMap.cc b/src/os/DBObjectMap.cc
index 7da9a67be62..65627b5f818 100644
--- a/src/os/DBObjectMap.cc
+++ b/src/os/DBObjectMap.cc
@@ -519,6 +519,11 @@ bufferlist DBObjectMap::DBObjectMapIteratorImpl::value()
return cur_iter->value();
}
+std::string_view DBObjectMap::DBObjectMapIteratorImpl::value_as_sv()
+{
+ return cur_iter->value_as_sv();
+}
+
int DBObjectMap::DBObjectMapIteratorImpl::status()
{
return r;
diff --git a/src/os/DBObjectMap.h b/src/os/DBObjectMap.h
index 444f21eb815..1e1452010e7 100644
--- a/src/os/DBObjectMap.h
+++ b/src/os/DBObjectMap.h
@@ -393,6 +393,7 @@ private:
int next() override { ceph_abort(); return 0; }
std::string key() override { ceph_abort(); return ""; }
ceph::buffer::list value() override { ceph_abort(); return ceph::buffer::list(); }
+ std::string_view value_as_sv() override { ceph_abort(); return std::string_view(); }
int status() override { return 0; }
};
@@ -431,6 +432,7 @@ private:
int next() override;
std::string key() override;
ceph::buffer::list value() override;
+ std::string_view value_as_sv() override;
int status() override;
bool on_parent() {
diff --git a/src/os/ObjectStore.h b/src/os/ObjectStore.h
index 521435b6c31..df3ae920a2f 100644
--- a/src/os/ObjectStore.h
+++ b/src/os/ObjectStore.h
@@ -29,6 +29,7 @@
#include <errno.h>
#include <sys/stat.h>
+#include <functional>
#include <map>
#include <memory>
#include <vector>
@@ -735,15 +736,6 @@ public:
std::map<std::string, ceph::buffer::list> *out ///< [out] Returned keys and values
) = 0;
-#ifdef WITH_SEASTAR
- virtual int omap_get_values(
- CollectionHandle &c, ///< [in] Collection containing oid
- const ghobject_t &oid, ///< [in] Object containing omap
- const std::optional<std::string> &start_after, ///< [in] Keys to get
- std::map<std::string, ceph::buffer::list> *out ///< [out] Returned keys and values
- ) = 0;
-#endif
-
/// Filters keys into out which are defined on oid
virtual int omap_check_keys(
CollectionHandle &c, ///< [in] Collection containing oid
@@ -766,6 +758,48 @@ public:
const ghobject_t &oid ///< [in] object
) = 0;
+ struct omap_iter_seek_t {
+ std::string seek_position;
+ enum {
+ // start with provided key (seek_position), if it exists
+ LOWER_BOUND,
+ // skip provided key (seek_position) even if it exists
+ UPPER_BOUND
+ } seek_type = LOWER_BOUND;
+ static omap_iter_seek_t min_lower_bound() { return {}; }
+ };
+ enum class omap_iter_ret_t {
+ STOP,
+ NEXT
+ };
+ /**
+ * Iterate over object map with user-provided callable
+ *
+ * Warning! The callable is executed under lock on bluestore
+ * operations in c. Do not use bluestore methods on c while
+ * iterating. (Filling in a transaction is no problem).
+ *
+ * @param c collection
+ * @param oid object
+ * @param start_from where the iterator should point to at
+ * the beginning
+ * @param visitor callable that takes OMAP key and corresponding
+ * value as string_views and controls iteration
+ * by the return. It is executed for every object's
+ * OMAP entry from `start_from` till end of the
+ * object's OMAP or till the iteration is stopped
+ * by `STOP`. Please note that if there is no such
+ * entry, `visitor` will be called 0 times.
+ * @return error code, zero on success
+ */
+ virtual int omap_iterate(
+ CollectionHandle &c,
+ const ghobject_t &oid,
+ omap_iter_seek_t start_from,
+ std::function<omap_iter_ret_t(std::string_view,
+ std::string_view)> visitor
+ ) = 0;
+
virtual int flush_journal() { return -EOPNOTSUPP; }
virtual int dump_journal(std::ostream& out) { return -EOPNOTSUPP; }
diff --git a/src/os/bluestore/BlueFS.cc b/src/os/bluestore/BlueFS.cc
index 3b30722b652..50f293d45fd 100644
--- a/src/os/bluestore/BlueFS.cc
+++ b/src/os/bluestore/BlueFS.cc
@@ -5,6 +5,7 @@
#include "bluestore_common.h"
#include "BlueFS.h"
+#include "common/Clock.h" // for ceph_clock_now()
#include "common/debug.h"
#include "common/errno.h"
#include "common/perf_counters.h"
@@ -12,6 +13,12 @@
#include "include/ceph_assert.h"
#include "common/admin_socket.h"
+#if defined(WITH_SEASTAR) && !defined(WITH_ALIEN)
+#include "crimson/common/perf_counters_collection.h"
+#else
+#include "common/perf_counters_collection.h"
+#endif
+
#define dout_context cct
#define dout_subsys ceph_subsys_bluefs
#undef dout_prefix
@@ -1699,7 +1706,8 @@ int BlueFS::_replay(bool noop, bool to_stdout)
<< " fnode=" << fnode
<< " delta=" << delta
<< dendl;
- ceph_assert(delta.offset == fnode.allocated);
+ // be leanient, if there is no extents just produce error message
+ ceph_assert(delta.offset == fnode.allocated || delta.extents.empty());
}
if (cct->_conf->bluefs_log_replay_check_allocations) {
int r = _check_allocations(fnode,
@@ -3823,6 +3831,7 @@ int BlueFS::truncate(FileWriter *h, uint64_t offset)/*_WF_L*/
if (changed_extents) {
fnode.size = offset;
fnode.reset_delta();
+ fnode.recalc_allocated();
log.t.op_file_update(fnode);
// sad, but is_dirty must be set to signal flushing of the log
h->file->is_dirty = true;
diff --git a/src/os/bluestore/BlueStore.cc b/src/os/bluestore/BlueStore.cc
index 535cf166f0a..8f1d995fa8d 100644
--- a/src/os/bluestore/BlueStore.cc
+++ b/src/os/bluestore/BlueStore.cc
@@ -4830,7 +4830,7 @@ void BlueStore::Onode::rewrite_omap_key(const string& old, string *out)
out->append(old.c_str() + out->length(), old.size() - out->length());
}
-void BlueStore::Onode::decode_omap_key(const string& key, string *user_key)
+size_t BlueStore::Onode::calc_userkey_offset_in_omap_key() const
{
size_t pos = sizeof(uint64_t) + 1;
if (!onode.is_pgmeta_omap()) {
@@ -4840,9 +4840,15 @@ void BlueStore::Onode::decode_omap_key(const string& key, string *user_key)
pos += sizeof(uint64_t);
}
}
- *user_key = key.substr(pos);
+ return pos;
+}
+
+void BlueStore::Onode::decode_omap_key(const string& key, string *user_key)
+{
+ *user_key = key.substr(calc_userkey_offset_in_omap_key());
}
+
void BlueStore::Onode::finish_write(TransContext* txc, uint32_t offset, uint32_t length)
{
while (true) {
@@ -5519,7 +5525,13 @@ BlueStore::OmapIteratorImpl::OmapIteratorImpl(
if (o->onode.has_omap()) {
o->get_omap_key(string(), &head);
o->get_omap_tail(&tail);
+ auto start1 = mono_clock::now();
it->lower_bound(head);
+ c->store->log_latency(
+ __func__,
+ l_bluestore_omap_seek_to_first_lat,
+ mono_clock::now() - start1,
+ c->store->cct->_conf->bluestore_log_omap_iterator_age);
}
}
BlueStore::OmapIteratorImpl::~OmapIteratorImpl()
@@ -5654,6 +5666,13 @@ bufferlist BlueStore::OmapIteratorImpl::value()
return it->value();
}
+std::string_view BlueStore::OmapIteratorImpl::value_as_sv()
+{
+ std::shared_lock l(c->lock);
+ ceph_assert(it->valid());
+ return it->value_as_sv();
+}
+
// =====================================
@@ -6911,8 +6930,19 @@ int BlueStore::_check_main_bdev_label()
return -EIO;
}
if (bluestore_bdev_label_require_all && r != 0) {
- derr << __func__ << " not all labels read properly" << dendl;
- return -EIO;
+ // We are about to complain that some labels failed.
+ // But in case if we expanded block device some labels will not be good.
+ uint64_t lsize = std::max(BDEV_LABEL_BLOCK_SIZE, min_alloc_size);
+ uint32_t valid_locations = 0;
+ for (uint64_t loc : bdev_label_positions) {
+ if (loc + lsize <= bdev_label.size) {
+ ++valid_locations;
+ }
+ }
+ if (valid_locations != bdev_label_valid_locations.size()) {
+ derr << __func__ << " not all labels read properly" << dendl;
+ return -EIO;
+ }
}
return 0;
}
@@ -8948,11 +8978,25 @@ int BlueStore::expand_devices(ostream& out)
_close_db_and_around();
// mount in read/write to sync expansion changes
+ if (bdev_label_multi) {
+ // We need not do fsck, because we can be broken - size is increased,
+ // but we might not have labels set.
+ cct->_conf.set_val_or_die("bluestore_fsck_on_mount", "false");
+ }
r = _mount();
ceph_assert(r == 0);
if (fm && fm->is_null_manager()) {
// we grow the allocation range, must reflect it in the allocation file
alloc->init_add_free(size0, size - size0);
+ if (bdev_label_multi) {
+ uint64_t lsize = std::max(BDEV_LABEL_BLOCK_SIZE, min_alloc_size);
+ for (uint64_t loc : bdev_label_positions) {
+ if ((loc >= size0) && (loc + lsize <= size)) {
+ bdev_label_valid_locations.push_back(loc);
+ }
+ }
+ _write_bdev_label(cct, bdev, path + "/block", bdev_label, bdev_label_valid_locations);
+ }
need_to_destage_allocation_file = true;
}
umount();
@@ -13601,52 +13645,6 @@ int BlueStore::omap_get_values(
return r;
}
-#ifdef WITH_SEASTAR
-int BlueStore::omap_get_values(
- CollectionHandle &c_, ///< [in] Collection containing oid
- const ghobject_t &oid, ///< [in] Object containing omap
- const std::optional<string> &start_after, ///< [in] Keys to get
- map<string, bufferlist> *output ///< [out] Returned keys and values
- )
-{
- Collection *c = static_cast<Collection *>(c_.get());
- dout(15) << __func__ << " " << c->get_cid() << " oid " << oid << dendl;
- if (!c->exists)
- return -ENOENT;
- std::shared_lock l(c->lock);
- int r = 0;
- OnodeRef o = c->get_onode(oid, false);
- if (!o || !o->exists) {
- r = -ENOENT;
- goto out;
- }
- if (!o->onode.has_omap()) {
- goto out;
- }
- o->flush();
- {
- ObjectMap::ObjectMapIterator iter = get_omap_iterator(c_, oid);
- if (!iter) {
- r = -ENOENT;
- goto out;
- }
- if (start_after) {
- iter->upper_bound(*start_after);
- } else {
- iter->seek_to_first();
- }
- for (; iter->valid(); iter->next()) {
- output->insert(make_pair(iter->key(), iter->value()));
- }
- }
-
-out:
- dout(10) << __func__ << " " << c->get_cid() << " oid " << oid << " = " << r
- << dendl;
- return r;
-}
-#endif
-
int BlueStore::omap_check_keys(
CollectionHandle &c_, ///< [in] Collection containing oid
const ghobject_t &oid, ///< [in] Object containing omap
@@ -13724,6 +13722,94 @@ ObjectMap::ObjectMapIterator BlueStore::get_omap_iterator(
return ObjectMap::ObjectMapIterator(new OmapIteratorImpl(logger,c, o, it));
}
+int BlueStore::omap_iterate(
+ CollectionHandle &c_, ///< [in] collection
+ const ghobject_t &oid, ///< [in] object
+ ObjectStore::omap_iter_seek_t start_from, ///< [in] where the iterator should point to at the beginning
+ std::function<omap_iter_ret_t(std::string_view, std::string_view)> f
+ )
+{
+ Collection *c = static_cast<Collection *>(c_.get());
+ dout(10) << __func__ << " " << c->get_cid() << " " << oid << dendl;
+ if (!c->exists) {
+ return -ENOENT;
+ }
+ std::shared_lock l(c->lock);
+ OnodeRef o = c->get_onode(oid, false);
+ if (!o || !o->exists) {
+ dout(10) << __func__ << " " << oid << "doesn't exist" <<dendl;
+ return -ENOENT;
+ }
+ o->flush();
+ dout(10) << __func__ << " has_omap = " << (int)o->onode.has_omap() <<dendl;
+ if (!o->onode.has_omap()) {
+ // nothing to do
+ return 0;
+ }
+
+ KeyValueDB::Iterator it;
+ {
+ auto bounds = KeyValueDB::IteratorBounds();
+ std::string lower_bound, upper_bound;
+ o->get_omap_key(string(), &lower_bound);
+ o->get_omap_tail(&upper_bound);
+ bounds.lower_bound = std::move(lower_bound);
+ bounds.upper_bound = std::move(upper_bound);
+ it = db->get_iterator(o->get_omap_prefix(), 0, std::move(bounds));
+ }
+
+ // seek the iterator
+ {
+ std::string key;
+ o->get_omap_key(start_from.seek_position, &key);
+ auto start = ceph::mono_clock::now();
+ if (start_from.seek_type == omap_iter_seek_t::LOWER_BOUND) {
+ it->lower_bound(key);
+ c->store->log_latency(
+ __func__,
+ l_bluestore_omap_lower_bound_lat,
+ ceph::mono_clock::now() - start,
+ c->store->cct->_conf->bluestore_log_omap_iterator_age);
+ } else {
+ it->upper_bound(key);
+ c->store->log_latency(
+ __func__,
+ l_bluestore_omap_upper_bound_lat,
+ ceph::mono_clock::now() - start,
+ c->store->cct->_conf->bluestore_log_omap_iterator_age);
+ }
+ }
+
+ // iterate!
+ std::string tail;
+ o->get_omap_tail(&tail);
+ const std::string_view::size_type userkey_offset_in_dbkey =
+ o->calc_userkey_offset_in_omap_key();
+ ceph::timespan next_lat_acc{0};
+ while (it->valid()) {
+ const auto& db_key = it->raw_key_as_sv().second;
+ if (db_key >= tail) {
+ break;
+ }
+ std::string_view user_key = db_key.substr(userkey_offset_in_dbkey);
+ omap_iter_ret_t ret = f(user_key, it->value_as_sv());
+ if (ret == omap_iter_ret_t::STOP) {
+ break;
+ } else if (ret == omap_iter_ret_t::NEXT) {
+ ceph::time_guard<ceph::mono_clock>{next_lat_acc};
+ it->next();
+ } else {
+ ceph_abort();
+ }
+ }
+ c->store->log_latency(
+ __func__,
+ l_bluestore_omap_next_lat,
+ next_lat_acc,
+ c->store->cct->_conf->bluestore_log_omap_iterator_age);
+ return 0;
+}
+
// -----------------
// write helpers
@@ -14129,6 +14215,7 @@ void BlueStore::_txc_state_proc(TransContext *txc)
if (txc->had_ios)
kv_ios++;
kv_throttle_costs += txc->cost;
+ ++kv_throttle_txcs;
}
return;
case TransContext::STATE_KV_SUBMITTED:
@@ -14375,7 +14462,18 @@ void BlueStore::_txc_committed_kv(TransContext *txc)
mono_clock::now() - txc->start,
cct->_conf->bluestore_log_op_age,
[&](auto lat) {
- return ", txc = " + stringify(txc);
+ return ", txc = " + stringify(txc) +
+ ", txc bytes = " + stringify(txc->bytes) +
+ ", txc ios = " + stringify(txc->ios) +
+ ", txc cost = " + stringify(txc->cost) +
+ ", txc onodes = " + stringify(txc->onodes.size()) +
+ ", DB updates = " + stringify(txc->t->get_count()) +
+ ", DB bytes = " + stringify(txc->t->get_size_bytes()) +
+ ", cost max = " + stringify(throttle.bytes_observed_max) +
+ " on " + stringify(throttle.bytes_max_ts) +
+ ", txc max = " + stringify(throttle.transactions_observed_max) +
+ " on " + stringify(throttle.transactions_max_ts)
+ ;
},
l_bluestore_slow_committed_kv_count
);
@@ -14725,7 +14823,7 @@ void BlueStore::_kv_sync_thread()
} else {
deque<TransContext*> kv_submitting;
deque<DeferredBatch*> deferred_done, deferred_stable;
- uint64_t aios = 0, costs = 0;
+ uint64_t aios = 0, costs = 0, txcs = 0;
dout(20) << __func__ << " committing " << kv_queue.size()
<< " submitting " << kv_queue_unsubmitted.size()
@@ -14738,8 +14836,10 @@ void BlueStore::_kv_sync_thread()
deferred_stable.swap(deferred_stable_queue);
aios = kv_ios;
costs = kv_throttle_costs;
+ txcs = kv_throttle_txcs;
kv_ios = 0;
kv_throttle_costs = 0;
+ kv_throttle_txcs = 0;
l.unlock();
dout(30) << __func__ << " committing " << kv_committing << dendl;
@@ -14835,7 +14935,7 @@ void BlueStore::_kv_sync_thread()
// iteration there will already be ops awake. otherwise, we
// end up going to sleep, and then wake up when the very first
// transaction is ready for commit.
- throttle.release_kv_throttle(costs);
+ throttle.release_kv_throttle(costs, txcs);
// cleanup sync deferred keys
for (auto b : deferred_stable) {
@@ -18637,6 +18737,20 @@ bool BlueStore::BlueStoreThrottle::try_start_transaction(
TransContext &txc,
mono_clock::time_point start_throttle_acquire)
{
+ {
+ std::lock_guard l(lock);
+ auto cost0 = throttle_bytes.get_current();
+ if (cost0 + txc.cost > bytes_observed_max) {
+ bytes_observed_max = cost0 + txc.cost;
+ bytes_max_ts = ceph_clock_now();
+ }
+ auto txcs = ++transactions;
+ if (txcs > transactions_observed_max) {
+ transactions_observed_max = txcs;
+ transactions_max_ts = ceph_clock_now();
+ }
+ }
+
throttle_bytes.get(txc.cost);
if (!txc.deferred_txn || throttle_deferred_bytes.get_or_fail(txc.cost)) {
diff --git a/src/os/bluestore/BlueStore.h b/src/os/bluestore/BlueStore.h
index 207ae2ec7a2..5549f97ffea 100644
--- a/src/os/bluestore/BlueStore.h
+++ b/src/os/bluestore/BlueStore.h
@@ -1457,6 +1457,7 @@ public:
}
void rewrite_omap_key(const std::string& old, std::string *out);
+ size_t calc_userkey_offset_in_omap_key() const;
void decode_omap_key(const std::string& key, std::string *user_key);
void finish_write(TransContext* txc, uint32_t offset, uint32_t length);
@@ -1753,6 +1754,7 @@ public:
int next() override;
std::string key() override;
ceph::buffer::list value() override;
+ std::string_view value_as_sv() override;
std::string tail_key() override {
return tail;
}
@@ -2096,6 +2098,20 @@ public:
Throttle throttle_deferred_bytes; ///< submit to deferred complete
public:
+ ceph::mutex lock = ceph::make_mutex("BlueStoreThrottle::max_lock");
+
+ std::atomic<uint64_t> transactions = 0;
+
+ int64_t bytes_observed_max = 0;
+ utime_t bytes_max_ts;
+ uint64_t transactions_observed_max = 0;
+ utime_t transactions_max_ts;
+
+ uint64_t get_current() {
+ return throttle_bytes.get_current();
+ }
+
+ public:
BlueStoreThrottle(CephContext *cct) :
throttle_bytes(cct, "bluestore_throttle_bytes", 0),
throttle_deferred_bytes(cct, "bluestore_throttle_deferred_bytes", 0)
@@ -2121,8 +2137,9 @@ public:
KeyValueDB &db,
TransContext &txc,
ceph::mono_clock::time_point);
- void release_kv_throttle(uint64_t cost) {
+ void release_kv_throttle(uint64_t cost, uint64_t txcs) {
throttle_bytes.put(cost);
+ transactions -= txcs;
}
void release_deferred_throttle(uint64_t cost) {
throttle_deferred_bytes.put(cost);
@@ -2485,6 +2502,7 @@ private:
uint64_t kv_ios = 0;
uint64_t kv_throttle_costs = 0;
+ uint64_t kv_throttle_txcs = 0;
// cache trim control
uint64_t cache_size = 0; ///< total cache size
@@ -3400,15 +3418,6 @@ public:
std::map<std::string, ceph::buffer::list> *out ///< [out] Returned keys and values
) override;
-#ifdef WITH_SEASTAR
- int omap_get_values(
- CollectionHandle &c, ///< [in] Collection containing oid
- const ghobject_t &oid, ///< [in] Object containing omap
- const std::optional<std::string> &start_after, ///< [in] Keys to get
- std::map<std::string, ceph::buffer::list> *out ///< [out] Returned keys and values
- ) override;
-#endif
-
/// Filters keys into out which are defined on oid
int omap_check_keys(
CollectionHandle &c, ///< [in] Collection containing oid
@@ -3422,6 +3431,13 @@ public:
const ghobject_t &oid ///< [in] object
) override;
+ int omap_iterate(
+ CollectionHandle &c, ///< [in] collection
+ const ghobject_t &oid, ///< [in] object
+ omap_iter_seek_t start_from, ///< [in] where the iterator should point to at the beginning
+ std::function<omap_iter_ret_t(std::string_view, std::string_view)> f
+ ) override;
+
void set_fsid(uuid_d u) override {
fsid = u;
}
diff --git a/src/os/bluestore/bluefs_types.cc b/src/os/bluestore/bluefs_types.cc
index e18dd490140..fe77f7f74d8 100644
--- a/src/os/bluestore/bluefs_types.cc
+++ b/src/os/bluestore/bluefs_types.cc
@@ -154,7 +154,9 @@ mempool::bluefs::vector<bluefs_extent_t>::iterator bluefs_fnode_t::seek(
assert(it != extents_index.begin());
--it;
assert(offset >= *it);
- p += it - extents_index.begin();
+ uint32_t skip = it - extents_index.begin();
+ ceph_assert(skip <= extents.size());
+ p += skip;
offset -= *it;
}
diff --git a/src/os/bluestore/bluefs_types.h b/src/os/bluestore/bluefs_types.h
index 627118c12f8..08b3ca0cf41 100644
--- a/src/os/bluestore/bluefs_types.h
+++ b/src/os/bluestore/bluefs_types.h
@@ -89,6 +89,7 @@ struct bluefs_fnode_t {
void recalc_allocated() {
allocated = 0;
extents_index.reserve(extents.size());
+ extents_index.clear();
for (auto& p : extents) {
extents_index.emplace_back(allocated);
allocated += p.length;
diff --git a/src/os/bluestore/bluestore_tool.cc b/src/os/bluestore/bluestore_tool.cc
index d62721b4366..16f1e6434e0 100644
--- a/src/os/bluestore/bluestore_tool.cc
+++ b/src/os/bluestore/bluestore_tool.cc
@@ -1136,7 +1136,7 @@ int main(int argc, char **argv)
}
return r;
}
- } else if (action == "free-dump" || action == "free-score" || action == "fragmentation") {
+ } else if (action == "free-dump" || action == "free-score" || action == "free-fragmentation") {
AdminSocket *admin_socket = g_ceph_context->get_admin_socket();
ceph_assert(admin_socket);
std::string action_name = action == "free-dump" ? "dump" :
diff --git a/src/os/fs/btrfs_ioctl.h b/src/os/fs/btrfs_ioctl.h
deleted file mode 100644
index 277498ca8dc..00000000000
--- a/src/os/fs/btrfs_ioctl.h
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * Copyright (C) 2007 Oracle. All rights reserved.
- *
- * This program is free software; you can redistribute it and/or
- * modify it under the terms of the GNU General Public
- * License v2 as published by the Free Software Foundation.
- *
- * This program is distributed in the hope that it will be useful,
- * but WITHOUT ANY WARRANTY; without even the implied warranty of
- * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
- * General Public License for more details.
- *
- * You should have received a copy of the GNU General Public
- * License along with this program; if not, write to the
- * Free Software Foundation, Inc., 59 Temple Place - Suite 330,
- * Boston, MA 021110-1307, USA.
- */
-
-#ifndef __IOCTL_
-#define __IOCTL_
-
-#if defined(__linux__)
-#include <linux/ioctl.h>
-#elif defined(__FreeBSD__)
-#include <sys/ioctl.h>
-#endif
-
-#define BTRFS_IOCTL_MAGIC 0x94
-#define BTRFS_VOL_NAME_MAX 255
-
-/* this should be 4k */
-#define BTRFS_PATH_NAME_MAX 4087
-struct btrfs_ioctl_vol_args {
- __s64 fd;
- char name[BTRFS_PATH_NAME_MAX + 1];
-};
-
-#define BTRFS_SUBVOL_CREATE_ASYNC (1ULL << 0)
-
-#define BTRFS_SUBVOL_NAME_MAX 4039
-struct btrfs_ioctl_vol_args_v2 {
- __s64 fd;
- __u64 transid;
- __u64 flags;
- __u64 unused[4];
- char name[BTRFS_SUBVOL_NAME_MAX + 1];
-};
-
-#define BTRFS_INO_LOOKUP_PATH_MAX 4080
-struct btrfs_ioctl_ino_lookup_args {
- __u64 treeid;
- __u64 objectid;
- char name[BTRFS_INO_LOOKUP_PATH_MAX];
-};
-
-struct btrfs_ioctl_search_key {
- /* which root are we searching. 0 is the tree of tree roots */
- __u64 tree_id;
-
- /* keys returned will be >= min and <= max */
- __u64 min_objectid;
- __u64 max_objectid;
-
- /* keys returned will be >= min and <= max */
- __u64 min_offset;
- __u64 max_offset;
-
- /* max and min transids to search for */
- __u64 min_transid;
- __u64 max_transid;
-
- /* keys returned will be >= min and <= max */
- __u32 min_type;
- __u32 max_type;
-
- /*
- * how many items did userland ask for, and how many are we
- * returning
- */
- __u32 nr_items;
-
- /* align to 64 bits */
- __u32 unused;
-
- /* some extra for later */
- __u64 unused1;
- __u64 unused2;
- __u64 unused3;
- __u64 unused4;
-};
-
-struct btrfs_ioctl_search_header {
- __u64 transid;
- __u64 objectid;
- __u64 offset;
- __u32 type;
- __u32 len;
-};
-
-#define BTRFS_SEARCH_ARGS_BUFSIZE (4096 - sizeof(struct btrfs_ioctl_search_key))
-/*
- * the buf is an array of search headers where
- * each header is followed by the actual item
- * the type field is expanded to 32 bits for alignment
- */
-struct btrfs_ioctl_search_args {
- struct btrfs_ioctl_search_key key;
- char buf[BTRFS_SEARCH_ARGS_BUFSIZE];
-};
-
-struct btrfs_ioctl_clone_range_args {
- __s64 src_fd;
- __u64 src_offset, src_length;
- __u64 dest_offset;
-};
-
-/* flags for the defrag range ioctl */
-#define BTRFS_DEFRAG_RANGE_COMPRESS 1
-#define BTRFS_DEFRAG_RANGE_START_IO 2
-
-struct btrfs_ioctl_defrag_range_args {
- /* start of the defrag operation */
- __u64 start;
-
- /* number of bytes to defrag, use (u64)-1 to say all */
- __u64 len;
-
- /*
- * flags for the operation, which can include turning
- * on compression for this one defrag
- */
- __u64 flags;
-
- /*
- * any extent bigger than this will be considered
- * already defragged. Use 0 to take the kernel default
- * Use 1 to say every single extent must be rewritten
- */
- __u32 extent_thresh;
-
- /* spare for later */
- __u32 unused[5];
-};
-
-struct btrfs_ioctl_space_info {
- __u64 flags;
- __u64 total_bytes;
- __u64 used_bytes;
-};
-
-struct btrfs_ioctl_space_args {
- __u64 space_slots;
- __u64 total_spaces;
- struct btrfs_ioctl_space_info spaces[0];
-};
-
-#define BTRFS_IOC_SNAP_CREATE _IOW(BTRFS_IOCTL_MAGIC, 1, \
- struct btrfs_ioctl_vol_args)
-#define BTRFS_IOC_DEFRAG _IOW(BTRFS_IOCTL_MAGIC, 2, \
- struct btrfs_ioctl_vol_args)
-#define BTRFS_IOC_RESIZE _IOW(BTRFS_IOCTL_MAGIC, 3, \
- struct btrfs_ioctl_vol_args)
-#define BTRFS_IOC_SCAN_DEV _IOW(BTRFS_IOCTL_MAGIC, 4, \
- struct btrfs_ioctl_vol_args)
-/* trans start and trans end are dangerous, and only for
- * use by applications that know how to avoid the
- * resulting deadlocks
- */
-#define BTRFS_IOC_TRANS_START _IO(BTRFS_IOCTL_MAGIC, 6)
-#define BTRFS_IOC_TRANS_END _IO(BTRFS_IOCTL_MAGIC, 7)
-#define BTRFS_IOC_SYNC _IO(BTRFS_IOCTL_MAGIC, 8)
-
-#define BTRFS_IOC_CLONE _IOW(BTRFS_IOCTL_MAGIC, 9, int)
-#define BTRFS_IOC_ADD_DEV _IOW(BTRFS_IOCTL_MAGIC, 10, \
- struct btrfs_ioctl_vol_args)
-#define BTRFS_IOC_RM_DEV _IOW(BTRFS_IOCTL_MAGIC, 11, \
- struct btrfs_ioctl_vol_args)
-#define BTRFS_IOC_BALANCE _IOW(BTRFS_IOCTL_MAGIC, 12, \
- struct btrfs_ioctl_vol_args)
-
-#define BTRFS_IOC_CLONE_RANGE _IOW(BTRFS_IOCTL_MAGIC, 13, \
- struct btrfs_ioctl_clone_range_args)
-
-#define BTRFS_IOC_SUBVOL_CREATE _IOW(BTRFS_IOCTL_MAGIC, 14, \
- struct btrfs_ioctl_vol_args)
-#define BTRFS_IOC_SNAP_DESTROY _IOW(BTRFS_IOCTL_MAGIC, 15, \
- struct btrfs_ioctl_vol_args)
-#define BTRFS_IOC_DEFRAG_RANGE _IOW(BTRFS_IOCTL_MAGIC, 16, \
- struct btrfs_ioctl_defrag_range_args)
-#define BTRFS_IOC_TREE_SEARCH _IOWR(BTRFS_IOCTL_MAGIC, 17, \
- struct btrfs_ioctl_search_args)
-#define BTRFS_IOC_INO_LOOKUP _IOWR(BTRFS_IOCTL_MAGIC, 18, \
- struct btrfs_ioctl_ino_lookup_args)
-#define BTRFS_IOC_DEFAULT_SUBVOL _IOW(BTRFS_IOCTL_MAGIC, 19, u64)
-#define BTRFS_IOC_SPACE_INFO _IOWR(BTRFS_IOCTL_MAGIC, 20, \
- struct btrfs_ioctl_space_args)
-#define BTRFS_IOC_START_SYNC _IOR(BTRFS_IOCTL_MAGIC, 24, __u64)
-#define BTRFS_IOC_WAIT_SYNC _IOW(BTRFS_IOCTL_MAGIC, 22, __u64)
-#define BTRFS_IOC_SNAP_CREATE_V2 _IOW(BTRFS_IOCTL_MAGIC, 23, \
- struct btrfs_ioctl_vol_args_v2)
-#endif
diff --git a/src/os/kstore/KStore.cc b/src/os/kstore/KStore.cc
index 7158486ca38..a069d429155 100644
--- a/src/os/kstore/KStore.cc
+++ b/src/os/kstore/KStore.cc
@@ -1651,6 +1651,13 @@ bufferlist KStore::OmapIteratorImpl::value()
return it->value();
}
+std::string_view KStore::OmapIteratorImpl::value_as_sv()
+{
+ std::shared_lock l{c->lock};
+ ceph_assert(it->valid());
+ return it->value_as_sv();
+}
+
int KStore::omap_get(
CollectionHandle& ch, ///< [in] Collection containing oid
const ghobject_t &oid, ///< [in] Object containing omap
@@ -1866,6 +1873,71 @@ ObjectMap::ObjectMapIterator KStore::get_omap_iterator(
return ObjectMap::ObjectMapIterator(new OmapIteratorImpl(c, o, it));
}
+int KStore::omap_iterate(
+ CollectionHandle &ch, ///< [in] collection
+ const ghobject_t &oid, ///< [in] object
+ ObjectStore::omap_iter_seek_t start_from, ///< [in] where the iterator should point to at the beginning
+ std::function<omap_iter_ret_t(std::string_view, std::string_view)> f)
+{
+ dout(10) << __func__ << " " << ch->cid << " " << oid << dendl;
+ Collection *c = static_cast<Collection*>(ch.get());
+ {
+ std::shared_lock l{c->lock};
+
+ OnodeRef o = c->get_onode(oid, false);
+ if (!o || !o->exists) {
+ dout(10) << __func__ << " " << oid << "doesn't exist" <<dendl;
+ return -ENOENT;
+ }
+ o->flush();
+ dout(10) << __func__ << " header = " << o->onode.omap_head <<dendl;
+
+ KeyValueDB::Iterator it = db->get_iterator(PREFIX_OMAP);
+ std::string tail;
+ std::string seek_key;
+ if (o->onode.omap_head) {
+ return 0; // nothing to do
+ }
+
+ // acquire data depedencies for seek & iterate
+ get_omap_key(o->onode.omap_head, start_from.seek_position, &seek_key);
+ get_omap_tail(o->onode.omap_head, &tail);
+
+ // acquire the iterator
+ {
+ it = db->get_iterator(PREFIX_OMAP);
+ }
+
+ // seek the iterator
+ {
+ if (start_from.seek_type == omap_iter_seek_t::LOWER_BOUND) {
+ it->lower_bound(seek_key);
+ } else {
+ it->upper_bound(seek_key);
+ }
+ }
+
+ // iterate!
+ while (it->valid()) {
+ std::string user_key;
+ if (const auto& db_key = it->raw_key().second; db_key >= tail) {
+ break;
+ } else {
+ decode_omap_key(db_key, &user_key);
+ }
+ omap_iter_ret_t ret = f(user_key, it->value_as_sv());
+ if (ret == omap_iter_ret_t::STOP) {
+ break;
+ } else if (ret == omap_iter_ret_t::NEXT) {
+ it->next();
+ } else {
+ ceph_abort();
+ }
+ }
+ }
+ return 0;
+}
+
// -----------------
// write helpers
diff --git a/src/os/kstore/KStore.h b/src/os/kstore/KStore.h
index 9a9d413c66a..06115d3cab7 100644
--- a/src/os/kstore/KStore.h
+++ b/src/os/kstore/KStore.h
@@ -180,6 +180,7 @@ public:
int next() override;
std::string key() override;
ceph::buffer::list value() override;
+ std::string_view value_as_sv() override;
int status() override {
return 0;
}
@@ -553,6 +554,13 @@ public:
const ghobject_t &oid ///< [in] object
) override;
+ int omap_iterate(
+ CollectionHandle &c, ///< [in] collection
+ const ghobject_t &oid, ///< [in] object
+ omap_iter_seek_t start_from, ///< [in] where the iterator should point to at the beginning
+ std::function<omap_iter_ret_t(std::string_view, std::string_view)> f
+ ) override;
+
void set_fsid(uuid_d u) override {
fsid = u;
}
diff --git a/src/os/memstore/MemStore.cc b/src/os/memstore/MemStore.cc
index 89cb09361cf..f9d3bf0d8a2 100644
--- a/src/os/memstore/MemStore.cc
+++ b/src/os/memstore/MemStore.cc
@@ -537,30 +537,6 @@ int MemStore::omap_get_values(
return 0;
}
-#ifdef WITH_SEASTAR
-int MemStore::omap_get_values(
- CollectionHandle& ch, ///< [in] Collection containing oid
- const ghobject_t &oid, ///< [in] Object containing omap
- const std::optional<std::string> &start_after, ///< [in] Keys to get
- std::map<std::string, ceph::buffer::list> *out ///< [out] Returned keys and values
- )
-{
- dout(10) << __func__ << " " << ch->cid << " " << oid << dendl;
- Collection *c = static_cast<Collection*>(ch.get());
- ObjectRef o = c->get_object(oid);
- if (!o)
- return -ENOENT;
- assert(start_after);
- std::lock_guard lock{o->omap_mutex};
- for (auto it = o->omap.upper_bound(*start_after);
- it != std::end(o->omap);
- ++it) {
- out->insert(*it);
- }
- return 0;
-}
-#endif
-
int MemStore::omap_check_keys(
CollectionHandle& ch, ///< [in] Collection containing oid
const ghobject_t &oid, ///< [in] Object containing omap
@@ -622,6 +598,10 @@ public:
std::lock_guard lock{o->omap_mutex};
return it->second;
}
+ std::string_view value_as_sv() override {
+ std::lock_guard lock{o->omap_mutex};
+ return std::string_view{it->second.c_str(), it->second.length()};
+ }
int status() override {
return 0;
}
@@ -639,6 +619,48 @@ ObjectMap::ObjectMapIterator MemStore::get_omap_iterator(
return ObjectMap::ObjectMapIterator(new OmapIteratorImpl(c, o));
}
+int MemStore::omap_iterate(
+ CollectionHandle &ch, ///< [in] collection
+ const ghobject_t &oid, ///< [in] object
+ ObjectStore::omap_iter_seek_t start_from, ///< [in] where the iterator should point to at the beginning
+ std::function<omap_iter_ret_t(std::string_view, std::string_view)> f)
+{
+ Collection *c = static_cast<Collection*>(ch.get());
+ ObjectRef o = c->get_object(oid);
+ if (!o) {
+ return -ENOENT;
+ }
+
+ {
+ std::lock_guard lock{o->omap_mutex};
+
+ // obtain seek the iterator
+ decltype(o->omap)::iterator it;
+ {
+ if (start_from.seek_type == omap_iter_seek_t::LOWER_BOUND) {
+ it = o->omap.lower_bound(start_from.seek_position);
+ } else {
+ it = o->omap.upper_bound(start_from.seek_position);
+ }
+ }
+
+ // iterate!
+ while (it != o->omap.end()) {
+ // potentially rectifying memcpy but who cares for memstore?
+ omap_iter_ret_t ret =
+ f(it->first, std::string_view{it->second.c_str(), it->second.length()});
+ if (ret == omap_iter_ret_t::STOP) {
+ break;
+ } else if (ret == omap_iter_ret_t::NEXT) {
+ ++it;
+ } else {
+ ceph_abort();
+ }
+ }
+ }
+ return 0;
+}
+
// ---------------
// write operations
diff --git a/src/os/memstore/MemStore.h b/src/os/memstore/MemStore.h
index 2abe552891f..9621773598f 100644
--- a/src/os/memstore/MemStore.h
+++ b/src/os/memstore/MemStore.h
@@ -363,14 +363,6 @@ public:
const std::set<std::string> &keys, ///< [in] Keys to get
std::map<std::string, ceph::buffer::list> *out ///< [out] Returned keys and values
) override;
-#ifdef WITH_SEASTAR
- int omap_get_values(
- CollectionHandle &c, ///< [in] Collection containing oid
- const ghobject_t &oid, ///< [in] Object containing omap
- const std::optional<std::string> &start_after, ///< [in] Keys to get
- std::map<std::string, ceph::buffer::list> *out ///< [out] Returned keys and values
- ) override;
-#endif
using ObjectStore::omap_check_keys;
/// Filters keys into out which are defined on oid
@@ -387,6 +379,13 @@ public:
const ghobject_t &oid ///< [in] object
) override;
+ int omap_iterate(
+ CollectionHandle &c, ///< [in] collection
+ const ghobject_t &oid, ///< [in] object
+ omap_iter_seek_t start_from, ///< [in] where the iterator should point to at the beginning
+ std::function<omap_iter_ret_t(std::string_view, std::string_view)> f
+ ) override;
+
void set_fsid(uuid_d u) override;
uuid_d get_fsid() override;