#endif
// okay, try FileStore (journal).
- r = FileStore::get_block_device_fsid(path, fsid);
+ r = FileStore::get_block_device_fsid(cct, path, fsid);
if (r == 0) {
lgeneric_dout(cct, 0) << __func__ << " " << path << " is filestore, "
<< *fsid << dendl;
#if defined(__linux__)
-#define dout_context g_ceph_context
+#define dout_context cct()
#define dout_subsys ceph_subsys_filestore
#undef dout_prefix
#define dout_prefix *_dout << "btrfsfilestorebackend(" << get_basedir_path() << ") "
GenericFileStoreBackend(fs), has_clone_range(false),
has_snap_create(false), has_snap_destroy(false),
has_snap_create_v2(false), has_wait_sync(false), stable_commits(false),
- m_filestore_btrfs_clone_range(g_conf->filestore_btrfs_clone_range),
- m_filestore_btrfs_snap (g_conf->filestore_btrfs_snap) { }
+ m_filestore_btrfs_clone_range(cct()->_conf->filestore_btrfs_clone_range),
+ m_filestore_btrfs_snap (cct()->_conf->filestore_btrfs_snap) { }
int BtrfsFileStoreBackend::detect_features()
{
#include "common/RWLock.h"
/**
- * CollectionIndex provides an interface for manipulating indexed collections
+ CollectionIndex provides an interface for manipulating indexed collections
*/
class CollectionIndex {
+public:
+ CephContext* cct;
protected:
/**
* Object encapsulating a returned path.
/// Call prior to removing directory
virtual int prep_delete() { return 0; }
- explicit CollectionIndex(const coll_t& collection):
- access_lock("CollectionIndex::access_lock", true, false) {}
+ CollectionIndex(CephContext* cct, const coll_t& collection)
+ : cct(cct), access_lock("CollectionIndex::access_lock", true, false) {}
/*
* Pre-hash the collection, this collection should map to a PG folder.
#include "common/config.h"
#include "include/assert.h"
-#define dout_context g_ceph_context
+#define dout_context cct
#define dout_subsys ceph_subsys_filestore
#undef dout_prefix
#define dout_prefix *_dout << "filestore "
// bad: plana8923501-10...4c.3.ffffffffffffffff.2
// fixed: plana8923501-10...4c.3.CB767F2D.ffffffffffffffff.2
// returns 0 for false, 1 for true, negative for error
-int DBObjectMap::is_buggy_ghobject_key_v1(const string &in)
+int DBObjectMap::is_buggy_ghobject_key_v1(CephContext* cct,
+ const string &in)
{
int dots = 5; // skip 5 .'s
const char *s = in.c_str();
iter->valid() && count < 300;
iter->next()) {
dout(20) << __func__ << " key is " << iter->key() << dendl;
- int r = is_buggy_ghobject_key_v1(iter->key());
+ int r = is_buggy_ghobject_key_v1(cct, iter->key());
if (r < 0) {
derr << __func__ << " bad key '" << iter->key() << "'" << dendl;
return r;
/// String munging (public for testing)
static string ghobject_key(const ghobject_t &oid);
static string ghobject_key_v0(coll_t c, const ghobject_t &oid);
- static int is_buggy_ghobject_key_v1(const string &in);
+ static int is_buggy_ghobject_key_v1(CephContext* cct,
+ const string &in);
private:
/// Implicit lock on Header->seq
typedef ceph::shared_ptr<_Header> Header;
#define O_DSYNC O_SYNC
#endif
-#define dout_context g_ceph_context
+#define dout_context cct
#define dout_subsys ceph_subsys_journal
#undef dout_prefix
#define dout_prefix *_dout << "journal "
<< dendl;
max_size = bdev_sz;
- block_size = g_conf->journal_block_size;
+ block_size = cct->_conf->journal_block_size;
- if (g_conf->journal_discard) {
+ if (cct->_conf->journal_discard) {
discard = block_device_support_discard(fn.c_str());
dout(10) << fn << " support discard: " << (int)discard << dendl;
}
bool create)
{
int ret;
- int64_t conf_journal_sz(g_conf->osd_journal_size);
+ int64_t conf_journal_sz(cct->_conf->osd_journal_size);
conf_journal_sz <<= 20;
- if ((g_conf->osd_journal_size == 0) && (oldsize < ONE_MEG)) {
+ if ((cct->_conf->osd_journal_size == 0) && (oldsize < ONE_MEG)) {
derr << "I'm sorry, I don't know how large of a journal to create."
<< "Please specify a block device to use as the journal OR "
<< "set osd_journal_size in your ceph.conf" << dendl;
}
if (create && (oldsize < conf_journal_sz)) {
- uint64_t newsize(g_conf->osd_journal_size);
+ uint64_t newsize(cct->_conf->osd_journal_size);
newsize <<= 20;
dout(10) << "_open extending to " << newsize << " bytes" << dendl;
ret = ::ftruncate(fd, newsize);
else {
max_size = oldsize;
}
- block_size = g_conf->journal_block_size;
+ block_size = cct->_conf->journal_block_size;
- if (create && g_conf->journal_zero_on_create) {
+ if (create && cct->_conf->journal_zero_on_create) {
derr << "FileJournal::_open_file : zeroing journal" << dendl;
uint64_t write_size = 1 << 20;
char *buf;
header.fsid = fsid;
header.max_size = max_size;
header.block_size = block_size;
- if (g_conf->journal_block_align || directio)
+ if (cct->_conf->journal_block_align || directio)
header.alignment = block_size;
else
header.alignment = 16; // at least stay word aligned on 64bit machines...
goto free_buf;
}
- needed_space = ((int64_t)g_conf->osd_max_write_size) << 20;
+ needed_space = ((int64_t)cct->_conf->osd_max_write_size) << 20;
needed_space += (2 * sizeof(entry_header_t)) + get_top();
if (header.max_size - header.start < needed_space) {
derr << "FileJournal::create: OSD journal is not large enough to hold "
// gather queued writes
off64_t queue_pos = write_pos;
- int eleft = g_conf->journal_max_write_entries;
- unsigned bmax = g_conf->journal_max_write_bytes;
+ int eleft = cct->_conf->journal_max_write_entries;
+ unsigned bmax = cct->_conf->journal_max_write_bytes;
if (full_state != FULL_NOTFULL)
return -ENOSPC;
}
if (eleft) {
if (--eleft == 0) {
- dout(20) << "prepare_multi_write hit max events per write " << g_conf->journal_max_write_entries << dendl;
+ dout(20) << "prepare_multi_write hit max events per write "
+ << cct->_conf->journal_max_write_entries << dendl;
batch_unpop_write(items);
goto out;
}
}
if (bmax) {
if (bl.length() >= bmax) {
- dout(20) << "prepare_multi_write hit max write size " << g_conf->journal_max_write_bytes << dendl;
+ dout(20) << "prepare_multi_write hit max write size "
+ << cct->_conf->journal_max_write_bytes << dendl;
batch_unpop_write(items);
goto out;
}
return;
buffer::ptr hbp;
- if (g_conf->journal_write_header_frequency &&
+ if (cct->_conf->journal_write_header_frequency &&
(((++journaled_since_start) %
- g_conf->journal_write_header_frequency) == 0)) {
+ cct->_conf->journal_write_header_frequency) == 0)) {
must_write_header = true;
}
ceph_abort();
}
#ifdef HAVE_POSIX_FADVISE
- if (g_conf->filestore_fadvise)
+ if (cct->_conf->filestore_fadvise)
posix_fadvise(fd, 0, 0, POSIX_FADV_DONTNEED);
#endif
}
void FileJournal::do_aio_write(bufferlist& bl)
{
- if (g_conf->journal_write_header_frequency &&
+ if (cct->_conf->journal_write_header_frequency &&
(((++journaled_since_start) %
- g_conf->journal_write_header_frequency) == 0)) {
+ cct->_conf->journal_write_header_frequency) == 0)) {
must_write_header = true;
}
int FileJournal::prepare_entry(vector<ObjectStore::Transaction>& tls, bufferlist* tbl) {
dout(10) << "prepare_entry " << tls << dendl;
- int data_len = g_conf->journal_align_min_size - 1;
+ int data_len = cct->_conf->journal_align_min_size - 1;
int data_align = -1; // -1 indicates that we don't care about the alignment
bufferlist bl;
for (vector<ObjectStore::Transaction>::iterator p = tls.begin();
{
stringstream ss;
bool valid = throttle.set_params(
- g_conf->journal_throttle_low_threshhold,
- g_conf->journal_throttle_high_threshhold,
- g_conf->filestore_expected_throughput_bytes,
- g_conf->journal_throttle_high_multiple,
- g_conf->journal_throttle_max_multiple,
+ cct->_conf->journal_throttle_low_threshhold,
+ cct->_conf->journal_throttle_high_threshhold,
+ cct->_conf->filestore_expected_throughput_bytes,
+ cct->_conf->journal_throttle_high_multiple,
+ cct->_conf->journal_throttle_max_multiple,
header.max_size - get_top(),
&ss);
derr << "Unable to read past sequence " << seq
<< " but header indicates the journal has committed up through "
<< header.committed_up_to << ", journal is corrupt" << dendl;
- if (g_conf->journal_ignore_corruption) {
+ if (cct->_conf->journal_ignore_corruption) {
if (corrupt)
*corrupt = true;
return false;
# include <libaio.h>
#endif
-#define dout_context g_ceph_context
-
-
/**
* Implements journaling on top of block device or file.
*
}
public:
- FileJournal(uuid_d fsid, Finisher *fin, Cond *sync_cond, const char *f, bool dio=false, bool ai=true, bool faio=false) :
- Journal(fsid, fin, sync_cond),
- finisher_lock("FileJournal::finisher_lock", false, true, false, g_ceph_context),
+ FileJournal(CephContext* cct, uuid_d fsid, Finisher *fin, Cond *sync_cond,
+ const char *f, bool dio=false, bool ai=true, bool faio=false) :
+ Journal(cct, fsid, fin, sync_cond),
+ finisher_lock("FileJournal::finisher_lock", false, true, false, cct),
journaled_seq(0),
plug_journal_completions(false),
- writeq_lock("FileJournal::writeq_lock", false, true, false, g_ceph_context),
+ writeq_lock("FileJournal::writeq_lock", false, true, false, cct),
completions_lock(
- "FileJournal::completions_lock", false, true, false, g_ceph_context),
+ "FileJournal::completions_lock", false, true, false, cct),
fn(f),
zero_buf(NULL),
max_size(0), block_size(0),
full_state(FULL_NOTFULL),
fd(-1),
writing_seq(0),
- throttle(g_conf->filestore_caller_concurrency),
- write_lock("FileJournal::write_lock", false, true, false, g_ceph_context),
+ throttle(cct->_conf->filestore_caller_concurrency),
+ write_lock("FileJournal::write_lock", false, true, false, cct),
write_stop(true),
aio_stop(true),
write_thread(this),
write_finish_thread(this) {
if (aio && !directio) {
- derr << "FileJournal::_open_any: aio not supported without directio; disabling aio" << dendl;
+ lderr(cct) << "FileJournal::_open_any: aio not supported without directio; disabling aio" << dendl;
aio = false;
}
#ifndef HAVE_LIBAIO
if (aio) {
- derr << "FileJournal::_open_any: libaio not compiled in; disabling aio" << dendl;
+ lderr(cct) << "FileJournal::_open_any: libaio not compiled in; disabling aio" << dendl;
aio = false;
}
#endif
- g_conf->add_observer(this);
+ cct->_conf->add_observer(this);
}
~FileJournal() {
assert(fd == -1);
delete[] zero_buf;
- g_conf->remove_observer(this);
+ cct->_conf->remove_observer(this);
}
int check();
WRITE_CLASS_ENCODER(FileJournal::header_t)
-#undef dout_context
#endif
#define tracepoint(...)
#endif
-#define dout_context g_ceph_context
+#define dout_context cct
#define dout_subsys ceph_subsys_filestore
#undef dout_prefix
#define dout_prefix *_dout << "filestore(" << basedir << ") "
return len > m_filestore_max_xattr_value_size ? -ENAMETOOLONG : 0;
}
-int FileStore::get_block_device_fsid(const string& path, uuid_d *fsid)
+int FileStore::get_block_device_fsid(CephContext* cct, const string& path,
+ uuid_d *fsid)
{
// make sure we don't try to use aio or direct_io (and get annoying
// error messages from failing to do so); performance implications
// should be irrelevant for this use
- FileJournal j(*fsid, 0, 0, path.c_str(), false, false);
+ FileJournal j(cct, *fsid, 0, 0, path.c_str(), false, false);
return j.peek_fsid(*fsid);
}
if (create)
flags |= O_CREAT;
- if (g_conf->filestore_odsync_write) {
+ if (cct->_conf->filestore_odsync_write) {
flags |= O_DSYNC;
}
assert(!m_filestore_fail_eio || r != -EIO);
return r;
}
- if (g_conf->filestore_debug_inject_read_err) {
+ if (cct->_conf->filestore_debug_inject_read_err) {
debug_obj_on_delete(o);
}
if (!m_disable_wbthrottle) {
fsid_fd(-1), op_fd(-1),
basedir_fd(-1), current_fd(-1),
backend(NULL),
- index_manager(do_update),
+ index_manager(cct, do_update),
lock("FileStore::lock"),
force_sync(false),
sync_entry_timeo_lock("FileStore::sync_entry_timeo_lock"),
- timer(g_ceph_context, sync_entry_timeo_lock),
+ timer(cct, sync_entry_timeo_lock),
stop(false), sync_thread(this),
- fdcache(g_ceph_context),
- wbthrottle(g_ceph_context),
+ fdcache(cct),
+ wbthrottle(cct),
next_osr_id(0),
- m_disable_wbthrottle(g_conf->filestore_odsync_write ||
- !g_conf->filestore_wbthrottle_enable),
- throttle_ops(g_conf->filestore_caller_concurrency),
- throttle_bytes(g_conf->filestore_caller_concurrency),
- m_ondisk_finisher_num(g_conf->filestore_ondisk_finisher_threads),
- m_apply_finisher_num(g_conf->filestore_apply_finisher_threads),
- op_tp(g_ceph_context, "FileStore::op_tp", "tp_fstore_op", g_conf->filestore_op_threads, "filestore_op_threads"),
- op_wq(this, g_conf->filestore_op_thread_timeout,
- g_conf->filestore_op_thread_suicide_timeout, &op_tp),
+ m_disable_wbthrottle(cct->_conf->filestore_odsync_write ||
+ !cct->_conf->filestore_wbthrottle_enable),
+ throttle_ops(cct->_conf->filestore_caller_concurrency),
+ throttle_bytes(cct->_conf->filestore_caller_concurrency),
+ m_ondisk_finisher_num(cct->_conf->filestore_ondisk_finisher_threads),
+ m_apply_finisher_num(cct->_conf->filestore_apply_finisher_threads),
+ op_tp(cct, "FileStore::op_tp", "tp_fstore_op", cct->_conf->filestore_op_threads, "filestore_op_threads"),
+ op_wq(this, cct->_conf->filestore_op_thread_timeout,
+ cct->_conf->filestore_op_thread_suicide_timeout, &op_tp),
logger(NULL),
read_error_lock("FileStore::read_error_lock"),
- m_filestore_commit_timeout(g_conf->filestore_commit_timeout),
- m_filestore_journal_parallel(g_conf->filestore_journal_parallel ),
- m_filestore_journal_trailing(g_conf->filestore_journal_trailing),
- m_filestore_journal_writeahead(g_conf->filestore_journal_writeahead),
- m_filestore_fiemap_threshold(g_conf->filestore_fiemap_threshold),
- m_filestore_max_sync_interval(g_conf->filestore_max_sync_interval),
- m_filestore_min_sync_interval(g_conf->filestore_min_sync_interval),
- m_filestore_fail_eio(g_conf->filestore_fail_eio),
- m_filestore_fadvise(g_conf->filestore_fadvise),
+ m_filestore_commit_timeout(cct->_conf->filestore_commit_timeout),
+ m_filestore_journal_parallel(cct->_conf->filestore_journal_parallel ),
+ m_filestore_journal_trailing(cct->_conf->filestore_journal_trailing),
+ m_filestore_journal_writeahead(cct->_conf->filestore_journal_writeahead),
+ m_filestore_fiemap_threshold(cct->_conf->filestore_fiemap_threshold),
+ m_filestore_max_sync_interval(cct->_conf->filestore_max_sync_interval),
+ m_filestore_min_sync_interval(cct->_conf->filestore_min_sync_interval),
+ m_filestore_fail_eio(cct->_conf->filestore_fail_eio),
+ m_filestore_fadvise(cct->_conf->filestore_fadvise),
do_update(do_update),
- m_journal_dio(g_conf->journal_dio),
- m_journal_aio(g_conf->journal_aio),
- m_journal_force_aio(g_conf->journal_force_aio),
- m_osd_rollback_to_cluster_snap(g_conf->osd_rollback_to_cluster_snap),
- m_osd_use_stale_snap(g_conf->osd_use_stale_snap),
+ m_journal_dio(cct->_conf->journal_dio),
+ m_journal_aio(cct->_conf->journal_aio),
+ m_journal_force_aio(cct->_conf->journal_force_aio),
+ m_osd_rollback_to_cluster_snap(cct->_conf->osd_rollback_to_cluster_snap),
+ m_osd_use_stale_snap(cct->_conf->osd_use_stale_snap),
m_filestore_do_dump(false),
m_filestore_dump_fmt(true),
- m_filestore_sloppy_crc(g_conf->filestore_sloppy_crc),
- m_filestore_sloppy_crc_block_size(g_conf->filestore_sloppy_crc_block_size),
- m_filestore_max_alloc_hint_size(g_conf->filestore_max_alloc_hint_size),
+ m_filestore_sloppy_crc(cct->_conf->filestore_sloppy_crc),
+ m_filestore_sloppy_crc_block_size(cct->_conf->filestore_sloppy_crc_block_size),
+ m_filestore_max_alloc_hint_size(cct->_conf->filestore_max_alloc_hint_size),
m_fs_type(0),
m_filestore_max_inline_xattr_size(0),
m_filestore_max_inline_xattrs(0),
m_filestore_max_xattr_value_size(0)
{
- m_filestore_kill_at.set(g_conf->filestore_kill_at);
+ m_filestore_kill_at.set(cct->_conf->filestore_kill_at);
for (int i = 0; i < m_ondisk_finisher_num; ++i) {
ostringstream oss;
oss << "filestore-ondisk-" << i;
- Finisher *f = new Finisher(g_ceph_context, oss.str(), "fn_odsk_fstore");
+ Finisher *f = new Finisher(cct, oss.str(), "fn_odsk_fstore");
ondisk_finishers.push_back(f);
}
for (int i = 0; i < m_apply_finisher_num; ++i) {
ostringstream oss;
oss << "filestore-apply-" << i;
- Finisher *f = new Finisher(g_ceph_context, oss.str(), "fn_appl_fstore");
+ Finisher *f = new Finisher(cct, oss.str(), "fn_appl_fstore");
apply_finishers.push_back(f);
}
current_op_seq_fn = sss.str();
ostringstream omss;
- if (g_conf->filestore_omap_backend_path != "") {
- omap_dir = g_conf->filestore_omap_backend_path;
+ if (cct->_conf->filestore_omap_backend_path != "") {
+ omap_dir = cct->_conf->filestore_omap_backend_path;
} else {
omss << basedir << "/current/omap";
omap_dir = omss.str();
}
// initialize logger
- PerfCountersBuilder plb(g_ceph_context, internal_name, l_filestore_first, l_filestore_last);
+ PerfCountersBuilder plb(cct, internal_name, l_filestore_first, l_filestore_last);
plb.add_u64(l_filestore_journal_queue_ops, "journal_queue_ops", "Operations in journal queue");
plb.add_u64(l_filestore_journal_ops, "journal_ops", "Active journal entries to be applied");
logger = plb.create_perf_counters();
- g_ceph_context->get_perfcounters_collection()->add(logger);
- g_ceph_context->_conf->add_observer(this);
+ cct->get_perfcounters_collection()->add(logger);
+ cct->_conf->add_observer(this);
superblock.compat_features = get_fs_initial_compat_set();
}
delete *it;
*it = NULL;
}
- g_ceph_context->_conf->remove_observer(this);
- g_ceph_context->get_perfcounters_collection()->remove(logger);
+ cct->_conf->remove_observer(this);
+ cct->get_perfcounters_collection()->remove(logger);
if (journal)
journal->logger = NULL;
ss << "0x" << std::hex << m_fs_type << std::dec;
(*pm)["filestore_f_type"] = ss.str();
- if (g_conf->filestore_collect_device_partition_information) {
+ if (cct->_conf->filestore_collect_device_partition_information) {
rc = get_device_by_uuid(get_fsid(), "PARTUUID", partition_path,
dev_node);
} else {
{
if (journalpath.length()) {
dout(10) << "open_journal at " << journalpath << dendl;
- journal = new FileJournal(fsid, &finisher, &sync_cond, journalpath.c_str(),
- m_journal_dio, m_journal_aio, m_journal_force_aio);
+ journal = new FileJournal(cct, fsid, &finisher, &sync_cond,
+ journalpath.c_str(),
+ m_journal_dio, m_journal_aio,
+ m_journal_force_aio);
if (journal)
journal->logger = logger;
}
if (!journalpath.length())
return -EINVAL;
- FileJournal *journal = new FileJournal(fsid, &finisher, &sync_cond, journalpath.c_str(), m_journal_dio);
+ FileJournal *journal = new FileJournal(cct, fsid, &finisher, &sync_cond, journalpath.c_str(), m_journal_dio);
r = journal->dump(out);
delete journal;
return r;
}
// superblock
- superblock.omap_backend = g_conf->filestore_omap_backend;
+ superblock.omap_backend = cct->_conf->filestore_omap_backend;
ret = write_superblock();
if (ret < 0) {
derr << "mkfs: write_superblock() failed: "
}
ret = KeyValueDB::test_init(superblock.omap_backend, omap_dir);
if (ret < 0) {
- derr << "mkfs failed to create " << g_conf->filestore_omap_backend << dendl;
+ derr << "mkfs failed to create " << cct->_conf->filestore_omap_backend << dendl;
goto close_fsid_fd;
}
// create fsid under omap
dout(1) << "FileStore::mkfs: omap fsid is already set to " << fsid << dendl;
}
- dout(1) << g_conf->filestore_omap_backend << " db exists/created" << dendl;
+ dout(1) << cct->_conf->filestore_omap_backend << " db exists/created" << dendl;
// journal?
ret = mkjournal();
<< cpp_strerror(ret) << dendl;
goto close_fsid_fd;
} else if (ret == 0) {
- if (do_update || (int)version_stamp < g_conf->filestore_update_to) {
+ if (do_update || (int)version_stamp < cct->_conf->filestore_update_to) {
derr << "FileStore::mount: stale version stamp detected: "
<< version_stamp
<< ". Proceeding, do_update "
derr << "FileStore::mount: stale version stamp " << version_stamp
<< ". Please run the FileStore update script before starting the "
<< "OSD, or set filestore_update_to to " << target_version
- << " (currently " << g_conf->filestore_update_to << ")"
+ << " (currently " << cct->_conf->filestore_update_to << ")"
<< dendl;
goto close_fsid_fd;
}
dout(0) << "start omap initiation" << dendl;
if (!(generic_flags & SKIP_MOUNT_OMAP)) {
- KeyValueDB * omap_store = KeyValueDB::create(g_ceph_context,
+ KeyValueDB * omap_store = KeyValueDB::create(cct,
superblock.omap_backend,
omap_dir);
if (omap_store == NULL)
}
if (superblock.omap_backend == "rocksdb")
- ret = omap_store->init(g_conf->filestore_rocksdb_options);
+ ret = omap_store->init(cct->_conf->filestore_rocksdb_options);
else
ret = omap_store->init();
}
stringstream err2;
- if (g_conf->filestore_debug_omap_check && !dbomap->check(err2)) {
+ if (cct->_conf->filestore_debug_omap_check && !dbomap->check(err2)) {
derr << err2.str() << dendl;
delete dbomap;
ret = -EINVAL;
wbthrottle.start();
} else {
dout(0) << "mount INFO: WbThrottle is disabled" << dendl;
- if (g_conf->filestore_odsync_write) {
+ if (cct->_conf->filestore_odsync_write) {
dout(0) << "mount INFO: O_DSYNC write is enabled" << dendl;
}
}
{
stringstream err2;
- if (g_conf->filestore_debug_omap_check && !object_map->check(err2)) {
+ if (cct->_conf->filestore_debug_omap_check && !object_map->check(err2)) {
derr << err2.str() << dendl;
ret = -EINVAL;
goto stop_sync;
timer.init();
// upgrade?
- if (g_conf->filestore_update_to >= (int)get_target_version()) {
+ if (cct->_conf->filestore_update_to >= (int)get_target_version()) {
int err = upgrade();
if (err < 0) {
derr << "error converting store" << dendl;
wbthrottle.throttle();
}
// inject a stall?
- if (g_conf->filestore_inject_stall) {
- int orig = g_conf->filestore_inject_stall;
+ if (cct->_conf->filestore_inject_stall) {
+ int orig = cct->_conf->filestore_inject_stall;
dout(5) << "_do_op filestore_inject_stall " << orig << ", sleeping" << dendl;
- for (int n = 0; n < g_conf->filestore_inject_stall; n++)
+ for (int n = 0; n < cct->_conf->filestore_inject_stall; n++)
sleep(1);
- g_conf->set_val("filestore_inject_stall", "0");
+ cct->_conf->set_val("filestore_inject_stall", "0");
dout(5) << "_do_op done stalling" << dendl;
}
ObjectStore::Transaction::collect_contexts(
tls, &onreadable, &ondisk, &onreadable_sync);
- if (g_conf->objectstore_blackhole) {
+ if (cct->_conf->objectstore_blackhole) {
dout(0) << __func__ << " objectstore_blackhole = TRUE, dropping transaction"
<< dendl;
delete ondisk;
dout(5) << "queue_transactions existing " << osr << " " << *osr << dendl;
} else {
osr = new OpSequencer(cct, next_osr_id.inc());
- osr->set_cct(g_ceph_context);
+ osr->set_cct(cct);
osr->parent = posr;
posr->p = osr;
dout(5) << "queue_transactions new " << osr << " " << *osr << dendl;
*_dout << dendl;
if (r == -EMFILE) {
- dump_open_fds(g_ceph_context);
+ dump_open_fds(cct);
}
assert(0 == "unexpected error");
<< " = " << r
<< " (size " << st->st_size << ")" << dendl;
}
- if (g_conf->filestore_debug_inject_read_err &&
+ if (cct->_conf->filestore_debug_inject_read_err &&
debug_mdata_eio(oid)) {
return -EIO;
} else {
dout(10) << "FileStore::read " << cid << "/" << oid << " " << offset << "~"
<< got << "/" << len << dendl;
- if (g_conf->filestore_debug_inject_read_err &&
+ if (cct->_conf->filestore_debug_inject_read_err &&
debug_data_eio(oid)) {
return -EIO;
} else {
dout(15) << "zero " << cid << "/" << oid << " " << offset << "~" << len << dendl;
int ret = 0;
- if (g_conf->filestore_punch_hole) {
+ if (cct->_conf->filestore_punch_hole) {
#ifdef CEPH_HAVE_FALLOCATE
# if !defined(DARWIN) && !defined(__FreeBSD__)
# ifdef FALLOC_FL_KEEP_SIZE
class SyncEntryTimeout : public Context {
public:
- explicit SyncEntryTimeout(int commit_timeo)
- : m_commit_timeo(commit_timeo)
+ CephContext* cct;
+ explicit SyncEntryTimeout(CephContext* cct, int commit_timeo)
+ : cct(cct), m_commit_timeo(commit_timeo)
{
}
sync_entry_timeo_lock.Lock();
SyncEntryTimeout *sync_entry_timeo =
- new SyncEntryTimeout(m_filestore_commit_timeout);
+ new SyncEntryTimeout(cct, m_filestore_commit_timeout);
timer.add_event_after(m_filestore_commit_timeout, sync_entry_timeo);
sync_entry_timeo_lock.Unlock();
dout(15) << "sync_entry committing " << cp << dendl;
stringstream errstream;
- if (g_conf->filestore_debug_omap_check && !object_map->check(errstream)) {
+ if (cct->_conf->filestore_debug_omap_check && !object_map->check(errstream)) {
derr << errstream.str() << dendl;
ceph_abort();
}
}
lock.Lock();
- finish_contexts(g_ceph_context, fin, 0);
+ finish_contexts(cct, fin, 0);
fin.clear();
if (!sync_waiters.empty()) {
dout(10) << "sync_entry more waiters, committing again" << dendl;
{
dout(10) << "flush" << dendl;
- if (g_conf->filestore_blackhole) {
+ if (cct->_conf->filestore_blackhole) {
// wait forever
Mutex lock("FileStore::flush::lock");
Cond cond;
out:
dout(10) << "getattr " << cid << "/" << oid << " '" << name << "' = " << r << dendl;
assert(!m_filestore_fail_eio || r != -EIO);
- if (g_conf->filestore_debug_inject_read_err &&
+ if (cct->_conf->filestore_debug_inject_read_err &&
debug_mdata_eio(oid)) {
return -EIO;
} else {
dout(10) << "getattrs " << cid << "/" << oid << " = " << r << dendl;
assert(!m_filestore_fail_eio || r != -EIO);
- if (g_conf->filestore_debug_inject_read_err &&
+ if (cct->_conf->filestore_debug_inject_read_err &&
debug_mdata_eio(oid)) {
return -EIO;
} else {
dout(5) << "_inject_failure " << (final+1) << " -> " << final << dendl;
if (final == 0) {
derr << "_inject_failure KILLING" << dendl;
- g_ceph_context->_log->flush();
+ cct->_log->flush();
_exit(1);
}
}
_close_replay_guard(cid, spos);
_close_replay_guard(dest, spos);
}
- if (!r && g_conf->filestore_debug_verify_split) {
+ if (!r && cct->_conf->filestore_debug_verify_split) {
vector<ghobject_t> objects;
ghobject_t next;
while (1) {
{
stringstream ss;
bool valid = throttle_bytes.set_params(
- g_conf->filestore_queue_low_threshhold,
- g_conf->filestore_queue_high_threshhold,
- g_conf->filestore_expected_throughput_bytes,
- g_conf->filestore_queue_high_delay_multiple,
- g_conf->filestore_queue_max_delay_multiple,
- g_conf->filestore_queue_max_bytes,
+ cct->_conf->filestore_queue_low_threshhold,
+ cct->_conf->filestore_queue_high_threshhold,
+ cct->_conf->filestore_expected_throughput_bytes,
+ cct->_conf->filestore_queue_high_delay_multiple,
+ cct->_conf->filestore_queue_max_delay_multiple,
+ cct->_conf->filestore_queue_max_bytes,
&ss);
valid &= throttle_ops.set_params(
- g_conf->filestore_queue_low_threshhold,
- g_conf->filestore_queue_high_threshhold,
- g_conf->filestore_expected_throughput_ops,
- g_conf->filestore_queue_high_delay_multiple,
- g_conf->filestore_queue_max_delay_multiple,
- g_conf->filestore_queue_max_ops,
+ cct->_conf->filestore_queue_low_threshhold,
+ cct->_conf->filestore_queue_high_threshhold,
+ cct->_conf->filestore_expected_throughput_ops,
+ cct->_conf->filestore_queue_high_delay_multiple,
+ cct->_conf->filestore_queue_max_delay_multiple,
+ cct->_conf->filestore_queue_max_ops,
&ss);
logger->set(l_filestore_op_queue_max_ops, throttle_ops.get_max());
switch (m_fs_type) {
#if defined(__linux__)
case XFS_SUPER_MAGIC:
- fs_xattr_size = g_conf->filestore_max_inline_xattr_size_xfs;
- fs_xattrs = g_conf->filestore_max_inline_xattrs_xfs;
- fs_xattr_max_value_size = g_conf->filestore_max_xattr_value_size_xfs;
+ fs_xattr_size = cct->_conf->filestore_max_inline_xattr_size_xfs;
+ fs_xattrs = cct->_conf->filestore_max_inline_xattrs_xfs;
+ fs_xattr_max_value_size = cct->_conf->filestore_max_xattr_value_size_xfs;
break;
case BTRFS_SUPER_MAGIC:
- fs_xattr_size = g_conf->filestore_max_inline_xattr_size_btrfs;
- fs_xattrs = g_conf->filestore_max_inline_xattrs_btrfs;
- fs_xattr_max_value_size = g_conf->filestore_max_xattr_value_size_btrfs;
+ fs_xattr_size = cct->_conf->filestore_max_inline_xattr_size_btrfs;
+ fs_xattrs = cct->_conf->filestore_max_inline_xattrs_btrfs;
+ fs_xattr_max_value_size = cct->_conf->filestore_max_xattr_value_size_btrfs;
break;
#endif
default:
- fs_xattr_size = g_conf->filestore_max_inline_xattr_size_other;
- fs_xattrs = g_conf->filestore_max_inline_xattrs_other;
- fs_xattr_max_value_size = g_conf->filestore_max_xattr_value_size_other;
+ fs_xattr_size = cct->_conf->filestore_max_inline_xattr_size_other;
+ fs_xattrs = cct->_conf->filestore_max_inline_xattrs_other;
+ fs_xattr_max_value_size = cct->_conf->filestore_max_xattr_value_size_other;
break;
}
// Use override value if set
- if (g_conf->filestore_max_inline_xattr_size)
- m_filestore_max_inline_xattr_size = g_conf->filestore_max_inline_xattr_size;
+ if (cct->_conf->filestore_max_inline_xattr_size)
+ m_filestore_max_inline_xattr_size = cct->_conf->filestore_max_inline_xattr_size;
else
m_filestore_max_inline_xattr_size = fs_xattr_size;
// Use override value if set
- if (g_conf->filestore_max_inline_xattrs)
- m_filestore_max_inline_xattrs = g_conf->filestore_max_inline_xattrs;
+ if (cct->_conf->filestore_max_inline_xattrs)
+ m_filestore_max_inline_xattrs = cct->_conf->filestore_max_inline_xattrs;
else
m_filestore_max_inline_xattrs = fs_xattrs;
// Use override value if set
- if (g_conf->filestore_max_xattr_value_size)
- m_filestore_max_xattr_value_size = g_conf->filestore_max_xattr_value_size;
+ if (cct->_conf->filestore_max_xattr_value_size)
+ m_filestore_max_xattr_value_size = cct->_conf->filestore_max_xattr_value_size;
else
m_filestore_max_xattr_value_size = fs_xattr_max_value_size;
- if (m_filestore_max_xattr_value_size < g_conf->osd_max_object_name_len) {
+ if (m_filestore_max_xattr_value_size < cct->_conf->osd_max_object_name_len) {
derr << "WARNING: max attr value size ("
<< m_filestore_max_xattr_value_size
<< ") is smaller than osd_max_object_name_len ("
- << g_conf->osd_max_object_name_len
+ << cct->_conf->osd_max_object_name_len
<< "). Your backend filesystem appears to not support attrs large "
<< "enough to handle the configured max rados name size. You may get "
<< "unexpected ENAMETOOLONG errors on rados operations or buggy "
return target_version;
}
- static int get_block_device_fsid(const string& path, uuid_d *fsid);
-
+ static int get_block_device_fsid(CephContext* cct, const string& path,
+ uuid_d *fsid);
struct FSPerfTracker {
PerfCounters::avg_tracker<uint64_t> os_commit_latency;
PerfCounters::avg_tracker<uint64_t> os_apply_latency;
void flush() {
Mutex::Locker l(qlock);
- while (g_conf->filestore_blackhole)
+ while (cct->_conf->filestore_blackhole)
cond.Wait(qlock); // wait forever
explicit FileStoreBackend(FileStore *fs) : filestore(fs) {}
virtual ~FileStoreBackend() {}
+ CephContext* cct() const {
+ return filestore->cct;
+ }
+
static FileStoreBackend *create(long f_type, FileStore *fs);
virtual const char *get_name() = 0;
#define SLOPPY_CRC_XATTR "user.cephos.scrc"
-#define dout_context g_ceph_context
+#define dout_context cct()
#define dout_subsys ceph_subsys_filestore
#undef dout_prefix
#define dout_prefix *_dout << "genericfilestorebackend(" << get_basedir_path() << ") "
ioctl_fiemap(false),
seek_data_hole(false),
use_splice(false),
- m_filestore_fiemap(g_conf->filestore_fiemap),
- m_filestore_seek_data_hole(g_conf->filestore_seek_data_hole),
- m_filestore_fsync_flushes_journal_data(g_conf->filestore_fsync_flushes_journal_data),
- m_filestore_splice(g_conf->filestore_splice) {}
+ m_filestore_fiemap(cct()->_conf->filestore_fiemap),
+ m_filestore_seek_data_hole(cct()->_conf->filestore_seek_data_hole),
+ m_filestore_fsync_flushes_journal_data(cct()->_conf->filestore_fsync_flushes_journal_data),
+ m_filestore_splice(cct()->_conf->filestore_splice) {}
int GenericFileStoreBackend::detect_features()
{
#include "common/errno.h"
#include "common/debug.h"
-#define dout_context g_ceph_context
+#define dout_context cct
#define dout_subsys ceph_subsys_filestore
const string HashIndex::SUBDIR_ATTR = "contents";
public:
/// Constructor.
HashIndex(
+ CephContext* cct,
coll_t collection, ///< [in] Collection
const char *base_path, ///< [in] Path to the index root.
int merge_at, ///< [in] Merge threshhold.
int split_multiple, ///< [in] Split threshhold.
uint32_t index_version,///< [in] Index version
double retry_probability=0) ///< [in] retry probability
- : LFNIndex(collection, base_path, index_version, retry_probability),
+ : LFNIndex(cct, collection, base_path, index_version, retry_probability),
merge_threshold(merge_at),
split_multiplier(split_multiple) {}
int r = set_version(path, version);
if (r < 0)
return r;
- HashIndex index(c, path, g_conf->filestore_merge_threshold,
- g_conf->filestore_split_multiple,
+ HashIndex index(cct, c, path, cct->_conf->filestore_merge_threshold,
+ cct->_conf->filestore_split_multiple,
version,
- g_conf->filestore_index_retry_probability);
+ cct->_conf->filestore_index_retry_probability);
return index.init();
}
case CollectionIndex::HASH_INDEX_TAG_2: // fall through
case CollectionIndex::HOBJECT_WITH_POOL: {
// Must be a HashIndex
- *index = new HashIndex(c, path, g_conf->filestore_merge_threshold,
- g_conf->filestore_split_multiple, version);
+ *index = new HashIndex(cct, c, path,
+ cct->_conf->filestore_merge_threshold,
+ cct->_conf->filestore_split_multiple, version);
return 0;
}
default: ceph_abort();
} else {
// No need to check
- *index = new HashIndex(c, path, g_conf->filestore_merge_threshold,
- g_conf->filestore_split_multiple,
- CollectionIndex::HOBJECT_WITH_POOL,
- g_conf->filestore_index_retry_probability);
+ *index = new HashIndex(cct, c, path, cct->_conf->filestore_merge_threshold,
+ cct->_conf->filestore_split_multiple,
+ CollectionIndex::HOBJECT_WITH_POOL,
+ cct->_conf->filestore_index_retry_probability);
return 0;
}
}
* This is enforced by using CollectionIndex::access_lock
*/
class IndexManager {
+ CephContext* cct;
RWLock lock; ///< Lock for Index Manager
bool upgrade;
ceph::unordered_map<coll_t, CollectionIndex* > col_indices;
bool get_index_optimistic(coll_t c, Index *index);
public:
/// Constructor
- explicit IndexManager(bool upgrade) : lock("IndexManager lock"),
- upgrade(upgrade) {}
+ explicit IndexManager(CephContext* cct,
+ bool upgrade) : cct(cct),
+ lock("IndexManager lock"),
+ upgrade(upgrade) {}
~IndexManager();
uuid_d fsid;
Finisher *finisher;
public:
+ CephContext* cct;
PerfCounters *logger;
protected:
Cond *do_sync_cond;
bool wait_on_full;
public:
- Journal(uuid_d f, Finisher *fin, Cond *c=0) :
- fsid(f), finisher(fin), logger(NULL),
+ Journal(CephContext* cct, uuid_d f, Finisher *fin, Cond *c=0) :
+ fsid(f), finisher(fin), cct(cct), logger(NULL),
do_sync_cond(c),
wait_on_full(false) { }
virtual ~Journal() { }
#include "common/errno.h"
#include "common/debug.h"
-#define dout_context g_ceph_context
+#define dout_context cct
#define dout_subsys ceph_subsys_journal
#undef dout_prefix
#define dout_prefix *_dout << "journal "
{
dout(10) << "journal_replay fs op_seq " << fs_op_seq << dendl;
- if (g_conf->journal_replay_from) {
- dout(0) << "journal_replay forcing replay from " << g_conf->journal_replay_from
+ if (cct->_conf->journal_replay_from) {
+ dout(0) << "journal_replay forcing replay from "
+ << cct->_conf->journal_replay_from
<< " instead of " << fs_op_seq << dendl;
// the previous op is the last one committed
- fs_op_seq = g_conf->journal_replay_from - 1;
+ fs_op_seq = cct->_conf->journal_replay_from - 1;
}
uint64_t op_seq = fs_op_seq;
dout(10) << "op_apply_start blocked, waiting" << dendl;
blocked_cond.Wait(apply_lock);
}
- dout(10) << "op_apply_start " << op << " open_ops " << open_ops << " -> " << (open_ops+1) << dendl;
+ dout(10) << "op_apply_start " << op << " open_ops " << open_ops << " -> "
+ << (open_ops+1) << dendl;
assert(!blocked);
assert(op > committed_seq);
open_ops++;
void JournalingObjectStore::ApplyManager::op_apply_finish(uint64_t op)
{
Mutex::Locker l(apply_lock);
- dout(10) << "op_apply_finish " << op << " open_ops " << open_ops
- << " -> " << (open_ops-1)
- << ", max_applied_seq " << max_applied_seq << " -> " << MAX(op, max_applied_seq)
- << dendl;
+ dout(10) << "op_apply_finish " << op << " open_ops " << open_ops << " -> "
+ << (open_ops-1) << ", max_applied_seq " << max_applied_seq << " -> "
+ << MAX(op, max_applied_seq) << dendl;
--open_ops;
assert(open_ops >= 0);
{
Mutex::Locker l(apply_lock);
dout(10) << "commit_start max_applied_seq " << max_applied_seq
- << ", open_ops " << open_ops
- << dendl;
+ << ", open_ops " << open_ops << dendl;
blocked = true;
while (open_ops > 0) {
- dout(10) << "commit_start waiting for " << open_ops << " open ops to drain" << dendl;
+ dout(10) << "commit_start waiting for " << open_ops
+ << " open ops to drain" << dendl;
blocked_cond.Wait(apply_lock);
}
assert(open_ops == 0);
{
Mutex::Locker l(apply_lock);
// allow new ops. (underlying fs should now be committing all prior ops)
- dout(10) << "commit_started committing " << committing_seq << ", unblocking" << dendl;
+ dout(10) << "commit_started committing " << committing_seq << ", unblocking"
+ << dendl;
blocked = false;
blocked_cond.Signal();
}
class SubmitManager {
+ CephContext* cct;
Mutex lock;
uint64_t op_seq;
uint64_t op_submitted;
public:
- SubmitManager() :
- lock("JOS::SubmitManager::lock", false, true, false, g_ceph_context),
+ SubmitManager(CephContext* cct) :
+ cct(cct), lock("JOS::SubmitManager::lock", false, true, false, cct),
op_seq(0), op_submitted(0)
{}
uint64_t op_submit_start();
} submit_manager;
class ApplyManager {
+ CephContext* cct;
Journal *&journal;
Finisher &finisher;
uint64_t committing_seq, committed_seq;
public:
- ApplyManager(Journal *&j, Finisher &f) :
- journal(j), finisher(f),
- apply_lock("JOS::ApplyManager::apply_lock", false, true, false, g_ceph_context),
+ ApplyManager(CephContext* cct, Journal *&j, Finisher &f) :
+ cct(cct), journal(j), finisher(f),
+ apply_lock("JOS::ApplyManager::apply_lock", false, true, false, cct),
blocked(false),
open_ops(0),
max_applied_seq(0),
- com_lock("JOS::ApplyManager::com_lock", false, true, false, g_ceph_context),
+ com_lock("JOS::ApplyManager::com_lock", false, true, false, cct),
committing_seq(0), committed_seq(0) {}
void reset() {
assert(open_ops == 0);
: ObjectStore(cct, path),
journal(NULL),
finisher(cct, "JournalObjectStore", "fn_jrn_objstore"),
- apply_manager(journal, finisher),
+ submit_manager(cct),
+ apply_manager(cct, journal, finisher),
replaying(false) {}
~JournalingObjectStore() {
#include "LFNIndex.h"
using ceph::crypto::SHA1;
-#define dout_context g_ceph_context
+#define dout_context cct
#define dout_subsys ceph_subsys_filestore
#undef dout_prefix
#define dout_prefix *_dout << "LFNIndex(" << get_base_path() << ") "
public:
/// Constructor
LFNIndex(
+ CephContext* cct,
coll_t collection,
const char *base_path, ///< [in] path to Index root
uint32_t index_version,
double _error_injection_probability=0)
- : CollectionIndex(collection),
+ : CollectionIndex(cct, collection),
base_path(base_path),
index_version(index_version),
error_injection_enabled(false),
::fsync(**wb.get<1>());
#endif
#ifdef HAVE_POSIX_FADVISE
- if (g_conf->filestore_fadvise && wb.get<2>().nocache) {
+ if (cct->_conf->filestore_fadvise && wb.get<2>().nocache) {
int fa_r = posix_fadvise(**wb.get<1>(), 0, 0, POSIX_FADV_DONTNEED);
assert(fa_r == 0);
}
i != pending_wbs.end();
++i) {
#ifdef HAVE_POSIX_FADVISE
- if (g_conf->filestore_fadvise && i->second.first.nocache) {
+ if (cct->_conf->filestore_fadvise && i->second.first.nocache) {
int fa_r = posix_fadvise(**i->second.second, 0, 0, POSIX_FADV_DONTNEED);
assert(fa_r == 0);
}
#include "include/assert.h"
#include "include/compat.h"
-#define dout_context g_ceph_context
+#define dout_context cct()
#define dout_subsys ceph_subsys_filestore
#undef dout_prefix
#define dout_prefix *_dout << "xfsfilestorebackend(" << get_basedir_path() << ") "
goto out_close;
}
- if (g_conf->filestore_xfs_extsize) {
+ if (cct()->_conf->filestore_xfs_extsize) {
ret = set_extsize(fd, 1U << 15); // a few pages
if (ret) {
ret = 0;
ZFSFileStoreBackend::ZFSFileStoreBackend(FileStore *fs) :
GenericFileStoreBackend(fs), base_zh(NULL), current_zh(NULL),
- m_filestore_zfs_snap(g_conf->filestore_zfs_snap)
+ m_filestore_zfs_snap(cct->_conf->filestore_zfs_snap)
{
int ret = zfs.init();
if (ret < 0) {
class TestWrapLFNIndex : public LFNIndex {
public:
- TestWrapLFNIndex(coll_t collection,
+ TestWrapLFNIndex(CephContext* cct,
+ coll_t collection,
const char *base_path,
- uint32_t index_version) : LFNIndex(collection, base_path, index_version) {}
+ uint32_t index_version)
+ : LFNIndex(cct, collection, base_path, index_version) {}
virtual uint32_t collection_version() {
return index_version;
class TestHASH_INDEX_TAG : public TestWrapLFNIndex, public ::testing::Test {
public:
- TestHASH_INDEX_TAG() : TestWrapLFNIndex(coll_t(), "PATH_1", CollectionIndex::HASH_INDEX_TAG) {
+ TestHASH_INDEX_TAG()
+ : TestWrapLFNIndex(g_ceph_context, coll_t(), "PATH_1",
+ CollectionIndex::HASH_INDEX_TAG) {
}
};
class TestHASH_INDEX_TAG_2 : public TestWrapLFNIndex, public ::testing::Test {
public:
- TestHASH_INDEX_TAG_2() : TestWrapLFNIndex(coll_t(), "PATH_1", CollectionIndex::HASH_INDEX_TAG_2) {
+ TestHASH_INDEX_TAG_2()
+ : TestWrapLFNIndex(g_ceph_context,
+ coll_t(), "PATH_1", CollectionIndex::HASH_INDEX_TAG_2) {
}
};
class TestHOBJECT_WITH_POOL : public TestWrapLFNIndex, public ::testing::Test {
public:
- TestHOBJECT_WITH_POOL() : TestWrapLFNIndex(coll_t(), "PATH_1", CollectionIndex::HOBJECT_WITH_POOL) {
+ TestHOBJECT_WITH_POOL()
+ : TestWrapLFNIndex(g_ceph_context, coll_t(),
+ "PATH_1", CollectionIndex::HOBJECT_WITH_POOL) {
}
};
class TestLFNIndex : public TestWrapLFNIndex, public ::testing::Test {
public:
- TestLFNIndex() : TestWrapLFNIndex(coll_t(), "PATH_1", CollectionIndex::HOBJECT_WITH_POOL) {
+ TestLFNIndex()
+ : TestWrapLFNIndex(g_ceph_context, coll_t(), "PATH_1",
+ CollectionIndex::HOBJECT_WITH_POOL) {
}
virtual void SetUp() {
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
}
}
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
for (unsigned i = 0 ; i < 3; ++i) {
SCOPED_TRACE(subtests[i].description);
fsid.generate_random();
- FileJournal j(fsid, finisher, &sync_cond, path, subtests[i].directio,
- subtests[i].aio, subtests[i].faio);
+ FileJournal j(g_ceph_context, fsid, finisher, &sync_cond, path,
+ subtests[i].directio, subtests[i].aio, subtests[i].faio);
ASSERT_EQ(0, j.create());
j.make_writeable();
if (!journalpath.length())
return -EINVAL;
- FileJournal *journal = new FileJournal(uuid_d(), NULL, NULL, journalpath.c_str(), m_journal_dio);
+ FileJournal *journal = new FileJournal(g_ceph_context, uuid_d(), NULL, NULL,
+ journalpath.c_str(), m_journal_dio);
r = journal->_fdump(*f, false);
delete journal;
return r;