summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorPatrick Donnelly <pdonnell@redhat.com>2021-07-29 19:35:13 -0700
committerPatrick Donnelly <pdonnell@redhat.com>2021-07-29 19:35:13 -0700
commit8d4b767595d669fb872bea6ce18f921995c3a519 (patch)
tree1c7ca5268e42cbef06bd478465e3954536a756ae
parentd85c152d182151361c7384b2863c1793d41d7aa7 (diff)
parent75661374ff7fa14bf9bc053b9389571d8aa68f7a (diff)
downloadceph-8d4b767595d669fb872bea6ce18f921995c3a519.tar.gz
Merge PR #42537 into octopus
* refs/pull/42537/head: mon/MDSMonitor: propose if FSMap struct_v is too old mon/MDSMonitor: give a proper error message if FSMap struct_v is too old qa: add tests for fs dump of epoch and trimming qa: add file system support for dumping epoch mon/MDSMonitor: return mon_mds_force_trim_to even if equal to current epoch mon: add debugging for trimming methods mon: fix debug spacing Reviewed-by: Ramana Raja <rraja@redhat.com> Reviewed-by: Neha Ojha <nojha@redhat.com>
-rw-r--r--qa/tasks/cephfs/filesystem.py11
-rw-r--r--qa/tasks/cephfs/test_admin.py75
-rw-r--r--src/mds/FSMap.cc5
-rw-r--r--src/mds/FSMap.h14
-rw-r--r--src/mon/MDSMonitor.cc44
-rw-r--r--src/mon/MDSMonitor.h7
-rw-r--r--src/mon/PaxosService.cc18
7 files changed, 157 insertions, 17 deletions
diff --git a/qa/tasks/cephfs/filesystem.py b/qa/tasks/cephfs/filesystem.py
index 7f01b0ff480..0c02e2aa3d7 100644
--- a/qa/tasks/cephfs/filesystem.py
+++ b/qa/tasks/cephfs/filesystem.py
@@ -63,9 +63,12 @@ class FSStatus(object):
"""
Operations on a snapshot of the FSMap.
"""
- def __init__(self, mon_manager):
+ def __init__(self, mon_manager, epoch=None):
self.mon = mon_manager
- self.map = json.loads(self.mon.raw_cluster_cmd("fs", "dump", "--format=json"))
+ cmd = ["fs", "dump", "--format=json"]
+ if epoch is not None:
+ cmd.append(str(epoch))
+ self.map = json.loads(self.mon.raw_cluster_cmd(*cmd))
def __str__(self):
return json.dumps(self.map, indent = 2, sort_keys = True)
@@ -333,8 +336,8 @@ class MDSCluster(CephCluster):
def newfs(self, name='cephfs', create=True):
return Filesystem(self._ctx, name=name, create=create)
- def status(self):
- return FSStatus(self.mon_manager)
+ def status(self, epoch=None):
+ return FSStatus(self.mon_manager, epoch)
def delete_all_filesystems(self):
"""
diff --git a/qa/tasks/cephfs/test_admin.py b/qa/tasks/cephfs/test_admin.py
index 60198604b39..879dc05687c 100644
--- a/qa/tasks/cephfs/test_admin.py
+++ b/qa/tasks/cephfs/test_admin.py
@@ -1,4 +1,7 @@
+import errno
import json
+import logging
+import time
from teuthology.orchestra.run import CommandFailedError
@@ -7,6 +10,7 @@ from tasks.cephfs.fuse_mount import FuseMount
from tasks.cephfs.filesystem import FileLayout
+log = logging.getLogger(__name__)
class TestAdminCommands(CephFSTestCase):
"""
@@ -169,6 +173,77 @@ class TestAdminCommands(CephFSTestCase):
pool_names[i], 'cephfs', keys[i], fs_name)
+class TestDump(CephFSTestCase):
+ CLIENTS_REQUIRED = 0
+ MDSS_REQUIRED = 1
+
+ def test_fs_dump_epoch(self):
+ """
+ That dumping a specific epoch works.
+ """
+
+ status1 = self.fs.status()
+ status2 = self.fs.status(epoch=status1["epoch"]-1)
+ self.assertEqual(status1["epoch"], status2["epoch"]+1)
+
+ def test_fsmap_trim(self):
+ """
+ That the fsmap is trimmed normally.
+ """
+
+ paxos_service_trim_min = 25
+ self.config_set('mon', 'paxos_service_trim_min', paxos_service_trim_min)
+ mon_max_mdsmap_epochs = 20
+ self.config_set('mon', 'mon_max_mdsmap_epochs', mon_max_mdsmap_epochs)
+
+ status = self.fs.status()
+ epoch = status["epoch"]
+
+ # for N mutations
+ mutations = paxos_service_trim_min + mon_max_mdsmap_epochs
+ b = False
+ for i in range(mutations):
+ self.fs.set_joinable(b)
+ b = not b
+
+ time.sleep(10) # for tick/compaction
+
+ try:
+ self.fs.status(epoch=epoch)
+ except CommandFailedError as e:
+ self.assertEqual(e.exitstatus, errno.ENOENT, "invalid error code when trying to fetch FSMap that was trimmed")
+ else:
+ self.fail("trimming did not occur as expected")
+
+ def test_fsmap_force_trim(self):
+ """
+ That the fsmap is trimmed forcefully.
+ """
+
+ status = self.fs.status()
+ epoch = status["epoch"]
+
+ paxos_service_trim_min = 1
+ self.config_set('mon', 'paxos_service_trim_min', paxos_service_trim_min)
+ mon_mds_force_trim_to = epoch+1
+ self.config_set('mon', 'mon_mds_force_trim_to', mon_mds_force_trim_to)
+
+ # force a new fsmap
+ self.fs.set_joinable(False)
+ time.sleep(10) # for tick/compaction
+
+ status = self.fs.status()
+ log.debug(f"new epoch is {status['epoch']}")
+ self.fs.status(epoch=epoch+1) # epoch+1 is not trimmed, may not == status["epoch"]
+
+ try:
+ self.fs.status(epoch=epoch)
+ except CommandFailedError as e:
+ self.assertEqual(e.exitstatus, errno.ENOENT, "invalid error code when trying to fetch FSMap that was trimmed")
+ else:
+ self.fail("trimming did not occur as expected")
+
+
class TestConfigCommands(CephFSTestCase):
"""
Test that daemons and clients respond to the otherwise rarely-used
diff --git a/src/mds/FSMap.cc b/src/mds/FSMap.cc
index 8f4cffa43a0..f8bebc0507d 100644
--- a/src/mds/FSMap.cc
+++ b/src/mds/FSMap.cc
@@ -472,7 +472,7 @@ void FSMap::update_compat(const CompatSet &c)
void FSMap::encode(bufferlist& bl, uint64_t features) const
{
- ENCODE_START(7, 6, bl);
+ ENCODE_START(STRUCT_VERSION, 6, bl);
encode(epoch, bl);
encode(next_filesystem_id, bl);
encode(legacy_client_fscid, bl);
@@ -497,7 +497,8 @@ void FSMap::decode(bufferlist::const_iterator& p)
// MDSMonitor to store an FSMap instead of an MDSMap was
// 5, so anything older than 6 is decoded as an MDSMap,
// and anything newer is decoded as an FSMap.
- DECODE_START_LEGACY_COMPAT_LEN_16(7, 4, 4, p);
+ DECODE_START_LEGACY_COMPAT_LEN_16(STRUCT_VERSION, 4, 4, p);
+ struct_version = struct_v;
if (struct_v < 6) {
// Because the mon used to store an MDSMap where we now
// store an FSMap, FSMap knows how to decode the legacy
diff --git a/src/mds/FSMap.h b/src/mds/FSMap.h
index feed962c818..72e57231d0e 100644
--- a/src/mds/FSMap.h
+++ b/src/mds/FSMap.h
@@ -88,6 +88,9 @@ public:
friend class PaxosFSMap;
using mds_info_t = MDSMap::mds_info_t;
+ static const version_t STRUCT_VERSION = 7;
+ static const version_t STRUCT_VERSION_TRIM_TO = 7;
+
FSMap() : compat(MDSMap::get_compat_set_default()) {}
FSMap(const FSMap &rhs)
@@ -100,7 +103,8 @@ public:
ever_enabled_multiple(rhs.ever_enabled_multiple),
mds_roles(rhs.mds_roles),
standby_daemons(rhs.standby_daemons),
- standby_epochs(rhs.standby_epochs)
+ standby_epochs(rhs.standby_epochs),
+ struct_version(rhs.struct_version)
{
filesystems.clear();
for (const auto &i : rhs.filesystems) {
@@ -339,6 +343,11 @@ public:
epoch_t get_epoch() const { return epoch; }
void inc_epoch() { epoch++; }
+ version_t get_struct_version() const { return struct_version; }
+ bool is_struct_old() const {
+ return struct_version < STRUCT_VERSION_TRIM_TO;
+ }
+
size_t filesystem_count() const {return filesystems.size();}
bool filesystem_exists(fs_cluster_id_t fscid) const {return filesystems.count(fscid) > 0;}
Filesystem::const_ref get_filesystem(fs_cluster_id_t fscid) const {return std::const_pointer_cast<const Filesystem>(filesystems.at(fscid));}
@@ -409,6 +418,9 @@ protected:
// For MDS daemons not yet assigned to a Filesystem
std::map<mds_gid_t, mds_info_t> standby_daemons;
std::map<mds_gid_t, epoch_t> standby_epochs;
+
+private:
+ epoch_t struct_version = 0;
};
WRITE_CLASS_ENCODER_FEATURES(FSMap)
diff --git a/src/mon/MDSMonitor.cc b/src/mon/MDSMonitor.cc
index 7ff5f889e69..21604acb847 100644
--- a/src/mon/MDSMonitor.cc
+++ b/src/mon/MDSMonitor.cc
@@ -120,7 +120,12 @@ void MDSMonitor::update_from_paxos(bool *need_bootstrap)
ceph_assert(fsmap_bl.length() > 0);
dout(10) << __func__ << " got " << version << dendl;
- PaxosFSMap::decode(fsmap_bl);
+ try {
+ PaxosFSMap::decode(fsmap_bl);
+ } catch (const ceph::buffer::malformed_input& e) {
+ derr << "unable to decode FSMap: " << e.what() << dendl;
+ throw;
+ }
// new map
dout(0) << "new map" << dendl;
@@ -264,7 +269,7 @@ version_t MDSMonitor::get_trim_to() const
{
version_t floor = 0;
if (g_conf()->mon_mds_force_trim_to > 0 &&
- g_conf()->mon_mds_force_trim_to < (int)get_last_committed()) {
+ g_conf()->mon_mds_force_trim_to <= (int)get_last_committed()) {
floor = g_conf()->mon_mds_force_trim_to;
dout(10) << __func__ << " explicit mon_mds_force_trim_to = "
<< floor << dendl;
@@ -273,8 +278,11 @@ version_t MDSMonitor::get_trim_to() const
unsigned max = g_conf()->mon_max_mdsmap_epochs;
version_t last = get_last_committed();
- if (last - get_first_committed() > max && floor < last - max)
- return last - max;
+ if (last - get_first_committed() > max && floor < last - max) {
+ floor = last-max;
+ }
+
+ dout(20) << __func__ << " = " << floor << dendl;
return floor;
}
@@ -2171,6 +2179,34 @@ void MDSMonitor::tick()
bool do_propose = false;
bool propose_osdmap = false;
+ if (check_fsmap_struct_version) {
+ /* Allow time for trimming otherwise PaxosService::is_writeable will always
+ * be false.
+ */
+
+ auto now = clock::now();
+ auto elapsed = now - last_fsmap_struct_flush;
+ if (elapsed > std::chrono::seconds(30)) {
+ FSMap fsmap;
+ bufferlist bl;
+ auto v = get_first_committed();
+ int err = get_version(v, bl);
+ if (err) {
+ derr << "could not get version " << v << dendl;
+ ceph_abort();
+ }
+ fsmap.decode(bl);
+ if (fsmap.is_struct_old()) {
+ dout(5) << "fsmap struct is too old; proposing to flush out old versions" << dendl;
+ do_propose = true;
+ last_fsmap_struct_flush = now;
+ } else {
+ dout(20) << "struct is recent" << dendl;
+ check_fsmap_struct_version = false;
+ }
+ }
+ }
+
do_propose |= pending.check_health();
/* Check health and affinity of ranks */
diff --git a/src/mon/MDSMonitor.h b/src/mon/MDSMonitor.h
index 56762a8afb7..51be614dae1 100644
--- a/src/mon/MDSMonitor.h
+++ b/src/mon/MDSMonitor.h
@@ -32,6 +32,9 @@ class FileSystemCommandHandler;
class MDSMonitor : public PaxosService, public PaxosFSMap, protected CommandHandler {
public:
+ using clock = ceph::coarse_mono_clock;
+ using time = ceph::coarse_mono_time;
+
MDSMonitor(Monitor *mn, Paxos *p, string service_name);
// service methods
@@ -142,6 +145,10 @@ protected:
// when the mon was not updating us for some period (e.g. during slow
// election) to reset last_beacon timeouts
mono_time last_tick = mono_clock::zero();
+
+private:
+ time last_fsmap_struct_flush = clock::zero();
+ bool check_fsmap_struct_version = true;
};
#endif
diff --git a/src/mon/PaxosService.cc b/src/mon/PaxosService.cc
index dc994e194eb..1acadebbea6 100644
--- a/src/mon/PaxosService.cc
+++ b/src/mon/PaxosService.cc
@@ -372,11 +372,17 @@ void PaxosService::maybe_trim()
if (!is_writeable())
return;
+ const version_t first_committed = get_first_committed();
version_t trim_to = get_trim_to();
- if (trim_to < get_first_committed())
+ dout(20) << __func__ << " " << first_committed << "~" << trim_to << dendl;
+
+ if (trim_to < first_committed) {
+ dout(10) << __func__ << " trim_to " << trim_to << " < first_committed "
+ << first_committed << dendl;
return;
+ }
- version_t to_remove = trim_to - get_first_committed();
+ version_t to_remove = trim_to - first_committed;
const version_t trim_min = g_conf().get_val<version_t>("paxos_service_trim_min");
if (trim_min > 0 &&
to_remove < trim_min) {
@@ -385,13 +391,13 @@ void PaxosService::maybe_trim()
return;
}
- to_remove = [to_remove, this] {
+ to_remove = [to_remove, trim_to, this] {
const version_t trim_max = g_conf().get_val<version_t>("paxos_service_trim_max");
if (trim_max == 0 || to_remove < trim_max) {
return to_remove;
}
if (to_remove < trim_max * 1.5) {
- dout(10) << __func__ << " trim to " << get_trim_to() << " would only trim " << to_remove
+ dout(10) << __func__ << " trim to " << trim_to << " would only trim " << to_remove
<< " > paxos_service_trim_max, limiting to " << trim_max
<< dendl;
return trim_max;
@@ -404,11 +410,11 @@ void PaxosService::maybe_trim()
return new_trim_max;
}
}();
- trim_to = get_first_committed() + to_remove;
+ trim_to = first_committed + to_remove;
dout(10) << __func__ << " trimming to " << trim_to << ", " << to_remove << " states" << dendl;
MonitorDBStore::TransactionRef t = paxos->get_pending_transaction();
- trim(t, get_first_committed(), trim_to);
+ trim(t, first_committed, trim_to);
put_first_committed(t, trim_to);
cached_first_committed = trim_to;