// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- // vim: ts=8 sw=2 smarttab /* * Ceph - scalable distributed file system * * Author: Myoungwon Oh * * This is free software; you can redistribute it and/or * modify it under the terms of the GNU Lesser General Public * License version 2.1, as published by the Free Software * Foundation. See file COPYING. * */ #include "include/types.h" #include "include/rados/buffer.h" #include "include/rados/librados.hpp" #include "include/rados/rados_types.hpp" #include "acconfig.h" #include "common/Cond.h" #include "common/Formatter.h" #include "common/ceph_argparse.h" #include "common/ceph_crypto.h" #include "common/config.h" #include "common/debug.h" #include "common/errno.h" #include "common/obj_bencher.h" #include "global/global_init.h" #include #include #include #include #include #include #include #include #include #include #include #include #include "tools/RadosDump.h" #include "cls/cas/cls_cas_client.h" #include "cls/cas/cls_cas_internal.h" #include "include/stringify.h" #include "global/signal_handler.h" #include "common/CDC.h" struct EstimateResult { std::unique_ptr cdc; uint64_t chunk_size; ceph::mutex lock = ceph::make_mutex("EstimateResult::lock"); // < key, > map< string, pair > chunk_statistics; uint64_t total_bytes = 0; std::atomic total_objects = {0}; EstimateResult(std::string alg, int chunk_size) : cdc(CDC::create(alg, chunk_size)), chunk_size(1ull << chunk_size) {} void add_chunk(bufferlist& chunk, const std::string& fp_algo) { string fp; if (fp_algo == "sha1") { sha1_digest_t sha1_val = crypto::digest(chunk); fp = sha1_val.to_str(); } else if (fp_algo == "sha256") { sha256_digest_t sha256_val = crypto::digest(chunk); fp = sha256_val.to_str(); } else if (fp_algo == "sha512") { sha512_digest_t sha512_val = crypto::digest(chunk); fp = sha512_val.to_str(); } else { ceph_assert(0 == "no support fingerperint algorithm"); } std::lock_guard l(lock); auto p = chunk_statistics.find(fp); if (p != chunk_statistics.end()) { p->second.first++; if (p->second.second != chunk.length()) { cerr << "warning: hash collision on " << fp << ": was " << p->second.second << " now " << chunk.length() << std::endl; } } else { chunk_statistics[fp] = make_pair(1, chunk.length()); } total_bytes += chunk.length(); } void dump(Formatter *f) const { f->dump_unsigned("target_chunk_size", chunk_size); uint64_t dedup_bytes = 0; uint64_t dedup_objects = chunk_statistics.size(); for (auto& j : chunk_statistics) { dedup_bytes += j.second.second; } //f->dump_unsigned("dedup_bytes", dedup_bytes); //f->dump_unsigned("original_bytes", total_bytes); f->dump_float("dedup_bytes_ratio", (double)dedup_bytes / (double)total_bytes); f->dump_float("dedup_objects_ratio", (double)dedup_objects / (double)total_objects); uint64_t avg = total_bytes / dedup_objects; uint64_t sqsum = 0; for (auto& j : chunk_statistics) { sqsum += (avg - j.second.second) * (avg - j.second.second); } uint64_t stddev = sqrt(sqsum / dedup_objects); f->dump_unsigned("chunk_size_average", avg); f->dump_unsigned("chunk_size_stddev", stddev); } }; map dedup_estimates; // chunk size -> result using namespace librados; unsigned default_op_size = 1 << 26; unsigned default_max_thread = 2; int32_t default_report_period = 10; ceph::mutex glock = ceph::make_mutex("glock"); void usage() { cout << " usage: [--op ] [--pool ] " << std::endl; cout << " --object " << std::endl; cout << " --chunk-size chunk-size (byte) " << std::endl; cout << " --chunk-algorithm " << std::endl; cout << " --fingerprint-algorithm " << std::endl; cout << " --chunk-pool " << std::endl; cout << " --max-thread " << std::endl; cout << " --report-period " << std::endl; cout << " --max-seconds " << std::endl; cout << " --max-read-size " << std::endl; exit(1); } template static int rados_sistrtoll(I &i, T *val) { std::string err; *val = strict_iecstrtoll(i->second.c_str(), &err); if (err != "") { cerr << "Invalid value for " << i->first << ": " << err << std::endl; return -EINVAL; } else { return 0; } } class EstimateDedupRatio; class ChunkScrub; class CrawlerThread : public Thread { IoCtx io_ctx; int n; int m; ObjectCursor begin; ObjectCursor end; ceph::mutex m_lock = ceph::make_mutex("CrawlerThread::Locker"); ceph::condition_variable m_cond; int32_t report_period; bool m_stop = false; uint64_t total_bytes = 0; uint64_t total_objects = 0; uint64_t examined_objects = 0; uint64_t examined_bytes = 0; uint64_t max_read_size = 0; bool debug = false; #define COND_WAIT_INTERVAL 10 public: CrawlerThread(IoCtx& io_ctx, int n, int m, ObjectCursor begin, ObjectCursor end, int32_t report_period, uint64_t num_objects, uint64_t max_read_size = default_op_size): io_ctx(io_ctx), n(n), m(m), begin(begin), end(end), report_period(report_period), total_objects(num_objects), max_read_size(max_read_size) {} void signal(int signum) { std::lock_guard l{m_lock}; m_stop = true; m_cond.notify_all(); } virtual void print_status(Formatter *f, ostream &out) {} uint64_t get_examined_objects() { return examined_objects; } uint64_t get_examined_bytes() { return examined_bytes; } uint64_t get_total_bytes() { return total_bytes; } uint64_t get_total_objects() { return total_objects; } void set_debug(const bool debug_) { debug = debug_; } friend class EstimateDedupRatio; friend class ChunkScrub; }; class EstimateDedupRatio : public CrawlerThread { string chunk_algo; string fp_algo; uint64_t chunk_size; uint64_t max_seconds; public: EstimateDedupRatio( IoCtx& io_ctx, int n, int m, ObjectCursor begin, ObjectCursor end, string chunk_algo, string fp_algo, uint64_t chunk_size, int32_t report_period, uint64_t num_objects, uint64_t max_read_size, uint64_t max_seconds): CrawlerThread(io_ctx, n, m, begin, end, report_period, num_objects, max_read_size), chunk_algo(chunk_algo), fp_algo(fp_algo), chunk_size(chunk_size), max_seconds(max_seconds) { } void* entry() { estimate_dedup_ratio(); return NULL; } void estimate_dedup_ratio(); }; class ChunkScrub: public CrawlerThread { IoCtx chunk_io_ctx; int damaged_objects = 0; public: ChunkScrub(IoCtx& io_ctx, int n, int m, ObjectCursor begin, ObjectCursor end, IoCtx& chunk_io_ctx, int32_t report_period, uint64_t num_objects): CrawlerThread(io_ctx, n, m, begin, end, report_period, num_objects), chunk_io_ctx(chunk_io_ctx) { } void* entry() { chunk_scrub_common(); return NULL; } void chunk_scrub_common(); int get_damaged_objects() { return damaged_objects; } void print_status(Formatter *f, ostream &out); }; vector> estimate_threads; static void print_dedup_estimate(std::ostream& out, std::string chunk_algo) { /* uint64_t total_bytes = 0; uint64_t total_objects = 0; */ uint64_t examined_objects = 0; uint64_t examined_bytes = 0; for (auto &et : estimate_threads) { examined_objects += et->get_examined_objects(); examined_bytes += et->get_examined_bytes(); } auto f = Formatter::create("json-pretty"); f->open_object_section("results"); f->dump_string("chunk_algo", chunk_algo); f->open_array_section("chunk_sizes"); for (auto& i : dedup_estimates) { f->dump_object("chunker", i.second); } f->close_section(); f->open_object_section("summary"); f->dump_unsigned("examined_objects", examined_objects); f->dump_unsigned("examined_bytes", examined_bytes); /* f->dump_unsigned("total_objects", total_objects); f->dump_unsigned("total_bytes", total_bytes); f->dump_float("examined_ratio", (float)examined_bytes / (float)total_bytes); */ f->close_section(); f->close_section(); f->flush(out); } static void handle_signal(int signum) { std::lock_guard l{glock}; for (auto &p : estimate_threads) { p->signal(signum); } } void EstimateDedupRatio::estimate_dedup_ratio() { ObjectCursor shard_start; ObjectCursor shard_end; io_ctx.object_list_slice( begin, end, n, m, &shard_start, &shard_end); utime_t start = ceph_clock_now(); utime_t end; if (max_seconds) { end = start; end += max_seconds; } utime_t next_report; if (report_period) { next_report = start; next_report += report_period; } ObjectCursor c(shard_start); while (c < shard_end) { std::vector result; int r = io_ctx.object_list(c, shard_end, 12, {}, &result, &c); if (r < 0 ){ cerr << "error object_list : " << cpp_strerror(r) << std::endl; return; } unsigned op_size = max_read_size; for (const auto & i : result) { const auto &oid = i.oid; utime_t now = ceph_clock_now(); if (max_seconds && now > end) { m_stop = true; } if (m_stop) { return; } if (n == 0 && // first thread only next_report != utime_t() && now > next_report) { cerr << (int)(now - start) << "s : read " << dedup_estimates.begin()->second.total_bytes << " bytes so far..." << std::endl; print_dedup_estimate(cerr, chunk_algo); next_report = now; next_report += report_period; } // read entire object bufferlist bl; uint64_t offset = 0; while (true) { bufferlist t; int ret = io_ctx.read(oid, t, op_size, offset); if (ret <= 0) { break; } offset += ret; bl.claim_append(t); } examined_objects++; examined_bytes += bl.length(); // do the chunking for (auto& i : dedup_estimates) { vector> chunks; i.second.cdc->calc_chunks(bl, &chunks); for (auto& p : chunks) { bufferlist chunk; chunk.substr_of(bl, p.first, p.second); i.second.add_chunk(chunk, fp_algo); if (debug) { cout << " " << oid << " " << p.first << "~" << p.second << std::endl; } } ++i.second.total_objects; } } } } void ChunkScrub::chunk_scrub_common() { ObjectCursor shard_start; ObjectCursor shard_end; int ret; Rados rados; ret = rados.init_with_context(g_ceph_context); if (ret < 0) { cerr << "couldn't initialize rados: " << cpp_strerror(ret) << std::endl; return; } ret = rados.connect(); if (ret) { cerr << "couldn't connect to cluster: " << cpp_strerror(ret) << std::endl; return; } chunk_io_ctx.object_list_slice( begin, end, n, m, &shard_start, &shard_end); ObjectCursor c(shard_start); while(c < shard_end) { std::vector result; int r = chunk_io_ctx.object_list(c, shard_end, 12, {}, &result, &c); if (r < 0 ){ cerr << "error object_list : " << cpp_strerror(r) << std::endl; return; } for (const auto & i : result) { std::unique_lock l{m_lock}; if (m_stop) { Formatter *formatter = Formatter::create("json-pretty"); print_status(formatter, cout); delete formatter; return; } auto oid = i.oid; cout << oid << std::endl; chunk_refs_t refs; { bufferlist t; ret = chunk_io_ctx.getxattr(oid, CHUNK_REFCOUNT_ATTR, t); if (ret < 0) { continue; } auto p = t.cbegin(); decode(refs, p); } examined_objects++; if (refs.get_type() != chunk_refs_t::TYPE_BY_OBJECT) { // we can't do anything here continue; } // check all objects chunk_refs_by_object_t *byo = static_cast(refs.r.get()); set real_refs; uint64_t pool_missing = 0; uint64_t object_missing = 0; uint64_t does_not_ref = 0; for (auto& pp : byo->by_object) { IoCtx target_io_ctx; ret = rados.ioctx_create2(pp.pool, target_io_ctx); if (ret < 0) { cerr << oid << " ref " << pp << ": referencing pool does not exist" << std::endl; ++pool_missing; continue; } ret = cls_cas_references_chunk(target_io_ctx, pp.oid.name, oid); if (ret == -ENOENT) { cerr << oid << " ref " << pp << ": referencing object missing" << std::endl; ++object_missing; } else if (ret == -ENOLINK) { cerr << oid << " ref " << pp << ": referencing object does not reference chunk" << std::endl; ++does_not_ref; } } if (pool_missing || object_missing || does_not_ref) { ++damaged_objects; } } } cout << "--done--" << std::endl; } void ChunkScrub::print_status(Formatter *f, ostream &out) { if (f) { f->open_array_section("chunk_scrub"); f->dump_string("PID", stringify(get_pid())); f->open_object_section("Status"); f->dump_string("Total object", stringify(total_objects)); f->dump_string("Examined objects", stringify(examined_objects)); f->dump_string("damaged objects", stringify(damaged_objects)); f->close_section(); f->flush(out); cout << std::endl; } } int estimate_dedup_ratio(const std::map < std::string, std::string > &opts, std::vector &nargs) { Rados rados; IoCtx io_ctx; std::string chunk_algo = "fastcdc"; string fp_algo = "sha1"; string pool_name; uint64_t chunk_size = 0; uint64_t min_chunk_size = 8192; uint64_t max_chunk_size = 4*1024*1024; unsigned max_thread = default_max_thread; uint32_t report_period = default_report_period; uint64_t max_read_size = default_op_size; uint64_t max_seconds = 0; int ret; std::map::const_iterator i; bool debug = false; ObjectCursor begin; ObjectCursor end; librados::pool_stat_t s; list pool_names; map stats; i = opts.find("pool"); if (i != opts.end()) { pool_name = i->second.c_str(); } i = opts.find("chunk-algorithm"); if (i != opts.end()) { chunk_algo = i->second.c_str(); if (!CDC::create(chunk_algo, 12)) { cerr << "unrecognized chunk-algorithm " << chunk_algo << std::endl; exit(1); } } else { cerr << "must specify chunk-algorithm" << std::endl; exit(1); } i = opts.find("fingerprint-algorithm"); if (i != opts.end()) { fp_algo = i->second.c_str(); if (fp_algo != "sha1" && fp_algo != "sha256" && fp_algo != "sha512") { cerr << "unrecognized fingerprint-algorithm " << fp_algo << std::endl; exit(1); } } i = opts.find("chunk-size"); if (i != opts.end()) { if (rados_sistrtoll(i, &chunk_size)) { return -EINVAL; } } i = opts.find("min-chunk-size"); if (i != opts.end()) { if (rados_sistrtoll(i, &min_chunk_size)) { return -EINVAL; } } i = opts.find("max-chunk-size"); if (i != opts.end()) { if (rados_sistrtoll(i, &max_chunk_size)) { return -EINVAL; } } i = opts.find("max-thread"); if (i != opts.end()) { if (rados_sistrtoll(i, &max_thread)) { return -EINVAL; } } i = opts.find("report-period"); if (i != opts.end()) { if (rados_sistrtoll(i, &report_period)) { return -EINVAL; } } i = opts.find("max-seconds"); if (i != opts.end()) { if (rados_sistrtoll(i, &max_seconds)) { return -EINVAL; } } i = opts.find("max-read-size"); if (i != opts.end()) { if (rados_sistrtoll(i, &max_read_size)) { return -EINVAL; } } i = opts.find("debug"); if (i != opts.end()) { debug = true; } i = opts.find("pgid"); boost::optional pgid(i != opts.end(), pg_t()); ret = rados.init_with_context(g_ceph_context); if (ret < 0) { cerr << "couldn't initialize rados: " << cpp_strerror(ret) << std::endl; goto out; } ret = rados.connect(); if (ret) { cerr << "couldn't connect to cluster: " << cpp_strerror(ret) << std::endl; ret = -1; goto out; } if (pool_name.empty()) { cerr << "--create-pool requested but pool_name was not specified!" << std::endl; exit(1); } ret = rados.ioctx_create(pool_name.c_str(), io_ctx); if (ret < 0) { cerr << "error opening pool " << pool_name << ": " << cpp_strerror(ret) << std::endl; goto out; } // set up chunkers if (chunk_size) { dedup_estimates.emplace(std::piecewise_construct, std::forward_as_tuple(chunk_size), std::forward_as_tuple(chunk_algo, cbits(chunk_size)-1)); } else { for (size_t cs = min_chunk_size; cs <= max_chunk_size; cs *= 2) { dedup_estimates.emplace(std::piecewise_construct, std::forward_as_tuple(cs), std::forward_as_tuple(chunk_algo, cbits(cs)-1)); } } glock.lock(); begin = io_ctx.object_list_begin(); end = io_ctx.object_list_end(); pool_names.push_back(pool_name); ret = rados.get_pool_stats(pool_names, stats); if (ret < 0) { cerr << "error fetching pool stats: " << cpp_strerror(ret) << std::endl; glock.unlock(); return ret; } if (stats.find(pool_name) == stats.end()) { cerr << "stats can not find pool name: " << pool_name << std::endl; glock.unlock(); return ret; } s = stats[pool_name]; for (unsigned i = 0; i < max_thread; i++) { std::unique_ptr ptr ( new EstimateDedupRatio(io_ctx, i, max_thread, begin, end, chunk_algo, fp_algo, chunk_size, report_period, s.num_objects, max_read_size, max_seconds)); ptr->create("estimate_thread"); ptr->set_debug(debug); estimate_threads.push_back(move(ptr)); } glock.unlock(); for (auto &p : estimate_threads) { p->join(); } print_dedup_estimate(cout, chunk_algo); out: return (ret < 0) ? 1 : 0; } static void print_chunk_scrub() { uint64_t total_objects = 0; uint64_t examined_objects = 0; int damaged_objects = 0; for (auto &et : estimate_threads) { if (!total_objects) { total_objects = et->get_total_objects(); } examined_objects += et->get_examined_objects(); ChunkScrub *ptr = static_cast(et.get()); damaged_objects += ptr->get_damaged_objects(); } cout << " Total object : " << total_objects << std::endl; cout << " Examined object : " << examined_objects << std::endl; cout << " Damaged object : " << damaged_objects << std::endl; } int chunk_scrub_common(const std::map < std::string, std::string > &opts, std::vector &nargs) { Rados rados; IoCtx io_ctx, chunk_io_ctx; std::string object_name, target_object_name; string chunk_pool_name, op_name; int ret; unsigned max_thread = default_max_thread; std::map::const_iterator i; uint32_t report_period = default_report_period; ObjectCursor begin; ObjectCursor end; librados::pool_stat_t s; list pool_names; map stats; i = opts.find("op_name"); if (i != opts.end()) { op_name= i->second.c_str(); } else { cerr << "must specify op" << std::endl; exit(1); } i = opts.find("chunk-pool"); if (i != opts.end()) { chunk_pool_name = i->second.c_str(); } else { cerr << "must specify --chunk-pool" << std::endl; exit(1); } i = opts.find("max-thread"); if (i != opts.end()) { if (rados_sistrtoll(i, &max_thread)) { return -EINVAL; } } i = opts.find("report-period"); if (i != opts.end()) { if (rados_sistrtoll(i, &report_period)) { return -EINVAL; } } i = opts.find("pgid"); boost::optional pgid(i != opts.end(), pg_t()); ret = rados.init_with_context(g_ceph_context); if (ret < 0) { cerr << "couldn't initialize rados: " << cpp_strerror(ret) << std::endl; goto out; } ret = rados.connect(); if (ret) { cerr << "couldn't connect to cluster: " << cpp_strerror(ret) << std::endl; ret = -1; goto out; } ret = rados.ioctx_create(chunk_pool_name.c_str(), chunk_io_ctx); if (ret < 0) { cerr << "error opening pool " << chunk_pool_name << ": " << cpp_strerror(ret) << std::endl; goto out; } if (op_name == "chunk-get-ref" || op_name == "chunk-put-ref") { string target_object_name; uint64_t pool_id; i = opts.find("object"); if (i != opts.end()) { object_name = i->second.c_str(); } else { cerr << "must specify object" << std::endl; exit(1); } i = opts.find("target-ref"); if (i != opts.end()) { target_object_name = i->second.c_str(); } else { cerr << "must specify target ref" << std::endl; exit(1); } i = opts.find("target-ref-pool-id"); if (i != opts.end()) { if (rados_sistrtoll(i, &pool_id)) { return -EINVAL; } } else { cerr << "must specify target-ref-pool-id" << std::endl; exit(1); } uint32_t hash; ret = chunk_io_ctx.get_object_hash_position2(object_name, &hash); if (ret < 0) { return ret; } hobject_t oid(sobject_t(target_object_name, CEPH_NOSNAP), "", hash, pool_id, ""); ObjectWriteOperation op; if (op_name == "chunk-get-ref") { cls_cas_chunk_get_ref(op, oid); } else { cls_cas_chunk_put_ref(op, oid); } ret = chunk_io_ctx.operate(object_name, &op); if (ret < 0) { cerr << " operate fail : " << cpp_strerror(ret) << std::endl; } return ret; } else if (op_name == "dump-chunk-refs") { i = opts.find("object"); if (i != opts.end()) { object_name = i->second.c_str(); } else { cerr << "must specify object" << std::endl; exit(1); } bufferlist t; ret = chunk_io_ctx.getxattr(object_name, CHUNK_REFCOUNT_ATTR, t); if (ret < 0) { return ret; } chunk_refs_t refs; auto p = t.cbegin(); decode(refs, p); auto f = Formatter::create("json-pretty"); f->dump_object("refs", refs); f->flush(cout); return 0; } glock.lock(); begin = chunk_io_ctx.object_list_begin(); end = chunk_io_ctx.object_list_end(); pool_names.push_back(chunk_pool_name); ret = rados.get_pool_stats(pool_names, stats); if (ret < 0) { cerr << "error fetching pool stats: " << cpp_strerror(ret) << std::endl; glock.unlock(); return ret; } if (stats.find(chunk_pool_name) == stats.end()) { cerr << "stats can not find pool name: " << chunk_pool_name << std::endl; glock.unlock(); return ret; } s = stats[chunk_pool_name]; for (unsigned i = 0; i < max_thread; i++) { std::unique_ptr ptr ( new ChunkScrub(io_ctx, i, max_thread, begin, end, chunk_io_ctx, report_period, s.num_objects)); ptr->create("estimate_thread"); estimate_threads.push_back(move(ptr)); } glock.unlock(); for (auto &p : estimate_threads) { cout << "join " << std::endl; p->join(); cout << "joined " << std::endl; } print_chunk_scrub(); out: return (ret < 0) ? 1 : 0; } int main(int argc, const char **argv) { vector args; argv_to_vec(argc, argv, args); if (args.empty()) { cerr << argv[0] << ": -h or --help for usage" << std::endl; exit(1); } if (ceph_argparse_need_usage(args)) { usage(); exit(0); } std::string fn; string op_name; auto cct = global_init(NULL, args, CEPH_ENTITY_TYPE_CLIENT, CODE_ENVIRONMENT_UTILITY, 0); common_init_finish(g_ceph_context); init_async_signal_handler(); register_async_signal_handler_oneshot(SIGINT, handle_signal); register_async_signal_handler_oneshot(SIGTERM, handle_signal); std::map < std::string, std::string > opts; std::string val; std::vector::iterator i; for (i = args.begin(); i != args.end(); ) { if (ceph_argparse_double_dash(args, i)) { break; } else if (ceph_argparse_witharg(args, i, &val, "--op", (char*)NULL)) { opts["op_name"] = val; op_name = val; } else if (ceph_argparse_witharg(args, i, &val, "--pool", (char*)NULL)) { opts["pool"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--object", (char*)NULL)) { opts["object"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--chunk-algorithm", (char*)NULL)) { opts["chunk-algorithm"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--chunk-size", (char*)NULL)) { opts["chunk-size"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--fingerprint-algorithm", (char*)NULL)) { opts["fingerprint-algorithm"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--chunk-pool", (char*)NULL)) { opts["chunk-pool"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--target-ref", (char*)NULL)) { opts["target-ref"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--target-ref-pool-id", (char*)NULL)) { opts["target-ref-pool-id"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--max-thread", (char*)NULL)) { opts["max-thread"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--report-period", (char*)NULL)) { opts["report-period"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--max-read-size", (char*)NULL)) { opts["max-seconds"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--max-seconds", (char*)NULL)) { opts["max-seconds"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--min-chunk-size", (char*)NULL)) { opts["min-chunk-size"] = val; } else if (ceph_argparse_witharg(args, i, &val, "--max-chunk-size", (char*)NULL)) { opts["max-chunk-size"] = val; } else if (ceph_argparse_flag(args, i, "--debug", (char*)NULL)) { opts["debug"] = "true"; } else { if (val[0] == '-') { cerr << "unrecognized option " << val << std::endl; exit(1); } ++i; } } if (op_name == "estimate") { return estimate_dedup_ratio(opts, args); } else if (op_name == "chunk-scrub") { return chunk_scrub_common(opts, args); } else if (op_name == "chunk-get-ref" || op_name == "chunk-put-ref") { return chunk_scrub_common(opts, args); } else if (op_name == "dump-chunk-refs") { return chunk_scrub_common(opts, args); } else { cerr << "unrecognized op " << op_name << std::endl; exit(1); } unregister_async_signal_handler(SIGINT, handle_signal); unregister_async_signal_handler(SIGTERM, handle_signal); shutdown_async_signal_handler(); return 0; }