--- /dev/null
+======================
+Full RGW Object Dedup:
+======================
+Add a radosgw-admin command to collect and report deduplication stats
+
+.. note:: This utility doesn’t perform dedup and doesn’t make any
+ change to the existing system and will only collect
+ statistics and report them.
+
+----
+
+***************
+Admin commands:
+***************
+- ``radosgw-admin dedup stats``:
+ Collects & displays last dedup statistics
+- ``radosgw-admin dedup pause``:
+ Pauses active dedup session (dedup resources are not released)
+- ``radosgw-admin dedup resume``:
+ Resumes a paused dedup session
+- ``radosgw-admin dedup abort``:
+ Aborts active dedup session and release all resources used by it
+- ``radosgw-admin dedup estimate``
+ Starts a new dedup estimate session (aborting first existing session if exists)
+
+----
+
+****************
+Skipped Objects:
+****************
+Dedup Estimates skips the following objects:
+
+- Objects smaller than 4MB (unless they are multipart)
+- Objects with different placement rules
+- Objects with different pools
+- Objects with different same storage-classes
+
+The Dedup process itself (which will be released later) will also skip
+**compressed** and **user-encrypted** objects, but the estimate
+process will accept them (since we don't have access to that
+information during the estimate process)
+
+----
+
+********************
+Estimate Processing:
+********************
+The Dedup Estimate process collects all the needed information directly from
+the bucket-indices reading one full bucket-index object with 1000's of
+entries at a time.
+
+The Bucket-Indices objects are sharded between the participating
+members so every bucket-index object is read exactly one time.
+The sharding allow processing to scale almost linearly spliting the
+load evenly between the participating members.
+
+The Dedup Estimate process does not access the objects themselves
+(data/metadata) which means its processing time won't be affected by
+the underlined media storing the objects (SSD/HDD) since the bucket-indices are
+virtually always stored on a fast medium (SSD with heavy memory
+caching)
+
+----
+
+*************
+Memory Usage:
+*************
+ +---------------++-----------+
+ | RGW Obj Count | Memory |
+ +===============++===========+
+ | | ____1M | | ___8MB |
+ | | ____4M | | __16MB |
+ | | ___16M | | __32MB |
+ | | ___64M | | __64MB |
+ | | __256M | | _128MB |
+ | | _1024M( 1G) | | _256MB |
+ | | _4096M( 4G) | | _512MB |
+ | | 16384M(16G) | | 1024MB |
+ +---------------+------------+
SUBSYS(seastore_backref, 0, 5)
SUBSYS(alienstore, 0, 5)
SUBSYS(mclock, 1, 5)
+SUBSYS(rgw_dedup, 1, 5)
SUBSYS(cyanstore, 0, 5)
SUBSYS(ceph_exporter, 1, 5)
SUBSYS(memstore, 1, 5)
rgw_bucket_encryption.cc
rgw_tracer.cc
rgw_lua_background.cc
+ rgw_dedup.cc
+ rgw_dedup_table.cc
+ rgw_dedup_store.cc
+ rgw_dedup_utils.cc
+ rgw_dedup_cluster.cc
rgw_data_access.cc
driver/rados/account.cc
driver/rados/buckets.cc
#include "radosgw-admin/orphan.h"
#include "radosgw-admin/sync_checkpoint.h"
-
#include "rgw_user.h"
#include "rgw_otp.h"
#include "rgw_rados.h"
#include "rgw_data_access.h"
#include "rgw_account.h"
#include "rgw_bucket_logging.h"
-
+#include "rgw_dedup_cluster.h"
#include "services/svc_sync_modules.h"
#include "services/svc_cls.h"
#include "services/svc_bilog_rados.h"
cout << " user policy list attached list attached managed policies\n";
cout << " caps add add user capabilities\n";
cout << " caps rm remove user capabilities\n";
+ cout << " dedup stats Display dedup statistics from the last run\n";
+ cout << " dedup estimate Runs dedup in estimate mode (no changes will be made)\n";
+ cout << " dedup restart Restart dedup\n";
+ cout << " dedup abort Abort dedup\n";
+ cout << " dedup pause Pause dedup\n";
+ cout << " dedup resume Resume paused dedup\n";
cout << " subuser create create a new subuser\n" ;
cout << " subuser modify modify subuser\n";
cout << " subuser rm remove subuser\n";
QUOTA_SET,
QUOTA_ENABLE,
QUOTA_DISABLE,
+ DEDUP_STATS,
+ DEDUP_ESTIMATE,
+ DEDUP_ABORT,
+ DEDUP_RESTART,
+ DEDUP_PAUSE,
+ DEDUP_RESUME,
GC_LIST,
GC_PROCESS,
LC_LIST,
{ "ratelimit set", OPT::RATELIMIT_SET },
{ "ratelimit enable", OPT::RATELIMIT_ENABLE },
{ "ratelimit disable", OPT::RATELIMIT_DISABLE },
+ { "dedup stats", OPT::DEDUP_STATS },
+ { "dedup estimate", OPT::DEDUP_ESTIMATE },
+ { "dedup abort", OPT::DEDUP_ABORT },
+ { "dedup restart", OPT::DEDUP_RESTART },
+ { "dedup pause", OPT::DEDUP_PAUSE },
+ { "dedup resume", OPT::DEDUP_RESUME },
{ "gc list", OPT::GC_LIST },
{ "gc process", OPT::GC_PROCESS },
{ "lc list", OPT::LC_LIST },
OPT::BI_LIST,
OPT::OLH_GET,
OPT::OLH_READLOG,
+ OPT::DEDUP_STATS,
+ OPT::DEDUP_ESTIMATE,
+ OPT::DEDUP_ABORT, // TBD - not READ-ONLY
+ OPT::DEDUP_RESTART, // TBD - not READ-ONLY
+ OPT::DEDUP_PAUSE,
+ OPT::DEDUP_RESUME,
OPT::GC_LIST,
OPT::LC_LIST,
OPT::ORPHANS_LIST_JOBS,
}
}
+ if (opt_cmd == OPT::DEDUP_STATS ||
+ opt_cmd == OPT::DEDUP_ESTIMATE ||
+ opt_cmd == OPT::DEDUP_ABORT ||
+ opt_cmd == OPT::DEDUP_PAUSE ||
+ opt_cmd == OPT::DEDUP_RESUME ||
+ opt_cmd == OPT::DEDUP_RESTART) {
+
+ using namespace rgw::dedup;
+ rgw::sal::RadosStore *store = dynamic_cast<rgw::sal::RadosStore*>(driver);
+ if (!store) {
+ cerr << "ERROR: this command can only work when the cluster has a RADOS "
+ << "backing store." << std::endl;
+ return EPERM;
+ }
+
+ if (opt_cmd == OPT::DEDUP_STATS) {
+ int ret = cluster::collect_all_shard_stats(store, formatter.get(), dpp());
+ if (ret == 0) {
+ formatter->flush(cout);
+ }
+ else {
+ cerr << "ERROR: Failed reading stat counters" << std::endl;
+ }
+ return ret;
+ }
+
+ if (opt_cmd == OPT::DEDUP_ABORT || opt_cmd == OPT::DEDUP_PAUSE || opt_cmd == OPT::DEDUP_RESUME) {
+ urgent_msg_t urgent_msg;
+ if (opt_cmd == OPT::DEDUP_ABORT) {
+ urgent_msg = URGENT_MSG_ABORT;
+ }
+ else if (opt_cmd == OPT::DEDUP_PAUSE) {
+ urgent_msg = URGENT_MSG_PASUE;
+ }
+ else {
+ urgent_msg = URGENT_MSG_RESUME;
+ }
+ return cluster::dedup_control(store, dpp(), urgent_msg);
+ }
+
+ if (opt_cmd == OPT::DEDUP_RESTART || opt_cmd == OPT::DEDUP_ESTIMATE) {
+ dedup_req_type_t dedup_type = dedup_req_type_t::DEDUP_TYPE_NONE;
+ if (opt_cmd == OPT::DEDUP_ESTIMATE) {
+ dedup_type = dedup_req_type_t::DEDUP_TYPE_ESTIMATE;
+ }
+ else {
+ dedup_type = dedup_req_type_t::DEDUP_TYPE_FULL;
+#ifndef FULL_DEDUP_SUPPORT
+ std::cerr << "Only dedup estimate is supported!" << std::endl;
+ return EPERM;
+#endif
+ }
+
+ int ret = cluster::dedup_restart_scan(store, dedup_type, dpp());
+ if (ret == 0) {
+ std::cout << "Dedup was restarted successfully" << std::endl;
+ }
+ else {
+ std::cerr << "Dedup failed to restart" << std::endl;
+ std::cerr << "Error is: " << ret << "::" << cpp_strerror(ret) << std::endl;
+ }
+ return ret;
+ }
+ }
+
if (opt_cmd == OPT::GC_LIST) {
int index = 0;
bool truncated;
#include "rgw_asio_frontend.h"
#include "rgw_dmclock_scheduler_ctx.h"
#include "rgw_lua.h"
+#include "rgw_dedup.h"
#ifdef WITH_RADOSGW_DBSTORE
#include "rgw_sal_dbstore.h"
#endif
if (env.lua.background) {
rgw_pauser->add_pauser(env.lua.background);
}
+
+ if (dedup_background) {
+ rgw_pauser->add_pauser(dedup_background.get());
+ }
+
need_context_pool();
reloader = std::make_unique<RGWRealmReloader>(
env, *implicit_tenant_context, service_map_meta, rgw_pauser.get(), *context_pool);
rgw::sal::Driver* driver = env.driver;
int r{0};
std::string install_dir;
-
#ifdef WITH_RADOSGW_LUA_PACKAGES
rgw::lua::packages_t failed_packages;
r = rgw::lua::install_packages(dpp, driver, null_yield, g_conf().get_val<std::string>("rgw_luarocks_location"),
}
} /* init_lua */
+void rgw::AppMain::init_dedup()
+{
+ rgw::sal::Driver* driver = env.driver;
+ if (driver->get_name() == "rados") { /* Supported for only RadosStore */
+ try {
+ dedup_background = std::make_unique<rgw::dedup::Background>(driver, dpp->get_cct());
+ dedup_background->start();
+ dedup_background->watch_reload(dpp);
+ }
+ catch (const std::runtime_error&) {
+ ldpp_dout(dpp, 0) << __func__ << "::failed create dedup background job" << dendl;
+ }
+ }
+}
+
void rgw::AppMain::shutdown(std::function<void(void)> finalize_async_signals)
{
if (env.driver->get_name() == "rados") {
if (g_conf().get_val<bool>("rgw_lua_enable"))
static_cast<rgw::sal::RadosLuaManager*>(env.lua.manager.get())->
unwatch_reload(dpp);
+
+ if (dedup_background) {
+ dedup_background->unwatch_reload(dpp);
+ }
}
for (auto& fe : fes) {
ldh.reset(nullptr); // deletes ldap helper if it was created
rgw_log_usage_finalize();
+ if (dedup_background) {
+ dedup_background->shutdown();
+ }
+
if (lua_background) {
lua_background->shutdown();
}
#define RGW_ATTR_LC RGW_ATTR_PREFIX "lc"
#define RGW_ATTR_CORS RGW_ATTR_PREFIX "cors"
#define RGW_ATTR_ETAG RGW_ATTR_PREFIX "etag"
-#define RGW_ATTR_CKSUM RGW_ATTR_PREFIX "cksum"
+#define RGW_ATTR_CKSUM RGW_ATTR_PREFIX "cksum"
+#define RGW_ATTR_SHA256 RGW_ATTR_PREFIX "x-amz-content-sha256"
#define RGW_ATTR_BUCKETS RGW_ATTR_PREFIX "buckets"
#define RGW_ATTR_META_PREFIX RGW_ATTR_PREFIX RGW_AMZ_META_PREFIX
#define RGW_ATTR_CONTENT_TYPE RGW_ATTR_PREFIX "content_type"
#define RGW_ATTR_SHADOW_OBJ RGW_ATTR_PREFIX "shadow_name"
#define RGW_ATTR_MANIFEST RGW_ATTR_PREFIX "manifest"
#define RGW_ATTR_USER_MANIFEST RGW_ATTR_PREFIX "user_manifest"
+#define RGW_ATTR_SHARE_MANIFEST RGW_ATTR_PREFIX "shared_manifest"
#define RGW_ATTR_AMZ_WEBSITE_REDIRECT_LOCATION RGW_ATTR_PREFIX RGW_AMZ_WEBSITE_REDIRECT_LOCATION
#define RGW_ATTR_SLO_MANIFEST RGW_ATTR_PREFIX "slo_manifest"
/* Information whether an object is SLO or not must be exposed to
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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/rados/rados_types.hpp"
+#include "include/rados/buffer.h"
+#include "include/rados/librados.hpp"
+#include "rgw_tools.h"
+#include "svc_zone.h"
+#include "common/config.h"
+#include "common/Cond.h"
+#include "common/debug.h"
+#include "common/errno.h"
+#include "rgw_common.h"
+#include "rgw_sal.h"
+#include "rgw_zone.h"
+#include "rgw_cache.h"
+#include "rgw_acl.h"
+#include "rgw_acl_s3.h" /* for dumping s3policy in debug log */
+#include "rgw_aio_throttle.h"
+#include "driver/rados/rgw_bucket.h"
+#include "rgw_sal_config.h"
+#include "rgw_lib.h"
+#include "rgw_placement_types.h"
+#include "driver/rados/rgw_bucket.h"
+#include "driver/rados/rgw_sal_rados.h"
+#include "cls/rgw/cls_rgw_ops.h"
+#include "cls/rgw/cls_rgw_client.h"
+#include "cls/rgw/cls_rgw_const.h"
+#include "cls/refcount/cls_refcount_client.h"
+#include "cls/version/cls_version_client.h"
+#include "fmt/ranges.h"
+#include "osd/osd_types.h"
+#include "common/ceph_crypto.h"
+
+#include <filesystem>
+#include <algorithm>
+#include <iostream>
+#include <fstream>
+#include <stdlib.h>
+#include <time.h>
+#include <sstream>
+#include <errno.h>
+#include <dirent.h>
+#include <stdexcept>
+#include <limits>
+#include <climits>
+#include <cinttypes>
+#include <cstring>
+#include <span>
+#include <mutex>
+#include <thread>
+
+//using namespace std::chrono_literals;
+using namespace librados;
+using namespace std;
+using namespace rgw::dedup;
+
+#include "rgw_dedup_remap.h"
+#include "rgw_sal_rados.h"
+#include "rgw_dedup_table.h"
+#include "rgw_dedup_utils.h"
+#include "rgw_dedup.h"
+#include "rgw_dedup_store.h"
+#include "rgw_dedup_cluster.h"
+#include "rgw_dedup_epoch.h"
+#include "rgw_perf_counters.h"
+#include "include/ceph_assert.h"
+
+static constexpr auto dout_subsys = ceph_subsys_rgw_dedup;
+
+namespace rgw::dedup {
+ static inline constexpr unsigned MAX_STORAGE_CLASS_IDX = 128;
+ using storage_class_idx_t = uint8_t;
+
+ //---------------------------------------------------------------------------
+ void Background::DedupWatcher::handle_notify(uint64_t notify_id, uint64_t cookie,
+ uint64_t notifier_id, bufferlist &bl)
+ {
+ ldpp_dout(parent->dpp, 10) << __func__ << "::notify_id=" << notify_id
+ << "::cookie=" << cookie
+ << "::notifier_id=" << notifier_id << dendl;
+ if (parent->d_watch_handle != cookie) {
+ ldpp_dout(parent->dpp, 1) << __func__ << "::ERR: wrong cookie=" << cookie
+ << "::d_watch_handle=" << parent->d_watch_handle
+ << dendl;
+ return;
+ }
+ parent->handle_notify(notify_id, cookie, bl);
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::DedupWatcher::handle_error(uint64_t cookie, int err)
+ {
+ if (parent->d_watch_handle != cookie) {
+ ldpp_dout(parent->dpp, 1) << __func__ << "::ERR: wrong cookie=" << cookie
+ << "::d_watch_handle=" << parent->d_watch_handle
+ << dendl;
+ return;
+ }
+ ldpp_dout(parent->dpp, 1) << __func__ << "::error=" << err << dendl;
+
+ parent->unwatch_reload(parent->dpp);
+ parent->watch_reload(parent->dpp);
+ }
+
+ //---------------------------------------------------------------------------
+ void control_t::reset()
+ {
+ this->dedup_type = dedup_req_type_t::DEDUP_TYPE_NONE;
+ this->started = false;
+ this->dedup_exec = false;
+ this->shutdown_req = false;
+ this->shutdown_done = false;
+ this->local_pause_req = false;
+ this->local_paused = false;
+ this->remote_abort_req = false;
+ this->remote_aborted = false;
+ this->remote_pause_req = false;
+ this->remote_paused = false;
+ this->remote_restart_req = false;
+ }
+
+ //---------------------------------------------------------------------------
+ void encode(const control_t& ctl, ceph::bufferlist& bl)
+ {
+ ENCODE_START(1, 1, bl);
+ encode(static_cast<int32_t>(ctl.dedup_type), bl);
+ encode(ctl.started, bl);
+ encode(ctl.dedup_exec, bl);
+ encode(ctl.shutdown_req, bl);
+ encode(ctl.shutdown_done, bl);
+ encode(ctl.local_pause_req, bl);
+ encode(ctl.local_paused, bl);
+ encode(ctl.remote_abort_req, bl);
+ encode(ctl.remote_aborted, bl);
+ encode(ctl.remote_pause_req, bl);
+ encode(ctl.remote_paused, bl);
+ encode(ctl.remote_restart_req, bl);
+ ENCODE_FINISH(bl);
+ }
+
+ //---------------------------------------------------------------------------
+ void decode(control_t& ctl, ceph::bufferlist::const_iterator& bl)
+ {
+ DECODE_START(1, bl);
+ int32_t dedup_type;
+ decode(dedup_type, bl);
+ ctl.dedup_type = static_cast<dedup_req_type_t> (dedup_type);
+ decode(ctl.started, bl);
+ decode(ctl.dedup_exec, bl);
+ decode(ctl.shutdown_req, bl);
+ decode(ctl.shutdown_done, bl);
+ decode(ctl.local_pause_req, bl);
+ decode(ctl.local_paused, bl);
+ decode(ctl.remote_abort_req, bl);
+ decode(ctl.remote_aborted, bl);
+ decode(ctl.remote_pause_req, bl);
+ decode(ctl.remote_paused, bl);
+ decode(ctl.remote_restart_req, bl);
+ DECODE_FINISH(bl);
+ }
+
+ //---------------------------------------------------------------------------
+ std::ostream& operator<<(std::ostream &out, const control_t &ctl)
+ {
+ out << ctl.dedup_type;
+ if (ctl.started) {
+ out << "::started";
+ }
+ if (ctl.dedup_exec) {
+ out << "::dedup_exec";
+ }
+ if (ctl.shutdown_req) {
+ out << "::shutdown_req";
+ }
+ if (ctl.shutdown_done) {
+ out << "::shutdown_done";
+ }
+ if (ctl.local_pause_req) {
+ out << "::local_pause_req";
+ }
+ if (ctl.local_paused) {
+ out << "::local_paused";
+ }
+ if (ctl.remote_abort_req) {
+ out << "::remote_abort_req";
+ }
+ if (ctl.remote_aborted) {
+ out << "::remote_aborted";
+ }
+ if (ctl.remote_pause_req) {
+ out << "::remote_pause_req";
+ }
+ if (ctl.remote_paused) {
+ out << "::remote_paused";
+ }
+ if (ctl.remote_restart_req) {
+ out << "::remote_restart_req";
+ }
+
+ return out;
+ }
+
+ //===========================================================================
+ // rgw::dedup::Background
+ //===========================================================================
+ //---------------------------------------------------------------------------
+ int Background::init_rados_access_handles()
+ {
+ store = dynamic_cast<rgw::sal::RadosStore*>(driver);
+ if (!store) {
+ ldpp_dout(dpp, 0) << "ERR: failed dynamic_cast to RadosStore" << dendl;
+ // this is the return code used in rgw_bucket.cc
+ return -ENOTSUP;
+ }
+
+ rados = store->getRados();
+ rados_handle = rados->get_rados_handle();
+
+ int ret = init_dedup_pool_ioctx(rados, dpp, d_dedup_cluster_ioctx);
+ ldpp_dout(dpp, 5) << __func__ << "::dedup background: ioctx="
+ << d_dedup_cluster_ioctx.get_instance_id() << dendl;
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ Background::Background(rgw::sal::Driver* _driver, CephContext* _cct) :
+ driver(_driver),
+ dp(_cct, dout_subsys, "dedup background: "),
+ dpp(&dp),
+ cct(_cct),
+ d_cluster(dpp, cct, driver),
+ d_watcher_ctx(this)
+ {
+ d_min_obj_size_for_dedup = cct->_conf->rgw_max_chunk_size;
+ d_head_object_size = cct->_conf->rgw_max_chunk_size;
+ //ceph_assert(4*1024*1024 == d_head_object_size);
+
+ int ret = init_rados_access_handles();
+ if (ret != 0) {
+ derr << __func__ << "::ERR: failed init_rados_access_handles() ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ throw std::runtime_error("Failed init_dedup_pool_ioctx()");
+ }
+
+ d_heart_beat_last_update = ceph_clock_now();
+ d_heart_beat_max_elapsed_sec = 3;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::add_disk_rec_from_bucket_idx(disk_block_array_t &disk_arr,
+ const rgw::sal::Bucket *p_bucket,
+ const parsed_etag_t *p_parsed_etag,
+ const std::string &obj_name,
+ uint64_t obj_size,
+ const std::string &storage_class)
+ {
+ disk_record_t rec(p_bucket, obj_name, p_parsed_etag, obj_size, storage_class);
+ // First pass using only ETAG and size taken from bucket-index
+ rec.s.flags.set_fastlane();
+
+ auto p_disk = disk_arr.get_shard_block_seq(p_parsed_etag->md5_low);
+ disk_block_seq_t::record_info_t rec_info;
+ int ret = p_disk->add_record(d_dedup_cluster_ioctx, &rec, &rec_info);
+ if (unlikely(ret != 0)) {
+ return ret;
+ }
+ ldpp_dout(dpp, 20) << __func__ << "::" << p_bucket->get_name() << "/"
+ << obj_name << " was written to block_idx="
+ << rec_info.block_id << " rec_id=" << rec_info.rec_id << dendl;
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::add_record_to_dedup_table(dedup_table_t *p_table,
+ const disk_record_t *p_rec,
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ md5_stats_t *p_stats,
+ remapper_t *remapper)
+ {
+ uint32_t size_4k_units = byte_size_to_disk_blocks(p_rec->s.obj_bytes_size);
+ storage_class_idx_t sc_idx = remapper->remap(p_rec->stor_class, dpp,
+ &p_stats->failed_map_overflow);
+ if (unlikely(sc_idx == remapper_t::NULL_IDX)) {
+ // TBD: need stat counters
+ return -EOVERFLOW;
+ }
+ key_t key(p_rec->s.md5_high, p_rec->s.md5_low, size_4k_units,
+ p_rec->s.num_parts, sc_idx);
+ bool has_shared_manifest = p_rec->has_shared_manifest();
+ ldpp_dout(dpp, 20) << __func__ << "::bucket=" << p_rec->bucket_name
+ << ", obj=" << p_rec->obj_name << ", block_id="
+ << (uint32_t)block_id << ", rec_id=" << (uint32_t)rec_id
+ << ", shared_manifest=" << has_shared_manifest
+ << "::num_parts=" << p_rec->s.num_parts
+ << "::size_4k_units=" << key.size_4k_units
+ << "::ETAG=" << std::hex << p_rec->s.md5_high
+ << p_rec->s.md5_low << std::dec << dendl;
+
+ int ret = p_table->add_entry(&key, block_id, rec_id, has_shared_manifest);
+ if (ret == 0) {
+ p_stats->loaded_objects ++;
+ ldpp_dout(dpp, 20) << __func__ << "::" << p_rec->bucket_name << "/"
+ << p_rec->obj_name << " was added successfully to table"
+ << "::loaded_objects=" << p_stats->loaded_objects << dendl;
+ }
+ else {
+ // We allocate memory for the dedup on startup based on the existing obj count
+ // If the system grew significantly since that point we won't be able to
+ // accommodate all the objects in the hash-table.
+ // Please keep in mind that it is very unlikely since duplicates objects will
+ // consume a single entry and since we skip small objects so in reality
+ // I expect the allocation to be more than sufficient.
+ //
+ // However, if we filled up the system there is still value is continuing
+ // with this process since we might find duplicates to existing object (which
+ // don't take extra space)
+
+ int level = 15;
+ if (p_stats->failed_table_load % 0x10000 == 0) {
+ level = 5;
+ }
+ else if (p_stats->failed_table_load % 0x100 == 0) {
+ level = 10;
+ }
+ ldpp_dout(dpp, level) << __func__ << "::Failed p_table->add_entry (overflow)"
+ << "::loaded_objects=" << p_stats->loaded_objects
+ << "::failed_table_load=" << p_stats->failed_table_load
+ << dendl;
+
+ p_stats->failed_table_load++;
+ }
+ return ret;
+ }
+
+#ifdef FULL_DEDUP_SUPPORT
+
+ static constexpr uint64_t cost = 1; // 1 throttle unit per request
+ static constexpr uint64_t id = 0; // ids unused
+ //---------------------------------------------------------------------------
+ [[maybe_unused]]static void show_ref_tags(const DoutPrefixProvider* dpp, std::string &oid, rgw_rados_ref &obj)
+ {
+ unsigned idx = 0;
+ std::list<std::string> refs;
+ std::string wildcard_tag;
+ int ret = cls_refcount_read(obj.ioctx, oid, &refs, true);
+ if (ret < 0) {
+ ldpp_dout(dpp, 0) << __func__ << "::ERR: manifest::failed cls_refcount_read()"
+ << " idx=" << idx << dendl;
+ return;
+ }
+
+ for (list<string>::iterator iter = refs.begin(); iter != refs.end(); ++iter) {
+ ldpp_dout(dpp, 20) << __func__ << "::manifest::" << oid << "::" << idx
+ << "::TAG=" << *iter << dendl;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::free_tail_objs_by_manifest(const string &ref_tag,
+ const string &oid,
+ RGWObjManifest &tgt_manifest)
+ {
+ unsigned idx = 0;
+ for (auto p = tgt_manifest.obj_begin(dpp); p != tgt_manifest.obj_end(dpp); ++p, ++idx) {
+ rgw_raw_obj raw_obj = p.get_location().get_raw_obj(rados);
+ if (oid == raw_obj.oid) {
+ ldpp_dout(dpp, 20) << __func__ << "::[" << idx <<"] Skip HEAD OBJ: " << raw_obj.oid << dendl;
+ continue;
+ }
+
+ rgw_rados_ref obj;
+ int ret = rgw_get_rados_ref(dpp, rados_handle, raw_obj, &obj);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "ERR: manifest::failed to open context "
+ << obj << dendl;
+ continue;
+ }
+ librados::IoCtx ioctx = obj.ioctx;
+ ldpp_dout(dpp, 20) << __func__ << "::removing tail object: " << raw_obj.oid
+ << dendl;
+ ret = ioctx.remove(raw_obj.oid);
+ }
+
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::rollback_ref_by_manifest(const string &ref_tag,
+ const string &oid,
+ RGWObjManifest &manifest)
+ {
+ unsigned idx = 0;
+ int ret_code = 0;
+ std::unique_ptr<rgw::Aio> aio = rgw::make_throttle(cct->_conf->rgw_max_copy_obj_concurrent_io, null_yield);
+ for (auto p = manifest.obj_begin(dpp); p != manifest.obj_end(dpp); ++p, ++idx) {
+ rgw_raw_obj raw_obj = p.get_location().get_raw_obj(rados);
+ if (oid == raw_obj.oid) {
+ ldpp_dout(dpp, 20) << __func__ << "::[" << idx <<"] Skip HEAD OBJ: "
+ << raw_obj.oid << dendl;
+ continue;
+ }
+
+ rgw_rados_ref obj;
+ int local_ret = rgw_get_rados_ref(dpp, rados_handle, raw_obj, &obj);
+ if (local_ret < 0) {
+ ret_code = local_ret;
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: manifest::failed to open context "
+ << obj << dendl;
+ // skip bad objects, nothing we can do
+ continue;
+ }
+
+ ObjectWriteOperation op;
+ cls_refcount_put(op, ref_tag, true);
+ rgw::AioResultList completed = aio->get(obj.obj,
+ rgw::Aio::librados_op(obj.ioctx, std::move(op), null_yield),
+ cost, id);
+ }
+ rgw::AioResultList completed = aio->drain();
+ return ret_code;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::inc_ref_count_by_manifest(const string &ref_tag,
+ const string &oid,
+ RGWObjManifest &manifest)
+ {
+ std::unique_ptr<rgw::Aio> aio = rgw::make_throttle(cct->_conf->rgw_max_copy_obj_concurrent_io, null_yield);
+ rgw::AioResultList all_results;
+ int ret = 0;
+ unsigned idx = 0;
+ for (auto p = manifest.obj_begin(dpp); p != manifest.obj_end(dpp); ++p, ++idx) {
+ rgw_raw_obj raw_obj = p.get_location().get_raw_obj(rados);
+ if (oid == raw_obj.oid) {
+ ldpp_dout(dpp, 20) << __func__ << "::[" << idx <<"] Skip HEAD OBJ: " << raw_obj.oid << dendl;
+ continue;
+ }
+
+ rgw_rados_ref obj;
+ ret = rgw_get_rados_ref(dpp, rados_handle, raw_obj, &obj);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: manifest::failed to open context "
+ << obj << dendl;
+ break;
+ }
+
+ ObjectWriteOperation op;
+ cls_refcount_get(op, ref_tag, true);
+ ldpp_dout(dpp, 20) << __func__ << "::inc ref-count on tail object: " << raw_obj.oid << dendl;
+ rgw::AioResultList completed = aio->get(obj.obj,
+ rgw::Aio::librados_op(obj.ioctx, std::move(op), null_yield),
+ cost, id);
+ ret = rgw::check_for_errors(completed);
+ all_results.splice(all_results.end(), completed);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: failed to copy obj=" << obj
+ << ", the error code = " << ret << dendl;
+ break;
+ }
+ }
+
+ if (ret == 0) {
+ rgw::AioResultList completed = aio->drain();
+ int ret = rgw::check_for_errors(completed);
+ all_results.splice(all_results.end(), completed);
+ if (ret == 0) {
+ return 0;
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: manifest: failed to drain ios ret="
+ << ret <<dendl;
+ }
+ }
+
+ // if arrived here we failed somewhere -> rollback all ref-inc operations
+ /* wait all pending op done */
+ rgw::AioResultList completed = aio->drain();
+ all_results.splice(all_results.end(), completed);
+ int ret2 = 0;
+ for (auto& aio_res : all_results) {
+ if (aio_res.result < 0) {
+ continue; // skip errors
+ }
+ rgw_rados_ref obj;
+ ret2 = rgw_get_rados_ref(dpp, rados_handle, aio_res.obj, &obj);
+ if (ret2 < 0) {
+ continue;
+ }
+
+ ObjectWriteOperation op;
+ cls_refcount_put(op, ref_tag, true);
+ rgw::AioResultList completed = aio->get(obj.obj,
+ rgw::Aio::librados_op(obj.ioctx, std::move(op), null_yield),
+ cost, id);
+ ret2 = rgw::check_for_errors(completed);
+ if (ret2 < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: cleanup after error failed to drop reference on obj=" << aio_res.obj << dendl;
+ }
+ }
+ completed = aio->drain();
+ ret2 = rgw::check_for_errors(completed);
+ if (ret2 < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed to drain rollback ios, ret="
+ << ret2 <<dendl;
+ }
+
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ static int get_ioctx(const DoutPrefixProvider* const dpp,
+ rgw::sal::Driver* driver,
+ RGWRados* rados,
+ const disk_record_t *p_rec,
+ librados::IoCtx *p_ioctx,
+ std::string *oid)
+ {
+ unique_ptr<rgw::sal::Bucket> bucket;
+ {
+ rgw_bucket b{p_rec->tenant_name, p_rec->bucket_name, p_rec->bucket_id};
+ int ret = driver->load_bucket(dpp, b, &bucket, null_yield);
+ if (unlikely(ret != 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: driver->load_bucket(): "
+ << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+ }
+
+ build_oid(p_rec->bucket_id, p_rec->obj_name, oid);
+ //ldpp_dout(dpp, 0) << __func__ << "::OID=" << oid << " || bucket_id=" << bucket_id << dendl;
+ rgw_pool data_pool;
+ rgw_obj obj{bucket->get_key(), *oid};
+ if (!rados->get_obj_data_pool(bucket->get_placement_rule(), obj, &data_pool)) {
+ ldpp_dout(dpp, 1) << __func__ << "::failed to get data pool for bucket "
+ << bucket->get_name() << dendl;
+ return -EIO;
+ }
+ int ret = rgw_init_ioctx(dpp, rados->get_rados_handle(), data_pool, *p_ioctx);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed to get ioxtc from data pool:"
+ << data_pool.to_str() << dendl;
+ return -EIO;
+ }
+
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ static void init_cmp_pairs(const disk_record_t *p_rec,
+ const bufferlist &etag_bl,
+ bufferlist &sha256_bl, // OUT PARAM
+ librados::ObjectWriteOperation *p_op)
+ {
+ p_op->cmpxattr(RGW_ATTR_ETAG, CEPH_OSD_CMPXATTR_OP_EQ, etag_bl);
+ // TBD: do we really need the secondary compare using the full manifest?
+ // Can replace it with something cheaper like size/version?
+ p_op->cmpxattr(RGW_ATTR_MANIFEST, CEPH_OSD_CMPXATTR_OP_EQ, p_rec->manifest_bl);
+
+ // SHA has 256 bit splitted into multiple 64bit units
+ const unsigned units = (256 / (sizeof(uint64_t)*8));
+ static_assert(units == 4);
+ for (unsigned i = 0; i < units; i++) {
+ ceph::encode(p_rec->s.sha256[i], sha256_bl);
+ }
+
+ if (!p_rec->s.flags.sha256_calculated()) {
+ p_op->cmpxattr(RGW_ATTR_SHA256, CEPH_OSD_CMPXATTR_OP_EQ, sha256_bl);
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::dedup_object(const disk_record_t *p_src_rec,
+ const disk_record_t *p_tgt_rec,
+ md5_stats_t *p_stats,
+ bool has_shared_manifest_src)
+ {
+ RGWObjManifest src_manifest;
+ try {
+ auto bl_iter = p_src_rec->manifest_bl.cbegin();
+ decode(src_manifest, bl_iter);
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: bad src manifest" << dendl;
+ return -EINVAL;
+ }
+ RGWObjManifest tgt_manifest;
+ try {
+ auto bl_iter = p_tgt_rec->manifest_bl.cbegin();
+ decode(tgt_manifest, bl_iter);
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: bad tgt manifest" << dendl;
+ return -EINVAL;
+ }
+ ldpp_dout(dpp, 20) << __func__ << "::DEDUP From: "
+ << p_src_rec->bucket_name << "/" << p_src_rec->obj_name << " -> "
+ << p_tgt_rec->bucket_name << "/" << p_tgt_rec->obj_name << dendl;
+
+ bufferlist etag_bl;
+ etag_to_bufferlist(p_tgt_rec->s.md5_high, p_tgt_rec->s.md5_low, p_tgt_rec->s.num_parts, &etag_bl);
+ ldpp_dout(dpp, 20) << __func__ << "::num_parts=" << p_tgt_rec->s.num_parts
+ << "::ETAG=" << etag_bl.to_str() << dendl;
+
+ bufferlist hash_bl, manifest_hash_bl, tgt_sha256_bl;
+ crypto::digest<crypto::SHA1>(p_src_rec->manifest_bl).encode(hash_bl);
+ // Use a shorter hash (64bit instead of 160bit)
+ hash_bl.splice(0, 8, &manifest_hash_bl);
+ librados::ObjectWriteOperation tgt_op;
+ init_cmp_pairs(p_tgt_rec, etag_bl, tgt_sha256_bl, &tgt_op);
+ tgt_op.setxattr(RGW_ATTR_SHARE_MANIFEST, manifest_hash_bl);
+ tgt_op.setxattr(RGW_ATTR_MANIFEST, p_src_rec->manifest_bl);
+ if (p_tgt_rec->s.flags.sha256_calculated()) {
+ tgt_op.setxattr(RGW_ATTR_SHA256, tgt_sha256_bl);
+ p_stats->set_sha256_attrs++;
+ }
+
+ std::string src_oid, tgt_oid;
+ librados::IoCtx src_ioctx, tgt_ioctx;
+ int ret1 = get_ioctx(dpp, driver, rados, p_src_rec, &src_ioctx, &src_oid);
+ int ret2 = get_ioctx(dpp, driver, rados, p_tgt_rec, &tgt_ioctx, &tgt_oid);
+ if (unlikely(ret1 != 0 || ret2 != 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed get_ioctx()" << dendl;
+ return (ret1 ? ret1 : ret2);
+ }
+
+ // TBD: Do we need to remove target RGW_ATTR_TAIL_TAG??
+ string ref_tag = p_tgt_rec->ref_tag;
+ ldpp_dout(dpp, 20) << __func__ << "::ref_tag=" << ref_tag << dendl;
+ int ret = inc_ref_count_by_manifest(ref_tag, src_oid, src_manifest);
+ if (ret == 0) {
+ ldpp_dout(dpp, 20) << __func__ << "::send TGT CLS (Shared_Manifest)" << dendl;
+ ret = tgt_ioctx.operate(tgt_oid, &tgt_op);
+ if (unlikely(ret != 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed tgt_ioctx.operate("
+ << tgt_oid << "), err is " << cpp_strerror(-ret) << dendl;
+ rollback_ref_by_manifest(ref_tag, src_oid, src_manifest);
+ return ret;
+ }
+
+ // free tail objects based on TGT manifest
+ free_tail_objs_by_manifest(ref_tag, tgt_oid, tgt_manifest);
+
+ if (!has_shared_manifest_src) {
+ // When SRC OBJ A has two or more dups (B, C) we set SHARED_MANIFEST
+ // after deduping B and update it in dedup_table, but don't update the
+ // disk-record (as require an expensive random-disk-write).
+ // When deduping C we can trust the shared_manifest state in the table and
+ // skip a redundant update to SRC object attribute
+ bufferlist src_sha256_bl;
+ librados::ObjectWriteOperation src_op;
+ init_cmp_pairs(p_src_rec, etag_bl, src_sha256_bl, &src_op);
+ src_op.setxattr(RGW_ATTR_SHARE_MANIFEST, manifest_hash_bl);
+ if (p_src_rec->s.flags.sha256_calculated()) {
+ src_op.setxattr(RGW_ATTR_SHA256, src_sha256_bl);
+ p_stats->set_sha256_attrs++;
+ }
+
+ ldpp_dout(dpp, 20) << __func__ <<"::send SRC CLS (Shared_Manifest)"<< dendl;
+ ret = src_ioctx.operate(src_oid, &src_op);
+ if (unlikely(ret != 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed src_ioctx.operate("
+ << src_oid << "), err is " << cpp_strerror(-ret)<<dendl;
+ return ret;
+ }
+ }
+ }
+
+ // do we need to set compression on the head object or is it set on tail?
+ // RGW_ATTR_COMPRESSION
+ return ret;
+ }
+
+ using ceph::crypto::SHA256;
+ //---------------------------------------------------------------------------
+ int Background::calc_object_sha256(const disk_record_t *p_rec, uint8_t *p_sha256)
+ {
+ ldpp_dout(dpp, 20) << __func__ << "::p_rec->obj_name=" << p_rec->obj_name << dendl;
+ // Open questions -
+ // 1) do we need the secret if so what is the correct one to use?
+ // 2) are we passing the head/tail objects in the correct order?
+ RGWObjManifest manifest;
+ try {
+ auto bl_iter = p_rec->manifest_bl.cbegin();
+ decode(manifest, bl_iter);
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: bad src manifest" << dendl;
+ return -EINVAL;
+ }
+ std::string oid;
+ build_oid(p_rec->bucket_id, p_rec->obj_name, &oid);
+ librados::IoCtx head_ioctx;
+ const char *secret = "0555b35654ad1656d804f1b017cd26e9";
+ TOPNSPC::crypto::HMACSHA256 hmac((const uint8_t*)secret, strlen(secret));
+ for (auto p = manifest.obj_begin(dpp); p != manifest.obj_end(dpp); ++p) {
+ rgw_raw_obj raw_obj = p.get_location().get_raw_obj(rados);
+ rgw_rados_ref obj;
+ int ret = rgw_get_rados_ref(dpp, rados_handle, raw_obj, &obj);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::failed rgw_get_rados_ref() for raw_obj="
+ << raw_obj << dendl;
+ return ret;
+ }
+
+ if (oid == raw_obj.oid) {
+ ldpp_dout(dpp, 10) << __func__ << "::manifest: head object=" << oid << dendl;
+ head_ioctx = obj.ioctx;
+ }
+ bufferlist bl;
+ librados::IoCtx ioctx = obj.ioctx;
+ // read full object
+ ret = ioctx.read(raw_obj.oid, bl, 0, 0);
+ if (ret > 0) {
+ for (const auto& bptr : bl.buffers()) {
+ hmac.Update((const unsigned char *)bptr.c_str(), bptr.length());
+ }
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed to read " << oid
+ << ", error is " << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+ }
+ hmac.Final(p_sha256);
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ [[maybe_unused]]static void __attribute__ ((noinline))
+ print_record(const DoutPrefixProvider* dpp,
+ const disk_record_t *p_tgt_rec,
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ md5_shard_t md5_shard)
+ {
+ ldpp_dout(dpp, 20) << __func__ << "::bucket=" << p_tgt_rec->bucket_name
+ << ", obj=" << p_tgt_rec->obj_name
+ << ", block_id=" << block_id
+ << ", rec_id=" << (int)rec_id
+ << ", md5_shard=" << (int)md5_shard << dendl;
+
+ ldpp_dout(dpp, 20) << __func__ << "::md5_shard=" << (int)md5_shard
+ << "::" << p_tgt_rec->bucket_name
+ << "/" << p_tgt_rec->obj_name
+ << "::num_parts=" << p_tgt_rec->s.num_parts
+ << "::ETAG=" << std::hex << p_tgt_rec->s.md5_high
+ << p_tgt_rec->s.md5_low << std::dec << dendl;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::add_obj_attrs_to_record(rgw_bucket *p_rb,
+ disk_record_t *p_rec,
+ const rgw::sal::Attrs &attrs,
+ dedup_table_t *p_table,
+ md5_stats_t *p_stats) /*IN-OUT*/
+ {
+ // if TAIL_TAG exists -> use it as ref-tag, eitherwise take ID_TAG
+ auto itr = attrs.find(RGW_ATTR_TAIL_TAG);
+ if (itr != attrs.end()) {
+ p_rec->ref_tag = itr->second.to_str();
+ }
+ else {
+ itr = attrs.find(RGW_ATTR_ID_TAG);
+ if (itr != attrs.end()) {
+ p_rec->ref_tag = itr->second.to_str();
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::No TAIL_TAG and no ID_TAG" << dendl;
+ return -EINVAL;
+ }
+ }
+ p_rec->s.ref_tag_len = p_rec->ref_tag.length();
+
+ // clear bufferlist first
+ p_rec->manifest_bl.clear();
+
+ itr = attrs.find(RGW_ATTR_MANIFEST);
+ if (itr != attrs.end()) {
+ const bufferlist &bl = itr->second;
+ RGWObjManifest manifest;
+ try {
+ auto bl_iter = bl.cbegin();
+ decode(manifest, bl_iter);
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__
+ << "::ERROR: unable to decode manifest" << dendl;
+ return -EINVAL;
+ }
+
+ // force explicit tail_placement as the dedup could be on another bucket
+ const rgw_bucket_placement& tail_placement = manifest.get_tail_placement();
+ if (tail_placement.bucket.name.empty()) {
+ ldpp_dout(dpp, 20) << __func__ << "dedup::updating tail placement" << dendl;
+ manifest.set_tail_placement(tail_placement.placement_rule, *p_rb);
+ encode(manifest, p_rec->manifest_bl);
+ }
+ else {
+ p_rec->manifest_bl = bl;
+ }
+ p_rec->s.manifest_len = p_rec->manifest_bl.length();
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::ERROR: no manifest" << dendl;
+ return -EINVAL;
+ }
+
+ itr = attrs.find(RGW_ATTR_SHARE_MANIFEST);
+ if (itr != attrs.end()) {
+ uint64_t hash = 0;
+ try {
+ auto bl_iter = itr->second.cbegin();
+ ceph::decode(hash, bl_iter);
+ p_rec->s.shared_manifest = hash;
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: bad shared_manifest" << dendl;
+ return -EINVAL;
+ }
+ ldpp_dout(dpp, 20) << __func__ << "::Set Shared_Manifest::OBJ_NAME="
+ << p_rec->obj_name << "::shared_manifest=0x" << std::hex
+ << p_rec->s.shared_manifest << std::dec << dendl;
+ p_rec->s.flags.set_shared_manifest();
+ }
+ else {
+ memset(&p_rec->s.shared_manifest, 0, sizeof(p_rec->s.shared_manifest));
+ }
+
+ itr = attrs.find(RGW_ATTR_SHA256);
+ if (itr != attrs.end()) {
+ try {
+ auto bl_iter = itr->second.cbegin();
+ // SHA has 256 bit splitted into multiple 64bit units
+ const unsigned units = (256 / (sizeof(uint64_t)*8));
+ static_assert(units == 4);
+ for (unsigned i = 0; i < units; i++) {
+ uint64_t val;
+ ceph::decode(val, bl_iter);
+ p_rec->s.sha256[i] = val;
+ }
+ p_stats->valid_sha256_attrs++;
+ return 0;
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed SHA256 decode" << dendl;
+ return -EINVAL;
+ }
+ }
+
+ p_stats->invalid_sha256_attrs++;
+ // TBD: redundant memset...
+ memset(p_rec->s.sha256, 0, sizeof(p_rec->s.sha256));
+ // CEPH_CRYPTO_HMACSHA256_DIGESTSIZE is 32 Bytes (32*8=256)
+ int ret = calc_object_sha256(p_rec, (uint8_t*)p_rec->s.sha256);
+ if (ret == 0) {
+ p_rec->s.flags.set_sha256_calculated();
+ }
+
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ // We purged all entries not marked for-dedup (i.e. singleton bit is set) from the table
+ // so all entries left are sources of dedup with multiple copies.
+ // Need to read attributes from the Head-Object and output them to a new SLAB
+ int Background::read_object_attribute(dedup_table_t *p_table,
+ disk_record_t *p_rec,
+ disk_block_id_t old_block_id,
+ record_id_t old_rec_id,
+ md5_shard_t md5_shard,
+ md5_stats_t *p_stats /* IN-OUT */,
+ disk_block_seq_t *p_disk,
+ remapper_t *remapper)
+ {
+ bool should_print_debug = cct->_conf->subsys.should_gather<ceph_subsys_rgw_dedup, 20>();
+ if (unlikely(should_print_debug)) {
+ print_record(dpp, p_rec, old_block_id, old_rec_id, md5_shard);
+ }
+
+ p_stats->processed_objects ++;
+ uint32_t size_4k_units = byte_size_to_disk_blocks(p_rec->s.obj_bytes_size);
+ uint64_t ondisk_byte_size = disk_blocks_to_byte_size(size_4k_units);
+ storage_class_idx_t sc_idx = remapper->remap(p_rec->stor_class, dpp,
+ &p_stats->failed_map_overflow);
+ if (unlikely(sc_idx == remapper_t::NULL_IDX)) {
+ // TBD: need stat counters
+ return -EOVERFLOW;
+ }
+ key_t key_from_bucket_index(p_rec->s.md5_high, p_rec->s.md5_low, size_4k_units,
+ p_rec->s.num_parts, sc_idx);
+ dedup_table_t::value_t src_val;
+ int ret = p_table->get_val(&key_from_bucket_index, &src_val);
+ if (ret != 0) {
+ // record has no valid entry in table because it is a singleton
+ p_stats->skipped_singleton++;
+ p_stats->skipped_singleton_bytes += ondisk_byte_size;
+ ldpp_dout(dpp, 20) << __func__ << "::skipped singleton::" << p_rec->bucket_name
+ << "/" << p_rec->obj_name << std::dec << dendl;
+ return 0;
+ }
+
+ // Every object after this point was counted as a dedup potential
+ // If we conclude that it can't be dedup it should be accounted for
+ rgw_bucket b{p_rec->tenant_name, p_rec->bucket_name, p_rec->bucket_id};
+ unique_ptr<rgw::sal::Bucket> bucket;
+ ret = driver->load_bucket(dpp, b, &bucket, null_yield);
+ if (unlikely(ret != 0)) {
+ // could happen when the bucket is removed between passes
+ p_stats->ingress_failed_load_bucket++;
+ ldpp_dout(dpp, 15) << __func__ << "::Failed driver->load_bucket(): "
+ << cpp_strerror(-ret) << dendl;
+ return 0;
+ }
+
+ unique_ptr<rgw::sal::Object> p_obj = bucket->get_object(p_rec->obj_name);
+ if (unlikely(!p_obj)) {
+ // could happen when the object is removed between passes
+ p_stats->ingress_failed_get_object++;
+ ldpp_dout(dpp, 15) << __func__ << "::Failed bucket->get_object("
+ << p_rec->obj_name << ")" << dendl;
+ return 0;
+ }
+
+ ret = p_obj->get_obj_attrs(null_yield, dpp);
+ if (unlikely(ret < 0)) {
+ p_stats->ingress_failed_get_obj_attrs++;
+ ldpp_dout(dpp, 10) << __func__ << "::ERR: failed to stat object(" << p_rec->obj_name
+ << "), returned error: " << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+
+ const rgw::sal::Attrs& attrs = p_obj->get_attrs();
+ if (attrs.find(RGW_ATTR_CRYPT_MODE) != attrs.end()) {
+ p_stats->ingress_skip_encrypted++;
+ p_stats->ingress_skip_encrypted_bytes += ondisk_byte_size;
+ ldpp_dout(dpp, 20) <<__func__ << "::Skipping encrypted object "
+ << p_rec->obj_name << dendl;
+ return 0;
+ }
+
+ // TBD: We should be able to support RGW_ATTR_COMPRESSION when all copies are compressed
+ if (attrs.find(RGW_ATTR_COMPRESSION) != attrs.end()) {
+ p_stats->ingress_skip_compressed++;
+ p_stats->ingress_skip_compressed_bytes += ondisk_byte_size;
+ ldpp_dout(dpp, 20) <<__func__ << "::Skipping compressed object "
+ << p_rec->obj_name << dendl;
+ return 0;
+ }
+
+ // extract ETAG and Size and compare with values taken from the bucket-index
+ parsed_etag_t parsed_etag;
+ auto itr = attrs.find(RGW_ATTR_ETAG);
+ if (itr != attrs.end()) {
+ if (unlikely(!parse_etag_string(itr->second.to_str(), &parsed_etag))) {
+ p_stats->ingress_corrupted_etag++;
+ ldpp_dout(dpp, 10) << __func__ << "::ERROR: corrupted etag::" << p_rec->obj_name << dendl;
+ return -EINVAL;
+ }
+ }
+ else {
+ p_stats->ingress_corrupted_etag++;
+ ldpp_dout(dpp, 10) << __func__ << "::ERROR: no etag" << p_rec->obj_name << dendl;
+ return -EINVAL;
+ }
+
+ std::string storage_class;
+ itr = attrs.find(RGW_ATTR_STORAGE_CLASS);
+ if (itr != attrs.end()) {
+ storage_class = itr->second.to_str();
+ }
+ else {
+ storage_class = RGW_STORAGE_CLASS_STANDARD;
+ }
+ // no need to check for remap success as we compare keys bellow
+ sc_idx = remapper->remap(storage_class, dpp, &p_stats->failed_map_overflow);
+ key_t key_from_obj(parsed_etag.md5_high, parsed_etag.md5_low,
+ byte_size_to_disk_blocks(p_obj->get_size()),
+ parsed_etag.num_parts, sc_idx);
+ if (unlikely(key_from_obj != key_from_bucket_index ||
+ p_rec->s.obj_bytes_size != p_obj->get_size())) {
+ ldpp_dout(dpp, 15) <<__func__ << "::Skipping changed object "
+ << p_rec->obj_name << dendl;
+ p_stats->ingress_skip_changed_objs++;
+ return 0;
+ }
+
+ // reset flags
+ p_rec->s.flags.clear();
+ ret = add_obj_attrs_to_record(&b, p_rec, attrs, p_table, p_stats);
+ if (unlikely(ret != 0)) {
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: failed add_obj_attrs_to_record() ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+
+ disk_block_seq_t::record_info_t rec_info;
+ ret = p_disk->add_record(d_dedup_cluster_ioctx, p_rec, &rec_info);
+ if (ret == 0) {
+ // set the disk_block_id_t to this unless the existing disk_block_id is marked as shared-manifest
+ ceph_assert(rec_info.rec_id < MAX_REC_IN_BLOCK);
+ ldpp_dout(dpp, 20) << __func__ << "::" << p_rec->bucket_name << "/"
+ << p_rec->obj_name << " was written to block_idx="
+ << rec_info.block_id << "::rec_id=" << (int)rec_info.rec_id
+ << "::shared_manifest=" << p_rec->has_shared_manifest() << dendl;
+ p_table->update_entry(&key_from_bucket_index, rec_info.block_id,
+ rec_info.rec_id, p_rec->has_shared_manifest());
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: Failed p_disk->add_record()"<< dendl;
+ if (ret == -EINVAL) {
+ p_stats->ingress_corrupted_obj_attrs++;
+ }
+ }
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ static int write_sha256_object_attribute(const DoutPrefixProvider* const dpp,
+ rgw::sal::Driver* driver,
+ RGWRados* rados,
+ const disk_record_t *p_rec)
+ {
+ bufferlist etag_bl;
+ bufferlist sha256_bl;
+ librados::ObjectWriteOperation op;
+ etag_to_bufferlist(p_rec->s.md5_high, p_rec->s.md5_low, p_rec->s.num_parts,
+ &etag_bl);
+ init_cmp_pairs(p_rec, etag_bl, sha256_bl /*OUT PARAM*/, &op);
+ op.setxattr(RGW_ATTR_SHA256, sha256_bl);
+
+ std::string oid;
+ librados::IoCtx ioctx;
+ int ret = get_ioctx(dpp, driver, rados, p_rec, &ioctx, &oid);
+ if (unlikely(ret != 0)) {
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: failed get_ioctx()" << dendl;
+ return ret;
+ }
+
+ ret = ioctx.operate(oid, &op);
+ if (unlikely(ret != 0)) {
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: failed ioctx.operate("
+ << oid << "), err is " << cpp_strerror(-ret) << dendl;
+ }
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ // We purged all entries not marked for-dedup (i.e. singleton bit is set) from the table
+ // so all entries left are sources of dedup with multiple copies.
+ // If the record is marked as Shared-Manifest-Object -> skip it
+ // if the record's key doesn’t exist in table -> skip it (it is a singleton and it was purged)
+ // If the record block-index matches the hashtable entry -> skip it (it is the SRC object)
+ // All other entries are Dedicated-Manifest-Objects with a valid SRC object
+
+ // we can withstand most errors moving to the next object
+ // only report an error if we recived a stop scan request!
+ //
+ int Background::try_deduping_record(dedup_table_t *p_table,
+ const disk_record_t *p_tgt_rec,
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ md5_shard_t md5_shard,
+ md5_stats_t *p_stats, /* IN-OUT */
+ remapper_t *remapper)
+ {
+ bool should_print_debug = cct->_conf->subsys.should_gather<ceph_subsys_rgw, 20>();
+ if (unlikely(should_print_debug)) {
+ print_record(dpp, p_tgt_rec, block_id, rec_id, md5_shard);
+ }
+
+ uint32_t size_4k_units = byte_size_to_disk_blocks(p_tgt_rec->s.obj_bytes_size);
+ storage_class_idx_t sc_idx = remapper->remap(p_tgt_rec->stor_class, dpp,
+ &p_stats->failed_map_overflow);
+ ceph_assert(sc_idx != remapper_t::NULL_IDX);
+ key_t key(p_tgt_rec->s.md5_high, p_tgt_rec->s.md5_low, size_4k_units,
+ p_tgt_rec->s.num_parts, sc_idx);
+ dedup_table_t::value_t src_val;
+ int ret = p_table->get_val(&key, &src_val);
+ if (ret != 0) {
+ // record has no valid entry in table because it is a singleton
+ // should never happened since we purged all singletons before
+ ldpp_dout(dpp, 5) << __func__ << "::skipped singleton::" << p_tgt_rec->bucket_name
+ << "/" << p_tgt_rec->obj_name << "::num_parts=" << p_tgt_rec->s.num_parts
+ << "::ETAG=" << std::hex << p_tgt_rec->s.md5_high
+ << p_tgt_rec->s.md5_low << std::dec << dendl;
+ ceph_abort("Unexpcted singleton");
+ return 0;
+ }
+
+ disk_block_id_t src_block_id = src_val.block_idx;
+ record_id_t src_rec_id = src_val.rec_id;
+ if (block_id == src_block_id && rec_id == src_rec_id) {
+ // the table entry point to this record which means it is a dedup source so nothing to do
+ p_stats->skipped_source_record++;
+ ldpp_dout(dpp, 20) << __func__ << "::skipped source-record" << dendl;
+ return 0;
+ }
+
+ // ceph store full blocks so need to round up and multiply by block_size
+ uint64_t ondisk_byte_size = disk_blocks_to_byte_size(size_4k_units);
+ uint64_t dedupable_objects_bytes = calc_deduped_bytes(d_head_object_size,
+ p_tgt_rec->s.num_parts,
+ ondisk_byte_size);
+ if (p_tgt_rec->s.flags.has_shared_manifest()) {
+ // record holds a shared_manifest object so can't be a dedup target
+ p_stats->skipped_shared_manifest++;
+ p_stats->shared_manifest_dedup_bytes += dedupable_objects_bytes;
+ ldpp_dout(dpp, 20) << __func__ << "::skipped shared_manifest" << dendl;
+ return 0;
+ }
+
+ // This records is a dedup target with source record on source_block_id
+ disk_record_t src_rec;
+ ret = load_record(d_dedup_cluster_ioctx, p_tgt_rec, &src_rec, src_block_id,
+ src_rec_id, md5_shard, dpp);
+ if (unlikely(ret != 0)) {
+ p_stats->failed_src_load++;
+ // we can withstand most errors moving to the next object
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: Failed load_record("
+ << src_block_id << ", " << src_rec_id << ")" << dendl;
+ return 0;
+ }
+
+ ldpp_dout(dpp, 20) << __func__ << "::SRC=" << src_rec.bucket_name
+ << "/" << src_rec.obj_name << dendl;
+ // verify that SRC and TGT records don't refer to the same physical object
+ // This could happen in theory if we read the same objects twice
+ if (src_rec.obj_name == p_tgt_rec->obj_name && src_rec.bucket_name == p_tgt_rec->bucket_name) {
+ p_stats->duplicate_records++;
+ ldpp_dout(dpp, 10) << __func__ << "::WARN: Duplicate records for object="
+ << src_rec.obj_name << dendl;
+ return 0;
+ }
+
+ // the hash table size is rounded to the nearest 4KB and will wrap after 16G
+ if (unlikely(src_rec.s.obj_bytes_size != p_tgt_rec->s.obj_bytes_size)) {
+ p_stats->size_mismatch++;
+ ldpp_dout(dpp, 10) << __func__ << "::WARN: differnt byte size for objects::"
+ << src_rec.obj_name << "::" << src_rec.s.obj_bytes_size
+ << "::" << p_tgt_rec->obj_name << "::"
+ << p_tgt_rec->s.obj_bytes_size << dendl;
+ return 0;
+ }
+
+ if (memcmp(src_rec.s.sha256, p_tgt_rec->s.sha256, sizeof(src_rec.s.sha256)) != 0) {
+ p_stats->sha256_mismatch++;
+ ldpp_dout(dpp, 10) << __func__ << "::SHA256 mismatch" << dendl;
+ // TBD: set sha256 attributes on head objects to save calc next time
+ if (src_rec.s.flags.sha256_calculated()) {
+ write_sha256_object_attribute(dpp, driver, rados, &src_rec);
+ p_stats->set_sha256_attrs++;
+ }
+ if (p_tgt_rec->s.flags.sha256_calculated()) {
+ write_sha256_object_attribute(dpp, driver, rados, p_tgt_rec);
+ p_stats->set_sha256_attrs++;
+ }
+ return 0;
+ }
+
+ ret = dedup_object(&src_rec, p_tgt_rec, p_stats, src_val.has_shared_manifest());
+ if (ret == 0) {
+ p_stats->deduped_objects++;
+ p_stats->deduped_objects_bytes += dedupable_objects_bytes;
+ // mark the SRC object as a providor of a shared manifest
+ if (!src_val.has_shared_manifest()) {
+ p_stats->set_shared_manifest_src++;
+ // set the shared manifest flag in the dedup table
+ p_table->set_shared_manifest_src_mode(&key, src_block_id, src_rec_id);
+ }
+ else {
+ ldpp_dout(dpp, 20) << __func__ << "::SRC object already marked as shared_manifest" << dendl;
+ }
+ }
+ else {
+ ldpp_dout(dpp, 10) << __func__ << "::ERR: Failed dedup for "
+ << src_rec.bucket_name << "/" << src_rec.obj_name << dendl;
+ p_stats->failed_dedup++;
+ }
+
+ return 0;
+ }
+
+#endif // #ifdef FULL_DEDUP_SUPPORT
+ //---------------------------------------------------------------------------
+ const char* Background::dedup_step_name(dedup_step_t step)
+ {
+ static const char* names[] = {"STEP_NONE",
+ "STEP_BUCKET_INDEX_INGRESS",
+ "STEP_BUILD_TABLE",
+ "STEP_READ_ATTRIBUTES",
+ "STEP_REMOVE_DUPLICATES"};
+ static const char* undefined_step = "UNDEFINED_STEP";
+ if (step >= STEP_NONE && step <= STEP_REMOVE_DUPLICATES) {
+ return names[step];
+ }
+ else {
+ return undefined_step;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::process_all_slabs(dedup_table_t *p_table,
+ dedup_step_t step,
+ md5_shard_t md5_shard,
+ work_shard_t worker_id,
+ uint32_t *p_slab_count,
+ md5_stats_t *p_stats, /* IN-OUT */
+ disk_block_seq_t *p_disk_block_seq,
+ remapper_t *remapper)
+ {
+ char block_buff[sizeof(disk_block_t)];
+ const int MAX_OBJ_LOAD_FAILURE = 3;
+ const int MAX_BAD_BLOCKS = 2;
+ bool has_more = true;
+ uint32_t seq_number = 0;
+ int failure_count = 0;
+ ldpp_dout(dpp, 20) << __func__ << "::" << dedup_step_name(step) << "::worker_id="
+ << worker_id << ", md5_shard=" << md5_shard << dendl;
+ *p_slab_count = 0;
+ while (has_more) {
+ bufferlist bl;
+ int ret = load_slab(d_dedup_cluster_ioctx, bl, md5_shard, worker_id, seq_number, dpp);
+ if (unlikely(ret < 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR::Failed loading object!! md5_shard=" << md5_shard
+ << ", worker_id=" << worker_id << ", seq_number=" << seq_number
+ << ", failure_count=" << failure_count << dendl;
+ // skip to the next SLAB stopping after 3 bad objects
+ if (failure_count++ < MAX_OBJ_LOAD_FAILURE) {
+ seq_number += DISK_BLOCK_COUNT;
+ continue;
+ }
+ else {
+ return ret;
+ }
+ }
+
+ (*p_slab_count)++;
+ failure_count = 0;
+ unsigned slab_rec_count = 0;
+ auto bl_itr = bl.cbegin();
+ for (uint32_t block_num = 0; block_num < DISK_BLOCK_COUNT; block_num++, seq_number++) {
+ disk_block_id_t disk_block_id(worker_id, seq_number);
+ const char *p = get_next_data_ptr(bl_itr, block_buff, sizeof(block_buff),
+ dpp);
+ disk_block_t *p_disk_block = (disk_block_t*)p;
+ disk_block_header_t *p_header = p_disk_block->get_header();
+ p_header->deserialize();
+ if (unlikely(p_header->verify(disk_block_id, dpp) != 0)) {
+ p_stats->failed_block_load++;
+ // move to next block until reaching a valid block
+ if (failure_count++ < MAX_BAD_BLOCKS) {
+ continue;
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << "::Skipping slab with too many bad blocks::"
+ << (int)md5_shard << ", worker_id=" << (int)worker_id
+ << ", seq_number=" << seq_number << dendl;
+ failure_count = 0;
+ break;
+ }
+ }
+
+ if (p_header->rec_count == 0) {
+ ldpp_dout(dpp, 20) << __func__ << "::Block #" << block_num
+ << " has an empty header, no more blocks" << dendl;
+ has_more = false;
+ break;
+ }
+
+ for (unsigned rec_id = 0; rec_id < p_header->rec_count; rec_id++) {
+ unsigned offset = p_header->rec_offsets[rec_id];
+ // We deserialize the record inside the CTOR
+ disk_record_t rec(p + offset);
+ ret = rec.validate(__func__, dpp, disk_block_id, rec_id);
+ if (unlikely(ret != 0)) {
+ p_stats->failed_rec_load++;
+ return ret;
+ }
+
+ if (step == STEP_BUILD_TABLE) {
+ add_record_to_dedup_table(p_table, &rec, disk_block_id, rec_id, p_stats, remapper);
+ slab_rec_count++;
+ }
+#ifdef FULL_DEDUP_SUPPORT
+ else if (step == STEP_READ_ATTRIBUTES) {
+ read_object_attribute(p_table, &rec, disk_block_id, rec_id, md5_shard,
+ p_stats, p_disk_block_seq, remapper);
+ slab_rec_count++;
+ }
+ else if (step == STEP_REMOVE_DUPLICATES) {
+ try_deduping_record(p_table, &rec, disk_block_id, rec_id, md5_shard,
+ p_stats, remapper);
+ slab_rec_count++;
+ }
+#endif // #ifdef FULL_DEDUP_SUPPORT
+ else {
+ ceph_abort("unexpected step");
+ }
+ }
+
+ check_and_update_md5_heartbeat(md5_shard, p_stats->loaded_objects,
+ p_stats->processed_objects);
+ if (unlikely(d_ctl.should_pause())) {
+ handle_pause_req(__func__);
+ }
+ if (unlikely(d_ctl.should_stop())) {
+ return -ECANCELED;
+ }
+
+ has_more = (p_header->offset == BLOCK_MAGIC);
+ ceph_assert(p_header->offset == BLOCK_MAGIC || p_header->offset == LAST_BLOCK_MAGIC);
+ if (!has_more) {
+ ldpp_dout(dpp, 20) << __func__ << "::No more blocks! block_id=" << disk_block_id
+ << ", rec_count=" << p_header->rec_count << dendl;
+ break;
+ }
+ }
+ ldpp_dout(dpp, 20) <<__func__ << "::slab seq_number=" << seq_number
+ << ", rec_count=" << slab_rec_count << dendl;
+ }
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ static void __attribute__ ((noinline))
+ show_ingress_bucket_idx_obj(const DoutPrefixProvider *dpp,
+ const parsed_etag_t &parsed_etag,
+ const string &bucket_name,
+ const string &obj_name)
+ {
+ ldpp_dout(dpp, 20) << __func__ << "::(1)::" << bucket_name << "/" << obj_name
+ << "::num_parts=" << parsed_etag.num_parts
+ << "::ETAG=" << std::hex << parsed_etag.md5_high
+ << parsed_etag.md5_low << std::dec << dendl;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::ingress_bucket_idx_single_object(disk_block_array_t &disk_arr,
+ const rgw::sal::Bucket *p_bucket,
+ const rgw_bucket_dir_entry &entry,
+ worker_stats_t *p_worker_stats /*IN-OUT*/)
+ {
+ // ceph store full blocks so need to round up and multiply by block_size
+ uint64_t ondisk_byte_size = calc_on_disk_byte_size(entry.meta.size);
+ // count all objects including too small and non default storage_class objs
+ p_worker_stats->ingress_obj++;
+ p_worker_stats->ingress_obj_bytes += ondisk_byte_size;
+
+ parsed_etag_t parsed_etag;
+ if (unlikely(!parse_etag_string(entry.meta.etag, &parsed_etag))) {
+ p_worker_stats->ingress_corrupted_etag++;
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: corrupted etag" << dendl;
+ return -EINVAL;
+ }
+
+ if (unlikely((cct->_conf->subsys.should_gather<ceph_subsys_rgw_dedup, 20>()))) {
+ show_ingress_bucket_idx_obj(dpp, parsed_etag, p_bucket->get_name(), entry.key.name);
+ }
+
+ // We limit dedup to objects from the same storage_class
+ // TBD:
+ // Should we use a skip-list of storage_classes we should skip (like glacier) ?
+ const std::string& storage_class =
+ rgw_placement_rule::get_canonical_storage_class(entry.meta.storage_class);
+ if (storage_class == RGW_STORAGE_CLASS_STANDARD) {
+ p_worker_stats->default_storage_class_objs++;
+ p_worker_stats->default_storage_class_objs_bytes += ondisk_byte_size;
+ }
+ else {
+ ldpp_dout(dpp, 20) << __func__ << "::" << entry.key.name
+ << "::storage_class:" << entry.meta.storage_class << dendl;
+ p_worker_stats->non_default_storage_class_objs++;
+ p_worker_stats->non_default_storage_class_objs_bytes += ondisk_byte_size;
+ }
+
+ if (ondisk_byte_size <= d_min_obj_size_for_dedup) {
+ if (parsed_etag.num_parts == 0) {
+ // dedup only useful for objects bigger than 4MB
+ p_worker_stats->ingress_skip_too_small++;
+ p_worker_stats->ingress_skip_too_small_bytes += ondisk_byte_size;
+
+ if (ondisk_byte_size >= 64*1024) {
+ p_worker_stats->ingress_skip_too_small_64KB++;
+ p_worker_stats->ingress_skip_too_small_64KB_bytes += ondisk_byte_size;
+ }
+ return 0;
+ }
+ else {
+ // multipart objects are always good candidates for dedup
+ // the head object is empty and data is stored only in tail objs
+ p_worker_stats->small_multipart_obj++;
+ }
+ }
+ // multipart/single_part counters are for objects being fully processed
+ if (parsed_etag.num_parts > 0) {
+ p_worker_stats->multipart_objs++;
+ }
+ else {
+ p_worker_stats->single_part_objs++;
+ }
+
+ return add_disk_rec_from_bucket_idx(disk_arr, p_bucket, &parsed_etag,
+ entry.key.name, entry.meta.size,
+ storage_class);
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::check_and_update_heartbeat(unsigned shard_id, uint64_t count_a,
+ uint64_t count_b, const char *prefix)
+ {
+ utime_t now = ceph_clock_now();
+ utime_t time_elapsed = now - d_heart_beat_last_update;
+ if (unlikely(time_elapsed.tv.tv_sec >= d_heart_beat_max_elapsed_sec)) {
+ ldpp_dout(dpp, 20) << __func__ << "::max_elapsed_sec="
+ << d_heart_beat_max_elapsed_sec << dendl;
+ d_heart_beat_last_update = now;
+ d_cluster.update_shard_token_heartbeat(d_dedup_cluster_ioctx, shard_id,
+ count_a, count_b, prefix);
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::check_and_update_worker_heartbeat(work_shard_t worker_id,
+ int64_t ingress_obj_count)
+ {
+ check_and_update_heartbeat(worker_id, ingress_obj_count, 0, WORKER_SHARD_PREFIX);
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::check_and_update_md5_heartbeat(md5_shard_t md5_id,
+ uint64_t load_count,
+ uint64_t dedup_count)
+ {
+ check_and_update_heartbeat(md5_id, load_count, dedup_count, MD5_SHARD_PREFIX);
+ }
+
+ //---------------------------------------------------------------------------
+ static uint32_t move_to_next_bucket_index_shard(const DoutPrefixProvider* dpp,
+ unsigned current_shard,
+ unsigned num_work_shards,
+ const std::string &bucket_name,
+ rgw_obj_index_key *p_marker /* OUT-PARAM */)
+ {
+ uint32_t next_shard = current_shard + num_work_shards;
+ ldpp_dout(dpp, 20) << __func__ << "::" << bucket_name << "::curr_shard="
+ << current_shard << ", next shard=" << next_shard << dendl;
+ *p_marker = rgw_obj_index_key(); // reset marker to an empty index
+ return next_shard;
+ }
+
+ // This function process bucket-index shards of a given @bucket
+ // The bucket-index-shards are stored in a group of @oids
+ // The @oids are using a simple map from the shard-id to the oid holding bucket-indices
+ // We start by processing all bucket-indices owned by this @worker-id
+ // Once we are done with a given bucket-index shard we skip to the next
+ // bucket-index-shard owned by this worker-id
+ // if (bucket_index_shard % work_id) == 0) -> read and process bucket_index_shard
+ // else -> skip bucket_index_shard and don't read it
+ //---------------------------------------------------------------------------
+ int Background::process_bucket_shards(disk_block_array_t &disk_arr,
+ const rgw::sal::Bucket *bucket,
+ std::map<int, string> &oids,
+ librados::IoCtx &ioctx,
+ work_shard_t worker_id,
+ work_shard_t num_work_shards,
+ worker_stats_t *p_worker_stats /*IN-OUT*/)
+ {
+ const uint32_t num_shards = oids.size();
+ uint32_t current_shard = worker_id;
+ rgw_obj_index_key marker; // start with an empty marker
+ const string null_prefix, null_delimiter;
+ const bool list_versions = true;
+ const int max_entries = 1000;
+ uint32_t obj_count = 0;
+
+ while (current_shard < num_shards ) {
+ check_and_update_worker_heartbeat(worker_id, p_worker_stats->ingress_obj);
+ if (unlikely(d_ctl.should_pause())) {
+ handle_pause_req(__func__);
+ }
+ if (unlikely(d_ctl.should_stop())) {
+ return -ECANCELED;
+ }
+
+ const string& oid = oids[current_shard];
+ rgw_cls_list_ret result;
+ librados::ObjectReadOperation op;
+ // get bucket-indices of @current_shard
+ cls_rgw_bucket_list_op(op, marker, null_prefix, null_delimiter, max_entries,
+ list_versions, &result);
+ int ret = rgw_rados_operate(dpp, ioctx, oid, std::move(op), nullptr, null_yield);
+ if (unlikely(ret < 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed rgw_rados_operate() ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ current_shard = move_to_next_bucket_index_shard(dpp, current_shard, num_work_shards,
+ bucket->get_name(), &marker);
+ continue;
+ }
+ obj_count += result.dir.m.size();
+ for (auto& entry : result.dir.m) {
+ const rgw_bucket_dir_entry& dirent = entry.second;
+ if (unlikely((!dirent.exists && !dirent.is_delete_marker()) || !dirent.pending_map.empty())) {
+ // TBD: should we bailout ???
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: calling check_disk_state bucket="
+ << bucket->get_name() << " entry=" << dirent.key << dendl;
+ // make sure we're advancing marker
+ marker = dirent.key;
+ continue;
+ }
+ marker = dirent.key;
+ ret = ingress_bucket_idx_single_object(disk_arr, bucket, dirent, p_worker_stats);
+ }
+ // TBD: advance marker only once here!
+ if (result.is_truncated) {
+ ldpp_dout(dpp, 15) << __func__ << "::[" << current_shard
+ << "]result.is_truncated::count=" << obj_count << dendl;
+ }
+ else {
+ // we reached the end of this shard -> move to the next shard
+ current_shard = move_to_next_bucket_index_shard(dpp, current_shard, num_work_shards,
+ bucket->get_name(), &marker);
+ ldpp_dout(dpp, 15) << __func__ << "::move_to_next_bucket_index_shard::count="
+ << obj_count << "::new_shard=" << current_shard << dendl;
+ }
+ }
+ ldpp_dout(dpp, 15) << __func__ << "::Finished processing Bucket "
+ << bucket->get_name() << ", num_shards=" << num_shards
+ << ", obj_count=" << obj_count << dendl;
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::ingress_bucket_objects_single_shard(disk_block_array_t &disk_arr,
+ const rgw_bucket &bucket_rec,
+ work_shard_t worker_id,
+ work_shard_t num_work_shards,
+ worker_stats_t *p_worker_stats /*IN-OUT*/)
+ {
+ unique_ptr<rgw::sal::Bucket> bucket;
+ int ret = driver->load_bucket(dpp, bucket_rec, &bucket, null_yield);
+ if (unlikely(ret != 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: driver->load_bucket(): "
+ << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+
+ const std::string bucket_id = bucket->get_key().get_key();
+ RGWBucketInfo bucket_info;
+ ret = rados->get_bucket_instance_info(bucket_id, bucket_info,
+ nullptr, nullptr, null_yield, dpp);
+ if (unlikely(ret < 0)) {
+ if (ret == -ENOENT) {
+ // probably a race condition with bucket removal
+ ldpp_dout(dpp, 10) << __func__ << "::ret == -ENOENT" << dendl;
+ return 0;
+ }
+ ldpp_dout(dpp, 5) << __func__ << "::ERROR: get_bucket_instance_info(), ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+ const rgw::bucket_index_layout_generation idx_layout = bucket_info.layout.current_index;
+ librados::IoCtx ioctx;
+ // objects holding the bucket-listings
+ std::map<int, std::string> oids;
+ ret = store->svc()->bi_rados->open_bucket_index(dpp, bucket_info, std::nullopt,
+ idx_layout, &ioctx, &oids, nullptr);
+ if (ret >= 0) {
+ // process all the shards in this bucket owned by the worker_id
+ return process_bucket_shards(disk_arr, bucket.get(), oids, ioctx, worker_id,
+ num_work_shards, p_worker_stats);
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: open_bucket_index() ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ static void display_table_stat_counters(const DoutPrefixProvider* dpp,
+ uint64_t obj_count_in_shard,
+ const md5_stats_t *p_stats)
+ {
+ ldpp_dout(dpp, 10) << "\n>>>>>" << __func__ << "::FINISHED STEP_BUILD_TABLE\n"
+ << "::total_count=" << obj_count_in_shard
+ << "::loaded_objects=" << p_stats->loaded_objects
+ << "::singleton_count=" << p_stats->singleton_count
+ << "::unique_count=" << p_stats->unique_count << "\n"
+ << "::duplicate_count=" << p_stats->duplicate_count
+ << "::duplicated_bytes=" << p_stats->dedup_bytes_estimate
+ << dendl;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::objects_dedup_single_md5_shard(dedup_table_t *p_table,
+ md5_shard_t md5_shard,
+ md5_stats_t *p_stats,
+ work_shard_t num_work_shards)
+ {
+ remapper_t remapper(MAX_STORAGE_CLASS_IDX);
+ // make sure that the standard storage_class is always in the mapper!
+ storage_class_idx_t sc_idx = remapper.remap(RGW_STORAGE_CLASS_STANDARD, dpp,
+ &p_stats->failed_map_overflow);
+ ceph_assert(sc_idx == 0);
+ uint32_t slab_count_arr[num_work_shards];
+ // first load all etags to hashtable to find dedups
+ // the entries come from bucket-index and got minimal info (etag, size)
+ for (work_shard_t worker_id = 0; worker_id < num_work_shards; worker_id++) {
+ process_all_slabs(p_table, STEP_BUILD_TABLE, md5_shard, worker_id,
+ slab_count_arr+worker_id, p_stats, nullptr, &remapper);
+ if (unlikely(d_ctl.should_stop())) {
+ ldpp_dout(dpp, 5) << __func__ << "::STEP_BUILD_TABLE::STOPPED\n" << dendl;
+ return -ECANCELED;
+ }
+ }
+ p_table->count_duplicates(&p_stats->singleton_count, &p_stats->unique_count,
+ &p_stats->duplicate_count, &p_stats->dedup_bytes_estimate);
+ uint64_t obj_count_in_shard = (p_stats->singleton_count + p_stats->unique_count
+ + p_stats->duplicate_count);
+ display_table_stat_counters(dpp, obj_count_in_shard, p_stats);
+
+ ldpp_dout(dpp, 10) << __func__ << "::MD5 Loop::" << d_ctl.dedup_type << dendl;
+ if (d_ctl.dedup_type != dedup_req_type_t::DEDUP_TYPE_FULL) {
+ for (work_shard_t worker_id = 0; worker_id < num_work_shards; worker_id++) {
+ remove_slabs(worker_id, md5_shard, slab_count_arr[worker_id]);
+ }
+ return 0;
+ }
+
+#ifndef FULL_DEDUP_SUPPORT
+ // we don't support full dedup with this release
+ return 0;
+#endif
+
+ p_table->remove_singletons_and_redistribute_keys();
+ // The SLABs holds minimal data set brought from the bucket-index
+ // Objects participating in DEDUP need to read attributes from the Head-Object
+ // TBD - find a better name than num_work_shards for the combined output
+ {
+ disk_block_t arr[DISK_BLOCK_COUNT];
+ worker_stats_t wstat;
+ disk_block_seq_t disk_block_seq(dpp, arr, num_work_shards, md5_shard, &wstat);
+ for (work_shard_t worker_id = 0; worker_id < num_work_shards; worker_id++) {
+ process_all_slabs(p_table, STEP_READ_ATTRIBUTES, md5_shard, worker_id,
+ slab_count_arr+worker_id, p_stats, &disk_block_seq, &remapper);
+ if (unlikely(d_ctl.should_stop())) {
+ ldpp_dout(dpp, 5) << __func__ << "::STEP_READ_ATTRIBUTES::STOPPED\n" << dendl;
+ return -ECANCELED;
+ }
+ // we finished processing output SLAB from @worker_id -> remove them
+ remove_slabs(worker_id, md5_shard, slab_count_arr[worker_id]);
+ }
+ disk_block_seq.flush_disk_records(d_dedup_cluster_ioctx);
+ }
+
+ ldpp_dout(dpp, 10) << __func__ << "::STEP_REMOVE_DUPLICATES::started..." << dendl;
+ uint32_t slab_count = 0;
+ process_all_slabs(p_table, STEP_REMOVE_DUPLICATES, md5_shard, num_work_shards,
+ &slab_count, p_stats, nullptr, &remapper);
+ if (unlikely(d_ctl.should_stop())) {
+ ldpp_dout(dpp, 5) << __func__ << "::STEP_REMOVE_DUPLICATES::STOPPED\n" << dendl;
+ return -ECANCELED;
+ }
+ ldpp_dout(dpp, 10) << __func__ << "::STEP_REMOVE_DUPLICATES::finished..." << dendl;
+ // remove the special SLAB holding aggragted data
+ remove_slabs(num_work_shards, md5_shard, slab_count);
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::read_bucket_stats(const rgw_bucket &bucket_rec,
+ uint64_t *p_num_obj,
+ uint64_t *p_size)
+ {
+ unique_ptr<rgw::sal::Bucket> bucket;
+ int ret = driver->load_bucket(dpp, bucket_rec, &bucket, null_yield);
+ if (unlikely(ret != 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: driver->load_bucket(): "
+ << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+
+ const auto& index = bucket->get_info().get_current_index();
+ if (is_layout_indexless(index)) {
+ ldpp_dout(dpp, 1) << __func__
+ << "::ERR, indexless buckets do not maintain stats; bucket="
+ << bucket->get_name() << dendl;
+ return -EINVAL;
+ }
+
+ std::map<RGWObjCategory, RGWStorageStats> stats;
+ std::string bucket_ver, master_ver;
+ std::string max_marker;
+ ret = bucket->read_stats(dpp, null_yield, index, RGW_NO_SHARD, &bucket_ver,
+ &master_ver, stats, &max_marker);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR getting bucket stats bucket="
+ << bucket->get_name() << " ret=" << ret << dendl;
+ return ret;
+ }
+
+ for (auto itr = stats.begin(); itr != stats.end(); ++itr) {
+ RGWStorageStats& s = itr->second;
+ ldpp_dout(dpp, 20) << __func__ << "::" << bucket->get_name() << "::"
+ << to_string(itr->first) << "::num_obj=" << s.num_objects
+ << "::size=" << s.size << dendl;
+ *p_num_obj += s.num_objects;
+ *p_size += s.size;
+ }
+
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::collect_all_buckets_stats()
+ {
+ int ret = 0;
+ std::string section("bucket.instance");
+ std::string marker;
+ void *handle = nullptr;
+ ret = driver->meta_list_keys_init(dpp, section, marker, &handle);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: Failed meta_list_keys_init: "
+ << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+
+ d_all_buckets_obj_count = 0;
+ d_all_buckets_obj_size = 0;
+
+ bool has_more = true;
+ while (has_more) {
+ std::list<std::string> entries;
+ constexpr int max_keys = 1000;
+ ret = driver->meta_list_keys_next(dpp, handle, max_keys, entries, &has_more);
+ if (ret == 0) {
+ for (auto& entry : entries) {
+ ldpp_dout(dpp, 20) <<__func__ << "::bucket_name=" << entry << dendl;
+ rgw_bucket bucket;
+ ret = rgw_bucket_parse_bucket_key(cct, entry, &bucket, nullptr);
+ if (unlikely(ret < 0)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: Failed rgw_bucket_parse_bucket_key: "
+ << cpp_strerror(-ret) << dendl;
+ goto err;
+ }
+ ldpp_dout(dpp, 20) <<__func__ << "::bucket=" << bucket << dendl;
+ ret = read_bucket_stats(bucket, &d_all_buckets_obj_count,
+ &d_all_buckets_obj_size);
+ if (unlikely(ret != 0)) {
+ goto err;
+ }
+ }
+ driver->meta_list_keys_complete(handle);
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed driver->meta_list_keys_next()" << dendl;
+ goto err;
+ }
+ }
+ ldpp_dout(dpp, 10) <<__func__
+ << "::all_buckets_obj_count=" << d_all_buckets_obj_count
+ << "::all_buckets_obj_size=" << d_all_buckets_obj_size
+ << dendl;
+ return 0;
+
+ err:
+ ldpp_dout(dpp, 1) << __func__ << "::error handler" << dendl;
+ // reset counters to mark that we don't have the info
+ d_all_buckets_obj_count = 0;
+ d_all_buckets_obj_size = 0;
+ if (handle) {
+ driver->meta_list_keys_complete(handle);
+ }
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::objects_ingress_single_work_shard(work_shard_t worker_id,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards,
+ worker_stats_t *p_worker_stats,
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size)
+ {
+ int ret = 0;
+ std::string section("bucket.instance");
+ std::string marker;
+ void *handle = nullptr;
+ ret = driver->meta_list_keys_init(dpp, section, marker, &handle);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: Failed meta_list_keys_init: "
+ << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+ disk_block_array_t disk_arr(dpp, raw_mem, raw_mem_size, worker_id,
+ p_worker_stats, num_md5_shards);
+ bool has_more = true;
+ // iterate over all buckets
+ while (ret == 0 && has_more) {
+ std::list<std::string> entries;
+ constexpr int max_keys = 1000;
+ ret = driver->meta_list_keys_next(dpp, handle, max_keys, entries, &has_more);
+ if (ret == 0) {
+ ldpp_dout(dpp, 20) <<__func__ << "::entries.size()=" << entries.size() << dendl;
+ for (auto& entry : entries) {
+ ldpp_dout(dpp, 20) <<__func__ << "::bucket_name=" << entry << dendl;
+ rgw_bucket bucket;
+ ret = rgw_bucket_parse_bucket_key(cct, entry, &bucket, nullptr);
+ if (unlikely(ret < 0)) {
+ // bad bucket entry, skip to the next one
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: Failed rgw_bucket_parse_bucket_key: "
+ << cpp_strerror(-ret) << dendl;
+ continue;
+ }
+ ldpp_dout(dpp, 20) <<__func__ << "::bucket=" << bucket << dendl;
+ ret = ingress_bucket_objects_single_shard(disk_arr, bucket, worker_id,
+ num_work_shards, p_worker_stats);
+ if (unlikely(ret != 0)) {
+ if (d_ctl.should_stop()) {
+ driver->meta_list_keys_complete(handle);
+ return -ECANCELED;
+ }
+ ldpp_dout(dpp, 1) << __func__ << "::Failed ingress_bucket_objects_single_shard()" << dendl;
+ // skip bad bucket and move on to the next one
+ continue;
+ }
+ }
+ driver->meta_list_keys_complete(handle);
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << "::failed driver->meta_list_keys_next()" << dendl;
+ driver->meta_list_keys_complete(handle);
+ // TBD: what can we do here?
+ break;
+ }
+ }
+ ldpp_dout(dpp, 20) <<__func__ << "::flush_output_buffers() worker_id="
+ << worker_id << dendl;
+ disk_arr.flush_output_buffers(dpp, d_dedup_cluster_ioctx);
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::remove_slabs(unsigned worker_id, unsigned md5_shard, uint32_t slab_count)
+ {
+ unsigned failure_count = 0;
+
+ for (uint32_t slab_id = 0; slab_id < slab_count; slab_id++) {
+ uint32_t seq_number = disk_block_id_t::slab_id_to_seq_num(slab_id);
+ disk_block_id_t block_id(worker_id, seq_number);
+ std::string oid(block_id.get_slab_name(md5_shard));
+ ldpp_dout(dpp, 20) << __func__ << "::calling ioctx->remove(" << oid << ")" << dendl;
+ int ret = d_dedup_cluster_ioctx.remove(oid);
+ if (ret != 0) {
+ ldpp_dout(dpp, 0) << __func__ << "::ERR Failed ioctx->remove(" << oid << ")" << dendl;
+ failure_count++;
+ }
+ }
+
+ return failure_count;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::f_ingress_work_shard(unsigned worker_id,
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards)
+ {
+ ldpp_dout(dpp, 20) << __func__ << "::worker_id=" << worker_id << dendl;
+ utime_t start_time = ceph_clock_now();
+ worker_stats_t worker_stats;
+ int ret = objects_ingress_single_work_shard(worker_id, num_work_shards, num_md5_shards,
+ &worker_stats,raw_mem, raw_mem_size);
+ if (ret == 0) {
+ worker_stats.duration = ceph_clock_now() - start_time;
+ d_cluster.mark_work_shard_token_completed(d_dedup_cluster_ioctx, worker_id,
+ &worker_stats);
+ ldpp_dout(dpp, 10) << "stat counters [worker]:\n" << worker_stats << dendl;
+ ldpp_dout(dpp, 10) << "Shard Process Duration = "
+ << worker_stats.duration << dendl;
+ }
+ //ldpp_dout(dpp, 0) << __func__ << "::sleep for 2 seconds\n" << dendl;
+ //std::this_thread::sleep_for(std::chrono::seconds(2));
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::f_dedup_md5_shard(unsigned md5_shard,
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards)
+ {
+ utime_t start_time = ceph_clock_now();
+ md5_stats_t md5_stats;
+ //DEDUP_DYN_ALLOC
+ dedup_table_t table(dpp, d_head_object_size, raw_mem, raw_mem_size);
+ int ret = objects_dedup_single_md5_shard(&table, md5_shard, &md5_stats, num_work_shards);
+ if (ret == 0) {
+ md5_stats.duration = ceph_clock_now() - start_time;
+ d_cluster.mark_md5_shard_token_completed(d_dedup_cluster_ioctx, md5_shard,
+ &md5_stats);
+ ldpp_dout(dpp, 10) << "stat counters [md5]:\n" << md5_stats << dendl;
+ ldpp_dout(dpp, 10) << "Shard Process Duration = "
+ << md5_stats.duration << dendl;
+ }
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::process_all_shards(bool ingress_work_shards,
+ int (Background::*func)(unsigned, uint8_t*, uint64_t, work_shard_t, md5_shard_t),
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards)
+ {
+ while (true) {
+ d_heart_beat_last_update = ceph_clock_now();
+ uint16_t shard_id;
+ if (ingress_work_shards) {
+ shard_id = d_cluster.get_next_work_shard_token(d_dedup_cluster_ioctx, num_work_shards);
+ }
+ else {
+ shard_id = d_cluster.get_next_md5_shard_token(d_dedup_cluster_ioctx, num_md5_shards);
+ }
+
+ // start with a common error handler
+ if (shard_id != NULL_SHARD) {
+ ldpp_dout(dpp, 10) << __func__ << "::Got shard_id=" << shard_id << dendl;
+ int ret = (this->*func)(shard_id, raw_mem, raw_mem_size, num_work_shards,
+ num_md5_shards);
+ if (unlikely(ret != 0)) {
+ if (d_ctl.should_stop()) {
+ ldpp_dout(dpp, 5) << __func__ << "::stop execution" << dendl;
+ return -ECANCELED;
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::Skip shard #" << shard_id << dendl;
+ }
+ }
+ }
+ else {
+ ldpp_dout(dpp, 10) << __func__ << "::finished processing all shards" <<dendl;
+ break;
+ }
+ } // while loop
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ [[maybe_unused]]static int collect_pool_stats(const DoutPrefixProvider* const dpp,
+ RGWRados* rados,
+ uint64_t *p_num_objects,
+ uint64_t *p_num_objects_bytes)
+ {
+ *p_num_objects = 0;
+ *p_num_objects_bytes = 0;
+ list<string> vec;
+ vec.push_back("default.rgw.buckets.data");
+ map<string,librados::pool_stat_t> stats;
+ auto rados_handle = rados->get_rados_handle();
+ int ret = rados_handle->get_pool_stats(vec, stats);
+ if (ret < 0) {
+ ldpp_dout(dpp, 0) << __func__ << ":ERROR: fetching pool stats: "
+ << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+
+ for (auto i = stats.begin(); i != stats.end(); ++i) {
+ const char *pool_name = i->first.c_str();
+ librados::pool_stat_t& s = i->second;
+ // TBD: add support for EC
+ // We need to find the user byte size without the added protection
+ double replica_level = (double)s.num_object_copies / s.num_objects;
+ *p_num_objects = s.num_objects;
+ *p_num_objects_bytes = s.num_bytes / replica_level;
+ ldpp_dout(dpp, 10) <<__func__ << "::" << pool_name << "::num_objects="
+ << s.num_objects << "::num_copies=" << s.num_object_copies
+ << "::num_bytes=" << s.num_bytes << "/" << *p_num_objects_bytes << dendl;
+ }
+ return 0;
+ }
+
+ //-------------------------------------------------------------------------------
+ // 32B per object-entry in the hashtable
+ // 2MB per shard-buffer
+ //=============||==============||=========||===================================||
+ // Obj Count || shard count || memory || calculation ||
+ // ------------||--------------||---------||---------------------------------- ||
+ // 1M || 4 || 8MB || 8MB/32 = 0.25M * 4 = 1M ||
+ // 4M || 8 || 16MB || 16MB/32 = 0.50M * 8 = 4M ||
+ //-------------------------------------------------------------------------------
+ // 16M || 16 || 32MB || 32MB/32 = 1.00M * 16 = 16M ||
+ //-------------------------------------------------------------------------------
+ // 64M || 32 || 64MB || 64MB/32 = 2.00M * 32 = 64M ||
+ // 256M || 64 || 128MB || 128MB/32 = 4.00M * 64 = 256M ||
+ // 1024M( 1G) || 128 || 256MB || 256MB/32 = 8.00M * 128 = 1024M ||
+ // 4096M( 4G) || 256 || 512MB || 512MB/32 = 16M.00 * 256 = 4096M ||
+ // 16384M(16G) || 512 || 1024MB || 1024MB/32 = 32M.00 * 512 = 16384M ||
+ //-------------||--------------||---------||-----------------------------------||
+ static md5_shard_t calc_num_md5_shards(uint64_t obj_count)
+ {
+ // create headroom by allocating space for a 10% bigger system
+ obj_count = obj_count + (obj_count/10);
+
+ uint64_t M = 1024 * 1024;
+ if (obj_count < 1*M) {
+ // less than 1M objects -> use 4 shards (8MB)
+ return 4;
+ }
+ else if (obj_count < 4*M) {
+ // less than 4M objects -> use 8 shards (16MB)
+ return 8;
+ }
+ else if (obj_count < 16*M) {
+ // less than 16M objects -> use 16 shards (32MB)
+ return 16;
+ }
+ else if (obj_count < 64*M) {
+ // less than 64M objects -> use 32 shards (64MB)
+ return 32;
+ }
+ else if (obj_count < 256*M) {
+ // less than 256M objects -> use 64 shards (128MB)
+ return 64;
+ }
+ else if (obj_count < 1024*M) {
+ // less than 1024M objects -> use 128 shards (256MB)
+ return 128;
+ }
+ else if (obj_count < 4*1024*M) {
+ // less than 4096M objects -> use 256 shards (512MB)
+ return 256;
+ }
+ else {
+ return 512;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::setup(dedup_epoch_t *p_epoch)
+ {
+ int ret = collect_all_buckets_stats();
+ if (unlikely(ret != 0)) {
+ return ret;
+ }
+
+ md5_shard_t num_md5_shards = calc_num_md5_shards(d_all_buckets_obj_count);
+ num_md5_shards = std::min(num_md5_shards, MAX_MD5_SHARD);
+ num_md5_shards = std::max(num_md5_shards, MIN_MD5_SHARD);
+ work_shard_t num_work_shards = num_md5_shards;
+ num_work_shards = std::min(num_work_shards, MAX_WORK_SHARD);
+
+ ldpp_dout(dpp, 5) << __func__ << "::obj_count=" <<d_all_buckets_obj_count
+ << "::num_md5_shards=" << num_md5_shards
+ << "::num_work_shards=" << num_work_shards << dendl;
+ ret = d_cluster.reset(store, d_dedup_cluster_ioctx, p_epoch, num_work_shards,
+ num_md5_shards);
+ if (ret != 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed cluster.init()" << dendl;
+ return ret;
+ }
+
+ if (unlikely(p_epoch->num_work_shards > MAX_WORK_SHARD)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: p_epoch->num_work_shards="
+ << p_epoch->num_work_shards
+ << " is larger than MAX_WORK_SHARD ("
+ << MAX_WORK_SHARD << ")" << dendl;
+ return -EOVERFLOW;
+ }
+ if (unlikely(p_epoch->num_md5_shards > MAX_MD5_SHARD)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: p_epoch->num_md5_shards="
+ << p_epoch->num_md5_shards
+ << " is larger than MAX_MD5_SHARD ("
+ << MAX_MD5_SHARD << ")" << dendl;
+ return -EOVERFLOW;
+ }
+
+ ldpp_dout(dpp, 10) <<__func__ << "::" << *p_epoch << dendl;
+ d_ctl.dedup_type = p_epoch->dedup_type;
+#ifdef FULL_DEDUP_SUPPORT
+ ceph_assert(d_ctl.dedup_type == dedup_req_type_t::DEDUP_TYPE_FULL ||
+ d_ctl.dedup_type == dedup_req_type_t::DEDUP_TYPE_ESTIMATE);
+#else
+ ceph_assert(d_ctl.dedup_type == dedup_req_type_t::DEDUP_TYPE_ESTIMATE);
+#endif
+ ldpp_dout(dpp, 10) << __func__ << "::" << d_ctl.dedup_type << dendl;
+
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::watch_reload(const DoutPrefixProvider* dpp)
+ {
+ if (!d_dedup_cluster_ioctx.is_valid()) {
+ ldpp_dout(dpp, 1) << __func__
+ << "::ERR: invalid pool handler (missing pool)" << dendl;
+ return -ENOENT;
+ }
+ ldpp_dout(dpp, 5) << "dedup_bg::watch_reload(): ioctx="
+ << d_dedup_cluster_ioctx.get_instance_id() << dendl;
+ const std::string & oid = DEDUP_WATCH_OBJ;
+ // create the object to watch (object may already exist)
+ bool exclusive = true;
+ int ret = d_dedup_cluster_ioctx.create(oid, exclusive);
+ if (ret >= 0) {
+ ldpp_dout(dpp, 10) << "dedup_bg::watch_reload():" << oid
+ << " was created!" << dendl;
+ }
+ else if (ret == -EEXIST) {
+ ldpp_dout(dpp, 5) << __func__ << "::"<< oid << " exists" << dendl;
+ }
+ else {
+ ldpp_dout(dpp, 1) << "dedup_bg::watch_reload(): failed ioctx.create("
+ << oid << ") ret=" << ret << "::" << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+
+ ret = d_dedup_cluster_ioctx.watch2(oid, &d_watch_handle, &d_watcher_ctx);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << "dedup_bg::watch_reload(): failed watch2() " << oid
+ << ". error: " << cpp_strerror(-ret) << dendl;
+ d_watch_handle = 0;
+ return ret;
+ }
+ ldpp_dout(dpp, 5) << "dedup_bg::watch_reload(): Started watching "
+ << oid << "::d_watch_handle=" << d_watch_handle << dendl;
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int Background::unwatch_reload(const DoutPrefixProvider* dpp)
+ {
+ if (d_watch_handle == 0) {
+ // nothing to unwatch
+ ldpp_dout(dpp, 1) << "dedup_bg::unwatch_reload(): nothing to watch"
+ << dendl;
+ return 0;
+ }
+
+ if (!d_dedup_cluster_ioctx.is_valid()) {
+ ldpp_dout(dpp, 1) << "dedup_bg::unwatch_reload(): "
+ << "::ERR: invalid pool handler (missing pool)" << dendl;
+ return -ENOENT;
+ }
+
+ ldpp_dout(dpp, 5) << "dedup_bg::unwatch_reload(): ioctx="
+ << d_dedup_cluster_ioctx.get_instance_id()
+ << "::d_watch_handle=" << d_watch_handle << dendl;
+
+ const auto ret = d_dedup_cluster_ioctx.unwatch2(d_watch_handle);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << "dedup_bg::unwatch_reload() failed unwatch2() "
+ << DEDUP_WATCH_OBJ << "::" << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+ ldpp_dout(dpp, 5) << "dedup_bg::unwatch_reload():Stopped watching "
+ << DEDUP_WATCH_OBJ << "::d_watch_handle="
+ << d_watch_handle << dendl;
+
+ d_watch_handle = 0;
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::ack_notify(uint64_t notify_id, uint64_t cookie, int status)
+ {
+ if (!d_dedup_cluster_ioctx.is_valid()) {
+ ldpp_dout(dpp, 1) << __func__
+ << "::ERR: invalid pool handler (missing pool)" << dendl;
+ return;
+ }
+ ldpp_dout(dpp, 5) << __func__ << "::status=" << status << dendl;
+ bufferlist reply_bl;
+ ceph::encode(status, reply_bl);
+ encode(d_ctl, reply_bl);
+ d_dedup_cluster_ioctx.notify_ack(DEDUP_WATCH_OBJ, notify_id, cookie, reply_bl);
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::handle_notify(uint64_t notify_id, uint64_t cookie, bufferlist &bl)
+ {
+ int ret = 0;
+ int32_t urgent_msg = URGENT_MSG_NONE;
+ try {
+ auto bl_iter = bl.cbegin();
+ ceph::decode(urgent_msg, bl_iter);
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: bad urgent_msg" << dendl;
+ ret = -EINVAL;
+ }
+ ldpp_dout(dpp, 5) << __func__ << "::-->" << get_urgent_msg_names(urgent_msg) << dendl;
+
+ // use lock to prevent concurrent pause/resume requests
+ std::unique_lock cond_lock(d_cond_mutex); // [------>open lock block
+ if (unlikely(d_ctl.local_urgent_req())) {
+ // can't operate when the system is paused/shutdown
+ cond_lock.unlock(); // close lock block------>]
+ ldpp_dout(dpp, 5) << __func__
+ << "::system is paused/shutdown -> cancel notification" << dendl;
+ ack_notify(notify_id, cookie, -EBUSY);
+ return;
+ }
+
+ switch(urgent_msg) {
+ case URGENT_MSG_ABORT:
+ if (d_ctl.dedup_exec) {
+ d_ctl.remote_abort_req = true;
+ d_cond.notify_all();
+ d_cond.wait(cond_lock, [this]{return d_ctl.remote_aborted || d_ctl.local_urgent_req();});
+ d_ctl.remote_aborted ? ret = 0 : ret = -EBUSY;
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::inactive dedup->nothing to do" << dendl;
+ }
+ break;
+ case URGENT_MSG_RESTART:
+ if (!d_ctl.dedup_exec) {
+ d_ctl.remote_restart_req = true;
+ d_cond.notify_all();
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::\ncan't restart active dedup\n"<< dendl;
+ ret = -EEXIST;
+ }
+ break;
+ case URGENT_MSG_PASUE:
+ if (d_ctl.dedup_exec && !d_ctl.remote_paused) {
+ d_ctl.remote_pause_req = true;
+ d_cond.notify_all();
+ d_cond.wait(cond_lock, [this]{return d_ctl.remote_paused || d_ctl.local_urgent_req();});
+ d_ctl.remote_paused ? ret = 0 : ret = -EBUSY;
+ }
+ else {
+ if (d_ctl.remote_paused) {
+ ldpp_dout(dpp, 5) << __func__ << "::dedup is already paused" << dendl;
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::inactive dedup->nothing to do" << dendl;
+ }
+ }
+ break;
+ case URGENT_MSG_RESUME:
+ if (d_ctl.remote_pause_req || d_ctl.remote_paused) {
+ d_ctl.remote_pause_req = false;
+ d_ctl.remote_paused = false;
+ d_cond.notify_all();
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::dedup is not paused->nothing to do" << dendl;
+ }
+ break;
+ default:
+ ldpp_dout(dpp, 1) << __func__ << "::unexpected urgent_msg: "
+ << get_urgent_msg_names(urgent_msg) << dendl;
+ ret = -EINVAL;
+ }
+
+ cond_lock.unlock(); // close lock block------>]
+ ack_notify(notify_id, cookie, ret);
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::start()
+ {
+ const DoutPrefixProvider* const dpp = &dp;
+ ldpp_dout(dpp, 10) << __FILE__ << "::" <<__func__ << dendl;
+ {
+ std::unique_lock pause_lock(d_pause_mutex);
+ if (d_ctl.started) {
+ // start the thread only once
+ ldpp_dout(dpp, 1) << "dedup_bg already started" << dendl;
+ return;
+ }
+ d_ctl.started = true;
+ }
+ d_runner = std::thread(&Background::run, this);
+ const auto rc = ceph_pthread_setname("dedup_bg");
+ ldpp_dout(dpp, 5) << "dedup_bg start() = " << rc << dendl;
+ }
+
+ //------------------------- --------------------------------------------------
+ void Background::shutdown()
+ {
+ ldpp_dout(dpp, 5) <<__func__ << "::dedup_bg shutdown()" << dendl;
+ std::unique_lock cond_lock(d_cond_mutex);
+ bool nested_call = false;
+ if (d_ctl.shutdown_req) {
+ // should never happen!
+ ldpp_dout(dpp, 1) <<__func__ << "dedup_bg nested call" << dendl;
+ nested_call = true;
+ }
+ d_ctl.shutdown_req = true;
+ d_cond.notify_all();
+ ldpp_dout(dpp, 1) <<__func__ << "dedup_bg shutdown waiting..." << dendl;
+ d_cond.wait(cond_lock, [this]{return d_ctl.shutdown_done;});
+ if (nested_call) {
+ ldpp_dout(dpp, 1) <<__func__ << "::nested call:: repeat notify" << dendl;
+ d_cond.notify_all();
+ }
+
+ if (d_runner.joinable()) {
+ ldpp_dout(dpp, 5) <<__func__ << "::dedup_bg wait join()" << dendl;
+ d_runner.join();
+ ldpp_dout(dpp, 5) <<__func__ << "::dedup_bg finished join()" << dendl;
+ }
+ else {
+ ldpp_dout(dpp, 5) <<__func__ << "::dedup_bg not joinable()" << dendl;
+ }
+
+ d_ctl.reset();
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::pause()
+ {
+ ldpp_dout(dpp, 5) << "dedup_bg->pause() request: ioctx="
+ << d_dedup_cluster_ioctx.get_instance_id() << dendl;
+ std::unique_lock cond_lock(d_cond_mutex);
+
+ if (d_ctl.local_paused || d_ctl.shutdown_done) {
+ cond_lock.unlock();
+ ldpp_dout(dpp, 1) << __FILE__ << "::" <<__func__
+ << "::dedup_bg is already paused/stopped" << dendl;
+ return;
+ }
+
+ bool nested_call = false;
+ if (d_ctl.local_pause_req) {
+ // should never happen!
+ ldpp_dout(dpp, 1) <<__func__ << "::nested call" << dendl;
+ nested_call = true;
+ }
+ d_ctl.local_pause_req = true;
+ d_cond.notify_all();
+ d_cond.wait(cond_lock, [this]{return d_ctl.local_paused||d_ctl.shutdown_done;});
+ if (nested_call) {
+ ldpp_dout(dpp, 1) << "dedup_bg::nested call:: repeat notify" << dendl;
+ d_cond.notify_all();
+ }
+
+ // destory open watch request and pool handle before pause() is completed
+ unwatch_reload(dpp);
+ d_dedup_cluster_ioctx.close();
+ ldpp_dout(dpp, 5) << "dedup_bg paused" << dendl;
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::resume(rgw::sal::Driver* _driver)
+ {
+ ldpp_dout(dpp, 5) << "dedup_bg->resume()" << dendl;
+ // use lock to prevent concurrent pause/resume requests
+ std::unique_lock cond_lock(d_cond_mutex);
+
+ if (!d_ctl.local_paused) {
+ cond_lock.unlock();
+ ldpp_dout(dpp, 5) << "dedup_bg::resume thread is not paused!" << dendl;
+ if (_driver != driver) {
+ ldpp_dout(dpp, 1) << "dedup_bg attempt to change driver on an active system was refused" << dendl;
+ }
+ return;
+ }
+
+ driver = _driver;
+ int ret = init_rados_access_handles();
+ if (ret != 0) {
+ derr << "dedup_bg::resume() failed init_rados_access_handles() ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ throw std::runtime_error("Failed init_dedup_pool_ioctx()");
+ }
+ ldpp_dout(dpp, 5) << __func__ << "::dedup background: ioctx="
+ << d_dedup_cluster_ioctx.get_instance_id() << dendl;
+ // create new watch request using the new pool handle
+ watch_reload(dpp);
+ d_ctl.local_pause_req = false;
+ d_ctl.local_paused = false;
+
+ // wake up threads blocked after seeing pause state
+ d_cond.notify_all();
+ ldpp_dout(dpp, 5) << "dedup_bg was resumed" << dendl;
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::handle_pause_req(const char *caller)
+ {
+ ldpp_dout(dpp, 5) << __func__ << "::caller=" << caller << dendl;
+ ldpp_dout(dpp, 5) << __func__ << "::" << d_ctl << dendl;
+ while (d_ctl.local_pause_req || d_ctl.local_paused || d_ctl.remote_pause_req || d_ctl.remote_paused) {
+ std::unique_lock cond_lock(d_cond_mutex);
+ if (d_ctl.should_stop()) {
+ ldpp_dout(dpp, 5) << __func__ << "::should_stop!" << dendl;
+ return;
+ }
+
+ if (d_ctl.local_pause_req) {
+ d_ctl.local_pause_req = false;
+ d_ctl.local_paused = true;
+ }
+
+ if (d_ctl.remote_pause_req) {
+ d_ctl.remote_pause_req = false;
+ d_ctl.remote_paused = true;
+ }
+
+ d_cond.notify_all();
+
+ if (d_ctl.local_paused) {
+ ldpp_dout(dpp, 10) << __func__ << "::wait on d_ctl.local_paused" << dendl;
+ d_cond.wait(cond_lock, [this]{return !d_ctl.local_paused || d_ctl.should_stop() ;});
+ }
+
+ if (d_ctl.remote_paused) {
+ ldpp_dout(dpp, 10) << __func__ << "::wait on d_ctl.remote_paused" << dendl;
+ d_cond.wait(cond_lock, [this]{return !d_ctl.remote_paused || d_ctl.should_stop() || d_ctl.local_pause_req;});
+ }
+ } // while loop
+
+ ldpp_dout(dpp, 5) << "Dedup background thread resumed!" << dendl;
+ }
+
+ //---------------------------------------------------------------------------
+ static bool all_shards_completed(cluster *p_cluster,
+ librados::IoCtx &ioctx,
+ work_shard_t num_work_shards,
+ uint64_t *p_total_ingressed)
+ {
+ return p_cluster->all_work_shard_tokens_completed(ioctx, num_work_shards,
+ p_total_ingressed);
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::work_shards_barrier(work_shard_t num_work_shards)
+ {
+ // Wait for other worker to finish ingress step
+ unsigned ttl = 1;
+ uint64_t total_ingressed = 0;
+ while (!all_shards_completed(&d_cluster, d_dedup_cluster_ioctx, num_work_shards, &total_ingressed)) {
+ ldpp_dout(dpp, 10) << __func__ << "::Wait for object ingress completion, ttl="
+ << ttl << " seconds" << dendl;
+ std::unique_lock cond_lock(d_cond_mutex);
+ d_cond.wait_for(cond_lock, std::chrono::seconds(ttl),
+ [this]{return d_ctl.should_stop() || d_ctl.should_pause();});
+ if (unlikely(d_ctl.should_pause())) {
+ handle_pause_req(__func__);
+ }
+ if (unlikely(d_ctl.should_stop())) {
+ return;
+ }
+ }
+
+ ldpp_dout(dpp, 10) << "\n\n==Object Ingress step was completed on all shards! ("
+ << total_ingressed << ")==\n" << dendl;
+ if (unlikely(d_ctl.should_pause())) {
+ handle_pause_req(__func__);
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ void Background::run()
+ {
+ // 256x8KB=2MB
+ const uint64_t PER_SHARD_BUFFER_SIZE = DISK_BLOCK_COUNT *sizeof(disk_block_t);
+ ldpp_dout(dpp, 20) <<__func__ << "::dedup::main loop" << dendl;
+
+ while (!d_ctl.shutdown_req) {
+ if (unlikely(d_ctl.should_pause())) {
+ handle_pause_req(__func__);
+ if (unlikely(d_ctl.should_stop())) {
+ ldpp_dout(dpp, 5) <<__func__ << "::stop req after a pause" << dendl;
+ d_ctl.dedup_exec = false;
+ }
+ }
+
+ if (d_ctl.dedup_exec) {
+ dedup_epoch_t epoch;
+ if (setup(&epoch) != 0) {
+ ldpp_dout(dpp, 1) << "failed setup()" << dendl;
+ return;
+ }
+ work_shard_t num_work_shards = epoch.num_work_shards;
+ md5_shard_t num_md5_shards = epoch.num_md5_shards;
+ const uint64_t RAW_MEM_SIZE = PER_SHARD_BUFFER_SIZE * num_md5_shards;
+ ldpp_dout(dpp, 5) <<__func__ << "::RAW_MEM_SIZE=" << RAW_MEM_SIZE
+ << "::num_work_shards=" << num_work_shards
+ << "::num_md5_shards=" << num_md5_shards << dendl;
+ // DEDUP_DYN_ALLOC
+ auto raw_mem = std::make_unique<uint8_t[]>(RAW_MEM_SIZE);
+ if (raw_mem == nullptr) {
+ ldpp_dout(dpp, 1) << "failed slab memory allocation - size=" << RAW_MEM_SIZE << dendl;
+ return;
+ }
+
+ process_all_shards(true, &Background::f_ingress_work_shard, raw_mem.get(),
+ RAW_MEM_SIZE, num_work_shards, num_md5_shards);
+ if (!d_ctl.should_stop()) {
+ // Wait for all other workers to finish ingress step
+ work_shards_barrier(num_work_shards);
+ if (!d_ctl.should_stop()) {
+ process_all_shards(false, &Background::f_dedup_md5_shard, raw_mem.get(), RAW_MEM_SIZE,
+ num_work_shards, num_md5_shards);
+ ldpp_dout(dpp, 10) << "\n==DEDUP was completed on all shards! ==\n" << dendl;
+ }
+ else {
+ ldpp_dout(dpp, 5) <<__func__ << "::stop req from barrier" << dendl;
+ }
+ }
+ else {
+ ldpp_dout(dpp, 5) <<__func__ << "::stop req from ingress_work_shard" << dendl;
+ }
+ } // dedup_exec
+
+ std::unique_lock cond_lock(d_cond_mutex);
+ d_ctl.dedup_exec = false;
+ if (d_ctl.remote_abort_req) {
+ d_ctl.remote_aborted = true;
+
+ d_ctl.remote_abort_req = false;
+ d_ctl.remote_paused = false;
+ d_cond.notify_all();
+ ldpp_dout(dpp, 5) << __func__ << "::Dedup was aborted on a remote req" << dendl;
+ }
+ d_cond.wait(cond_lock, [this]{return d_ctl.remote_restart_req || d_ctl.should_stop() || d_ctl.should_pause();});
+ if (!d_ctl.should_stop() && !d_ctl.should_pause()) {
+ // TBD: should we release lock here ???
+ if (d_cluster.can_start_new_scan(store)) {
+ d_ctl.dedup_exec = true;
+ d_ctl.remote_aborted = false;
+ d_ctl.remote_paused = false;
+ d_ctl.remote_restart_req = false;
+ d_cond.notify_all();
+ }
+ }else if (d_ctl.should_stop()) {
+ ldpp_dout(dpp, 5) << "main loop::should_stop::" << d_ctl << dendl;
+ }
+ else {
+ ldpp_dout(dpp, 5) << "main loop::should_pause::" << d_ctl << dendl;
+ }
+ }
+ d_ctl.shutdown_done = true;
+ d_cond.notify_all();
+ // shutdown
+ ldpp_dout(dpp, 5) << __func__ << "::Dedup background thread stopped" << dendl;
+ }
+
+}; //namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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.
+ *
+ */
+
+#pragma once
+#include "common/dout.h"
+#include "rgw_common.h"
+#include "rgw_dedup_utils.h"
+#include "rgw_dedup_table.h"
+#include "rgw_dedup_cluster.h"
+#include "rgw_realm_reloader.h"
+#include <string>
+#include <unordered_map>
+#include <variant>
+#include <iostream>
+#include <ostream>
+
+namespace rgw::dedup {
+ struct dedup_epoch_t;
+ struct control_t {
+ control_t() {
+ reset();
+ }
+ void reset();
+ inline bool local_urgent_req() const {
+ return (shutdown_req || local_pause_req);
+ }
+ inline bool should_stop() const {
+ return (shutdown_req || remote_abort_req);
+ }
+ inline bool should_pause() const {
+ return (local_pause_req || remote_pause_req);
+ }
+
+ // allow to start/pasue/resume/stop execution
+ dedup_req_type_t dedup_type = dedup_req_type_t::DEDUP_TYPE_NONE;
+ bool started = false;
+ bool dedup_exec = false;
+ bool shutdown_req = false;
+ bool shutdown_done = false;
+ bool local_pause_req = false;
+ bool local_paused = false;
+ bool remote_abort_req = false;
+ bool remote_aborted = false;
+ bool remote_pause_req = false;
+ bool remote_paused = false;
+ bool remote_restart_req = false;
+ };
+ std::ostream& operator<<(std::ostream &out, const control_t &ctl);
+ void encode(const control_t& ctl, ceph::bufferlist& bl);
+ void decode(control_t& ctl, ceph::bufferlist::const_iterator& bl);
+ class remapper_t;
+ class disk_block_seq_t;
+ struct disk_record_t;
+ struct key_t;
+ //Interval between each execution of the script is set to 5 seconds
+ static inline constexpr int INIT_EXECUTE_INTERVAL = 5;
+ class Background : public RGWRealmReloader::Pauser {
+ class DedupWatcher : public librados::WatchCtx2 {
+ Background* const parent;
+ public:
+ DedupWatcher(Background* _parent) : parent(_parent) {}
+ ~DedupWatcher() override = default;
+ void handle_notify(uint64_t notify_id, uint64_t cookie,
+ uint64_t notifier_id, bufferlist& bl) override;
+ void handle_error(uint64_t cookie, int err) override;
+ };
+
+ public:
+ Background(rgw::sal::Driver* _driver, CephContext* _cct);
+ int watch_reload(const DoutPrefixProvider* dpp);
+ int unwatch_reload(const DoutPrefixProvider* dpp);
+ void handle_notify(uint64_t notify_id, uint64_t cookie, bufferlist &bl);
+ void start();
+ void shutdown();
+ void pause() override;
+ void resume(rgw::sal::Driver* _driver) override;
+
+ private:
+ enum dedup_step_t {
+ STEP_NONE,
+ STEP_BUCKET_INDEX_INGRESS,
+ STEP_BUILD_TABLE,
+ STEP_READ_ATTRIBUTES,
+ STEP_REMOVE_DUPLICATES
+ };
+
+ void ack_notify(uint64_t notify_id, uint64_t cookie, int status);
+ void run();
+ int setup(struct dedup_epoch_t*);
+ void work_shards_barrier(work_shard_t num_work_shards);
+ void handle_pause_req(const char* caller);
+ const char* dedup_step_name(dedup_step_t step);
+ int read_buckets();
+ void check_and_update_heartbeat(unsigned shard_id, uint64_t count_a, uint64_t count_b,
+ const char *prefix);
+
+ inline void check_and_update_worker_heartbeat(work_shard_t worker_id, int64_t obj_count);
+ inline void check_and_update_md5_heartbeat(md5_shard_t md5_id,
+ uint64_t load_count,
+ uint64_t dedup_count);
+ int ingress_bucket_idx_single_object(disk_block_array_t &disk_arr,
+ const rgw::sal::Bucket *bucket,
+ const rgw_bucket_dir_entry &entry,
+ worker_stats_t *p_worker_stats /*IN-OUT*/);
+ int process_bucket_shards(disk_block_array_t &disk_arr,
+ const rgw::sal::Bucket *bucket,
+ std::map<int,std::string> &oids,
+ librados::IoCtx &ioctx,
+ work_shard_t shard_id,
+ work_shard_t num_work_shards,
+ worker_stats_t *p_worker_stats /*IN-OUT*/);
+ int ingress_bucket_objects_single_shard(disk_block_array_t &disk_arr,
+ const rgw_bucket &bucket_rec,
+ work_shard_t worker_id,
+ work_shard_t num_work_shards,
+ worker_stats_t *p_worker_stats /*IN-OUT*/);
+ int objects_ingress_single_work_shard(work_shard_t worker_id,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards,
+ worker_stats_t *p_worker_stats,
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size);
+ int f_ingress_work_shard(unsigned shard_id,
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards);
+ int f_dedup_md5_shard(unsigned shard_id,
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards);
+ int process_all_shards(bool ingress_work_shards,
+ int (Background::* func)(unsigned, uint8_t*, uint64_t, work_shard_t, md5_shard_t),
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards);
+ int read_bucket_stats(const rgw_bucket &bucket_rec,
+ uint64_t *p_num_obj,
+ uint64_t *p_size);
+ int collect_all_buckets_stats();
+ int objects_dedup_single_md5_shard(dedup_table_t *p_table,
+ md5_shard_t md5_shard,
+ md5_stats_t *p_stats,
+ work_shard_t num_work_shards);
+ int add_disk_rec_from_bucket_idx(disk_block_array_t &disk_arr,
+ const rgw::sal::Bucket *p_bucket,
+ const parsed_etag_t *p_parsed_etag,
+ const std::string &obj_name,
+ uint64_t obj_size,
+ const std::string &storage_class);
+
+ int add_record_to_dedup_table(dedup_table_t *p_table,
+ const struct disk_record_t *p_rec,
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ md5_stats_t *p_stats,
+ remapper_t *remapper);
+
+ int process_all_slabs(dedup_table_t *p_table,
+ dedup_step_t step,
+ md5_shard_t md5_shard,
+ work_shard_t work_shard,
+ uint32_t *p_seq_count,
+ md5_stats_t *p_stats /* IN-OUT */,
+ disk_block_seq_t *p_disk_block_arr,
+ remapper_t *remapper);
+
+#ifdef FULL_DEDUP_SUPPORT
+ int calc_object_sha256(const disk_record_t *p_rec, uint8_t *p_sha256);
+ int add_obj_attrs_to_record(rgw_bucket *p_rb,
+ disk_record_t *p_rec,
+ const rgw::sal::Attrs &attrs,
+ dedup_table_t *p_table,
+ md5_stats_t *p_stats); /* IN-OUT */
+
+ int read_object_attribute(dedup_table_t *p_table,
+ disk_record_t *p_rec,
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ md5_shard_t md5_shard,
+ md5_stats_t *p_stats /* IN-OUT */,
+ disk_block_seq_t *p_disk,
+ remapper_t *remapper);
+ int try_deduping_record(dedup_table_t *p_table,
+ const disk_record_t *p_rec,
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ md5_shard_t md5_shard,
+ md5_stats_t *p_stats, /* IN-OUT */
+ remapper_t *remapper);
+ int inc_ref_count_by_manifest(const std::string &ref_tag,
+ const std::string &oid,
+ RGWObjManifest &manifest);
+ int rollback_ref_by_manifest(const std::string &ref_tag,
+ const std::string &oid,
+ RGWObjManifest &tgt_manifest);
+ int free_tail_objs_by_manifest(const std::string &ref_tag,
+ const std::string &oid,
+ RGWObjManifest &tgt_manifest);
+ int dedup_object(const disk_record_t *p_src_rec,
+ const disk_record_t *p_tgt_rec,
+ md5_stats_t *p_stats,
+ bool is_shared_manifest_src);
+#endif
+ int remove_slabs(unsigned worker_id, unsigned md5_shard, uint32_t slab_count);
+ int init_rados_access_handles();
+
+ // private data members
+ rgw::sal::Driver* driver = nullptr;
+ rgw::sal::RadosStore* store = nullptr;
+ RGWRados* rados = nullptr;
+ librados::Rados* rados_handle = nullptr;
+ const DoutPrefix dp;
+ const DoutPrefixProvider* const dpp;
+ CephContext* const cct;
+ cluster d_cluster;
+ librados::IoCtx d_dedup_cluster_ioctx;
+ utime_t d_heart_beat_last_update;
+ unsigned d_heart_beat_max_elapsed_sec;
+
+ // A pool with 6 billion objects has a 1/(2^64) chance for collison with a 128bit MD5
+ uint64_t d_max_protected_objects = (6ULL * 1024 * 1024 * 1024);
+ uint64_t d_all_buckets_obj_count = 0;
+ uint64_t d_all_buckets_obj_size = 0;
+ // we don't benefit from deduping RGW objects smaller than head-object size
+ uint32_t d_min_obj_size_for_dedup = (4ULL * 1024 * 1024);
+ uint32_t d_head_object_size = (4ULL * 1024 * 1024);
+ control_t d_ctl;
+ uint64_t d_watch_handle = 0;
+ DedupWatcher d_watcher_ctx;
+
+ std::thread d_runner;
+ std::mutex d_cond_mutex;
+ std::mutex d_pause_mutex;
+ std::condition_variable d_cond;
+ };
+
+} //namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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 "rgw_dedup_cluster.h"
+#include "rgw_dedup.h"
+#include "rgw_dedup_epoch.h"
+#include "rgw_common.h"
+#include "rgw_dedup_store.h"
+#include "include/rados/rados_types.hpp"
+#include "include/rados/buffer.h"
+#include "include/rados/librados.hpp"
+#include "svc_zone.h"
+#include "common/config.h"
+#include "common/Cond.h"
+#include "common/debug.h"
+#include "common/errno.h"
+#include "rgw_common.h"
+#include "include/denc.h"
+#include "rgw_sal.h"
+#include "driver/rados/rgw_sal_rados.h"
+#include <cstdlib>
+#include <ctime>
+#include <string>
+
+namespace rgw::dedup {
+ const char* DEDUP_EPOCH_TOKEN = "EPOCH_TOKEN";
+
+ static constexpr unsigned EPOCH_MAX_LOCK_DURATION_SEC = 30;
+ struct shard_progress_t;
+ static int collect_shard_stats(librados::IoCtx &ioctx,
+ const DoutPrefixProvider *dpp,
+ utime_t epoch_time,
+ unsigned shards_count,
+ const char *prefix,
+ bufferlist bl_arr[],
+ struct shard_progress_t *sp_arr);
+
+ const uint64_t SP_ALL_OBJECTS = ULLONG_MAX;
+ const uint64_t SP_NO_OBJECTS = 0ULL;
+ const char* SHARD_PROGRESS_ATTR = "shard_progress";
+
+ //---------------------------------------------------------------------------
+ static int get_epoch(librados::IoCtx &ioctx,
+ const DoutPrefixProvider *dpp,
+ dedup_epoch_t *p_epoch, /* OUT */
+ const char *caller)
+ {
+ std::string oid(DEDUP_EPOCH_TOKEN);
+ bufferlist bl;
+ int ret = ioctx.getxattr(oid, RGW_DEDUP_ATTR_EPOCH, bl);
+ if (ret > 0) {
+ try {
+ auto p = bl.cbegin();
+ decode(*p_epoch, p);
+ }catch (const buffer::error&) {
+ ldpp_dout(dpp, 0) << __func__ << "::failed epoch decode!" << dendl;
+ return -EINVAL;
+ }
+ if (caller) {
+ ldpp_dout(dpp, 10) << __func__ << "::"<< caller<< "::" << *p_epoch << dendl;
+ }
+ return 0;
+ }
+ else {
+ // zero length read means no data
+ if (ret == 0) {
+ ret = -ENODATA;
+ }
+ ldpp_dout(dpp, 10) << __func__ << "::" << (caller ? caller : "")
+ << "::failed ioctx.getxattr() with: "
+ << cpp_strerror(-ret) << ", ret=" << ret << dendl;
+ return ret;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ static int set_epoch(librados::IoCtx &ioctx,
+ const std::string &cluster_id,
+ const DoutPrefixProvider *dpp,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards)
+ {
+ std::string oid(DEDUP_EPOCH_TOKEN);
+ ldpp_dout(dpp, 10) << __func__ << "::oid=" << oid << dendl;
+ bool exclusive = true; // block overwrite of old objects
+ int ret = ioctx.create(oid, exclusive);
+ if (ret >= 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::successfully created Epoch object!" << dendl;
+ // now try and take ownership
+ }
+ else if (ret == -EEXIST) {
+ ldpp_dout(dpp, 10) << __func__ << "::Epoch object exists -> trying to take over" << dendl;
+ // try and take ownership
+ }
+ else{
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: failed to create " << oid
+ <<" with: "<< cpp_strerror(-ret) << ", ret=" << ret <<dendl;
+ return ret;
+ }
+
+ uint32_t serial = 0;
+ dedup_req_type_t dedup_type = dedup_req_type_t::DEDUP_TYPE_ESTIMATE;
+ dedup_epoch_t new_epoch = { serial, dedup_type, ceph_clock_now(),
+ num_work_shards, num_md5_shards };
+ bufferlist new_epoch_bl, empty_bl;
+ encode(new_epoch, new_epoch_bl);
+ librados::ObjectWriteOperation op;
+ op.cmpxattr(RGW_DEDUP_ATTR_EPOCH, CEPH_OSD_CMPXATTR_OP_EQ, empty_bl);
+ op.setxattr(RGW_DEDUP_ATTR_EPOCH, new_epoch_bl);
+
+ ldpp_dout(dpp, 10) << __func__ << "::send EPOCH CLS" << dendl;
+ ret = ioctx.operate(oid, &op);
+ if (ret == 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::Epoch object was written" << dendl;
+ }
+ // TBD: must check for failure caused by an existing EPOCH xattr!
+ // probably best to read attribute from epoch!
+ else if (ret == -ECANCELED) {
+ dedup_epoch_t epoch;
+ ret = get_epoch(ioctx, dpp, &epoch, __func__);
+ if (ret == 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::Accept existing Epoch object" << dendl;
+ }
+ return ret;
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: failed ioctx.operate("
+ << oid << "), err is " << cpp_strerror(-ret) << dendl;
+ }
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ static int swap_epoch(const DoutPrefixProvider *dpp,
+ librados::IoCtx &ioctx,
+ const dedup_epoch_t *p_old_epoch,
+ dedup_req_type_t dedup_type,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards)
+ {
+ dedup_epoch_t new_epoch = { p_old_epoch->serial + 1, dedup_type,
+ ceph_clock_now(), num_work_shards, num_md5_shards};
+ bufferlist old_epoch_bl, new_epoch_bl, err_bl;
+ encode(*p_old_epoch, old_epoch_bl);
+ encode(new_epoch, new_epoch_bl);
+ librados::ObjectWriteOperation op;
+ op.cmpxattr(RGW_DEDUP_ATTR_EPOCH, CEPH_OSD_CMPXATTR_OP_EQ, old_epoch_bl);
+ op.setxattr(RGW_DEDUP_ATTR_EPOCH, new_epoch_bl);
+
+ ldpp_dout(dpp, 10) << __func__ << "::send EPOCH CLS" << dendl;
+ std::string oid(DEDUP_EPOCH_TOKEN);
+ int ret = ioctx.operate(oid, &op);
+ if (ret != 0) {
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: failed ioctx.operate("
+ << oid << "), err is " << cpp_strerror(-ret) << dendl;
+ }
+
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ struct shard_progress_t {
+ shard_progress_t() {
+ // init an empty object
+ this->progress_a = SP_NO_OBJECTS;
+ this->progress_b = SP_NO_OBJECTS;
+ this->completed = false;
+
+ // set all timers to now
+ this->creation_time = utime_t();
+ this->completion_time = utime_t();
+ this->update_time = utime_t();
+
+ // owner and stats_bl are empty until set
+ }
+
+ shard_progress_t(uint64_t _progress_a,
+ uint64_t _progress_b,
+ bool _completed,
+ const std::string &_owner,
+ const bufferlist &_stats_bl) : owner(_owner), stats_bl(_stats_bl) {
+ this->progress_a = _progress_a;
+ this->progress_b = _progress_b;
+ this->completed = _completed;
+ this->update_time = ceph_clock_now();
+
+ if (_progress_a == SP_NO_OBJECTS && _progress_b == SP_NO_OBJECTS) {
+ this->creation_time = ceph_clock_now();
+ }
+ if (_completed) {
+ this->completion_time = ceph_clock_now();
+ }
+ }
+
+ bool is_completed() const {
+ if (this->progress_b == SP_ALL_OBJECTS) {
+ ceph_assert(this->completed);
+ return true;
+ }
+ else {
+ ceph_assert(!this->completed);
+ return false;
+ }
+ }
+ uint64_t progress_a;
+ uint64_t progress_b;
+ bool completed;
+ utime_t update_time;
+ utime_t creation_time;
+ utime_t completion_time;
+ std::string owner;
+ bufferlist stats_bl;
+ };
+
+ //---------------------------------------------------------------------------
+ void encode(const shard_progress_t& sp, ceph::bufferlist& bl)
+ {
+ ENCODE_START(1, 1, bl);
+ encode(sp.progress_a, bl);
+ encode(sp.progress_b, bl);
+ encode(sp.completed, bl);
+ encode(sp.creation_time, bl);
+ encode(sp.completion_time, bl);
+ encode(sp.update_time, bl);
+ encode(sp.owner, bl);
+ encode(sp.stats_bl, bl);
+ ENCODE_FINISH(bl);
+ }
+
+ //---------------------------------------------------------------------------
+ void decode(shard_progress_t & sp, ceph::bufferlist::const_iterator& bl)
+ {
+ DECODE_START(1, bl);
+ decode(sp.progress_a, bl);
+ decode(sp.progress_b, bl);
+ decode(sp.completed, bl);
+ decode(sp.creation_time, bl);
+ decode(sp.completion_time, bl);
+ decode(sp.update_time, bl);
+ decode(sp.owner, bl);
+ decode(sp.stats_bl, bl);
+ DECODE_FINISH(bl);
+ }
+
+ //---------------------------------------------------------------------------
+ int init_dedup_pool_ioctx(RGWRados *rados,
+ const DoutPrefixProvider *dpp,
+ librados::IoCtx &ioctx)
+ {
+ rgw_pool dedup_pool(DEDUP_POOL_NAME);
+ std::string pool_name(DEDUP_POOL_NAME);
+#if 0
+ // using Replica-1 for the intermediate data
+ // since it can be regenerated in case of a failure
+ std::string replica_count(std::to_string(1));
+#else
+ // temporary solution until we find a way to disable the health warn on replica1
+ std::string replica_count(std::to_string(2));
+#endif
+ librados::bufferlist inbl;
+ std::string output;
+ std::string command = R"(
+ {
+ "prefix": "osd pool create",
+ "pool": ")" + pool_name +
+ R"(",
+ "pool_type": "replicated",
+ "size": )" + replica_count +
+ R"(
+ })";
+
+ auto rados_handle = rados->get_rados_handle();
+ int ret = rados_handle->mon_command(command, inbl, nullptr, &output);
+ if (output.length()) {
+ if (output != "pool 'rgw_dedup_pool' already exists") {
+ ldpp_dout(dpp, 10) << __func__ << "::" << output << dendl;
+ }
+ }
+ if (ret != 0 && ret != -EEXIST) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed to create pool "
+ << DEDUP_POOL_NAME << " with: "
+ << cpp_strerror(-ret) << ", ret=" << ret << dendl;
+ return ret;
+ }
+
+ ret = rgw_init_ioctx(dpp, rados_handle, dedup_pool, ioctx);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::failed to initialize pool for listing with: "
+ << cpp_strerror(-ret) << dendl;
+ }
+
+ ret = ioctx.application_enable("dedup", false);
+ if (ret == 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::pool " << DEDUP_POOL_NAME
+ << " was associated with dedup app" << dendl;
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed to enable pool "
+ << DEDUP_POOL_NAME << " with: "
+ << cpp_strerror(-ret) << ", ret=" << ret << dendl;
+ }
+ return ret;
+ }
+
+ //==========================================================================
+
+ //---------------------------------------------------------------------------
+ void cluster::clear()
+ {
+ d_curr_md5_shard = 0;
+ d_curr_worker_shard = 0;
+
+ d_num_completed_workers = 0;
+ d_num_completed_md5 = 0;
+
+ memset(d_completed_workers, TOKEN_STATE_PENDING, sizeof(d_completed_workers));
+ memset(d_completed_md5, TOKEN_STATE_PENDING, sizeof(d_completed_md5));
+
+ d_total_ingressed_obj = 0;
+ d_num_failed_workers = 0;
+ }
+
+
+ static constexpr auto COOKIE_LEN = 15;
+ static constexpr auto CLUSTER_ID_LEN = 15;
+ //---------------------------------------------------------------------------
+ cluster::cluster(const DoutPrefixProvider *_dpp,
+ CephContext *cct,
+ rgw::sal::Driver* driver):
+ dpp(_dpp),
+ d_lock_cookie(gen_rand_alphanumeric(cct, COOKIE_LEN)),
+ d_cluster_id (gen_rand_alphanumeric(cct, CLUSTER_ID_LEN))
+ {
+ clear();
+
+ auto store = dynamic_cast<rgw::sal::RadosStore*>(driver);
+ if (!store) {
+ ldpp_dout(dpp, 0) << "ERR: failed dynamic_cast to RadosStore" << dendl;
+ ceph_abort("non-rados backend");
+ return;
+ }
+
+ librados::IoCtx ioctx;
+ if (init_dedup_pool_ioctx(store->getRados(), dpp, ioctx) != 0) {
+ throw std::runtime_error("Failed init_dedup_pool_ioctx()");
+ }
+
+ // generate an empty epoch with zero counters
+ int ret = set_epoch(ioctx, d_cluster_id, dpp, 0, 0);
+ if (ret != 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::failed set_epoch()! ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ throw std::runtime_error("Failed set_epoch()");
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ int cluster::reset(rgw::sal::RadosStore *store,
+ librados::IoCtx &ioctx,
+ dedup_epoch_t *p_epoch,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards)
+ {
+ ldpp_dout(dpp, 10) << __func__ << "::REQ num_work_shards=" << num_work_shards
+ << "::num_md5_shards=" << num_md5_shards << dendl;
+ clear();
+
+ while (true) {
+ int ret = get_epoch(ioctx, dpp, p_epoch, __func__);
+ if (ret != 0) {
+ return ret;
+ }
+ if (p_epoch->num_work_shards && p_epoch->num_md5_shards) {
+ ldpp_dout(dpp, 10) << __func__ << "::ACC num_work_shards=" << p_epoch->num_work_shards
+ << "::num_md5_shards=" << p_epoch->num_md5_shards << dendl;
+ break;
+ }
+ else if (!num_work_shards && !num_md5_shards) {
+ ldpp_dout(dpp, 10) << __func__ << "::Init flow, no need to wait" << dendl;
+ break;
+ }
+ else {
+ ret = swap_epoch(dpp, ioctx, p_epoch,
+ static_cast<dedup_req_type_t> (p_epoch->dedup_type),
+ num_work_shards, num_md5_shards);
+ }
+ }
+
+ d_epoch_time = p_epoch->time;
+ // retry cleanup 3 times before declaring failure
+ const unsigned RETRY_LIMIT = 3;
+ int ret = 1;
+ for (unsigned i = 0; i < RETRY_LIMIT && ret != 0; i++) {
+ ret = cleanup_prev_run(ioctx);
+ }
+ if (ret != 0) {
+ return ret;
+ }
+
+ create_shard_tokens(ioctx, p_epoch->num_work_shards, WORKER_SHARD_PREFIX);
+ create_shard_tokens(ioctx, p_epoch->num_md5_shards, MD5_SHARD_PREFIX);
+
+ ret = verify_all_shard_tokens(ioctx, p_epoch->num_work_shards,
+ WORKER_SHARD_PREFIX);
+ if (ret != 0) {
+ return ret;
+ }
+ return verify_all_shard_tokens(ioctx, p_epoch->num_md5_shards,
+ MD5_SHARD_PREFIX);
+ }
+
+ //---------------------------------------------------------------------------
+ int cluster::cleanup_prev_run(librados::IoCtx &ioctx)
+ {
+ int error_code = 0;
+ constexpr uint32_t max = 100;
+ std::string marker;
+ bool truncated = false;
+ rgw::AccessListFilter filter{};
+ unsigned deleted_count = 0, skipped_count = 0;
+ unsigned failed_count = 0, no_entry_count = 0;
+ do {
+ std::vector<std::string> oids;
+ int ret = rgw_list_pool(dpp, ioctx, max, filter, marker, &oids, &truncated);
+ if (ret == -ENOENT) {
+ ldpp_dout(dpp, 10) << __func__ << "::rgw_list_pool() ret == -ENOENT"<< dendl;
+ break;
+ }
+ else if (ret < 0) {
+ ldpp_dout(dpp, 1) << "failed rgw_list_pool()! ret=" << ret
+ << "::" << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+
+ for (const std::string& oid : oids) {
+ if (oid == DEDUP_WATCH_OBJ || oid == DEDUP_EPOCH_TOKEN) {
+ ldpp_dout(dpp, 10) << __func__ << "::skipping " << oid << dendl;
+ skipped_count++;
+ continue;
+ }
+ uint64_t size;
+ struct timespec tspec;
+ ret = ioctx.stat2(oid, &size, &tspec);
+ if (ret == -ENOENT) {
+ ldpp_dout(dpp, 20) << __func__ << "::" << oid
+ << " was removed by others" << dendl;
+ no_entry_count++;
+ continue;
+ }
+ else if (ret != 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::failed ioctx.stat( " << oid << " )" << dendl;
+ error_code = ret;
+ failed_count++;
+ continue;
+ }
+ utime_t mtime(tspec);
+ if (d_epoch_time < mtime) {
+ ldpp_dout(dpp, 10) << __func__ << "::skipping new obj! "
+ << "::EPOCH={" << d_epoch_time.tv.tv_sec << ":" << d_epoch_time.tv.tv_nsec << "} "
+ << "::mtime={" << mtime.tv.tv_sec << ":" << mtime.tv.tv_nsec << "}" << dendl;
+ skipped_count++;
+ continue;
+ }
+ ldpp_dout(dpp, 10) << __func__ << "::removing object: " << oid << dendl;
+ ret = ioctx.remove(oid);
+ if (ret == 0) {
+ deleted_count++;
+ }
+ else if (ret == -ENOENT) {
+ ldpp_dout(dpp, 20) << __func__ << "::" << oid
+ << " was removed by others" << dendl;
+ no_entry_count++;
+ continue;
+ }
+ else {
+ error_code = ret;
+ failed_count++;
+ ldpp_dout(dpp, 10) << __func__ << "::failed ioctx.remove( " << oid
+ << " ), ret=" << ret << "::" << cpp_strerror(-ret) << dendl;
+ }
+ }
+ ldpp_dout(dpp, 10) << __func__ << "::oids.size()=" << oids.size()
+ << "::deleted=" << deleted_count
+ << "::failed=" << failed_count
+ << "::no entry=" << no_entry_count
+ << "::skipped=" << skipped_count << dendl;
+ } while (truncated);
+
+ return error_code;
+ }
+
+ //---------------------------------------------------------------------------
+ int cluster::create_shard_tokens(librados::IoCtx &ioctx,
+ unsigned shards_count,
+ const char *prefix)
+ {
+ shard_token_oid sto(prefix);
+ for (unsigned shard = 0; shard < shards_count; shard++) {
+ sto.set_shard(shard);
+ std::string oid(sto.get_buff(), sto.get_buff_size());
+ ldpp_dout(dpp, 15) << __func__ << "::creating object: " << oid << dendl;
+ bool exclusive = true;
+ int ret = ioctx.create(oid, exclusive);
+ if (ret >= 0) {
+ ldpp_dout(dpp, 15) << __func__ << "::oid=" << oid << " was created!" << dendl;
+ }
+ else if (ret == -EEXIST) {
+ ldpp_dout(dpp, 15) << __func__ << "::failed ioctx.create("
+ << oid << ") -EEXIST!" << dendl;
+ }
+ else {
+ // TBD: can it happen legally ?
+ ldpp_dout(dpp, 1) << __func__ << "::failed ioctx.create(" << oid
+ << ") with: " << ret << "::" << cpp_strerror(-ret) << dendl;
+ }
+ }
+
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int cluster::verify_all_shard_tokens(librados::IoCtx &ioctx,
+ unsigned shards_count,
+ const char *prefix)
+ {
+ shard_token_oid sto(prefix);
+ for (unsigned shard = 0; shard < shards_count; shard++) {
+ sto.set_shard(shard);
+ std::string oid(sto.get_buff(), sto.get_buff_size());
+ ldpp_dout(dpp, 10) << __func__ << "::checking object: " << oid << dendl;
+
+ uint64_t size;
+ struct timespec tspec;
+ int ret = ioctx.stat2(oid, &size, &tspec);
+ if (ret != 0) {
+ ldpp_dout(dpp, 5) << __func__ << "::failed ioctx.stat( " << oid << " )"
+ << "::shards_count=" << shards_count << dendl;
+ return ret;
+ }
+ }
+
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ int cluster::update_shard_token_heartbeat(librados::IoCtx &ioctx,
+ unsigned shard,
+ uint64_t count_a,
+ uint64_t count_b,
+ const char *prefix)
+ {
+ shard_token_oid sto(prefix, shard);
+ std::string oid(sto.get_buff(), sto.get_buff_size());
+ bufferlist empty_bl;
+ shard_progress_t sp(count_a, count_b, false, d_cluster_id, empty_bl);
+ sp.creation_time = d_token_creation_time;
+ bufferlist sp_bl;
+ encode(sp, sp_bl);
+ return ioctx.setxattr(oid, SHARD_PROGRESS_ATTR, sp_bl);
+ }
+
+ //---------------------------------------------------------------------------
+ int cluster::mark_shard_token_completed(librados::IoCtx &ioctx,
+ unsigned shard,
+ uint64_t obj_count,
+ const char *prefix,
+ const bufferlist &bl)
+ {
+ shard_token_oid sto(prefix, shard);
+ std::string oid(sto.get_buff(), sto.get_buff_size());
+ ldpp_dout(dpp, 10) << __func__ << "::" << prefix << "::" << oid << dendl;
+
+ shard_progress_t sp(obj_count, SP_ALL_OBJECTS, true, d_cluster_id, bl);
+ sp.creation_time = d_token_creation_time;
+ bufferlist sp_bl;
+ encode(sp, sp_bl);
+ int ret = ioctx.setxattr(oid, SHARD_PROGRESS_ATTR, sp_bl);
+ if (ret == 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::Done ioctx.setxattr(" << oid << ")" << dendl;
+ }
+ else {
+ ldpp_dout(dpp, 0) << __func__ << "::Failed ioctx.setxattr(" << oid << ") ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ }
+
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int32_t cluster::get_next_shard_token(librados::IoCtx &ioctx,
+ uint16_t start_shard,
+ uint16_t max_shard,
+ const char *prefix)
+ {
+ // lock paramters:
+ const utime_t lock_duration; // zero duration means lock doesn't expire
+ const uint8_t lock_flags = 0; // no flags
+ const std::string lock_tag; // no tag
+
+ shard_token_oid sto(prefix);
+ for (auto shard = start_shard; shard < max_shard; shard++) {
+ sto.set_shard(shard);
+ std::string oid(sto.get_buff(), sto.get_buff_size());
+ ldpp_dout(dpp, 10) << __func__ << "::try garbbing " << oid << dendl;
+ librados::ObjectWriteOperation op;
+ op.assert_exists();
+ rados::cls::lock::lock(&op, oid, ClsLockType::EXCLUSIVE, d_lock_cookie,
+ lock_tag, "dedup_shard_token", lock_duration, lock_flags);
+ int ret = rgw_rados_operate(dpp, ioctx, oid, std::move(op), null_yield);
+ if (ret == -EBUSY) {
+ // someone else took this token -> move to the next one
+ ldpp_dout(dpp, 10) << __func__ << "::Failed lock. " << oid <<
+ " is owned by other rgw" << dendl;
+ continue;
+ }
+ else if (ret == -ENOENT) {
+ // token is deleted - processing will stop the next time we try to read from the queue
+ ldpp_dout(dpp, 5) << __func__ << "::" << oid
+ << " token doesn't exist, fail lock!" << dendl;
+ continue;
+ }
+ else if (ret < 0) {
+ // failed to lock for another reason, continue to process other queues
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: failed to lock token: " << oid
+ << ":: ret=" << ret << "::" << cpp_strerror(-ret) << dendl;
+ //has_error = true;
+ continue;
+ }
+ ldpp_dout(dpp, 10) << __func__ << "::successfully locked " << oid << dendl;
+ bufferlist empty_bl;
+ shard_progress_t sp(SP_NO_OBJECTS, SP_NO_OBJECTS, false, d_cluster_id, empty_bl);
+ d_token_creation_time = sp.creation_time;
+
+ bufferlist sp_bl;
+ encode(sp, sp_bl);
+ ret = ioctx.setxattr(oid, SHARD_PROGRESS_ATTR, sp_bl);
+ if (ret == 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::SUCCESS!::" << oid << dendl;
+ return shard;
+ }
+ }
+
+ return NULL_SHARD;
+ }
+
+ //---------------------------------------------------------------------------
+ work_shard_t cluster::get_next_work_shard_token(librados::IoCtx &ioctx,
+ work_shard_t num_work_shards)
+ {
+ int32_t shard = get_next_shard_token(ioctx, d_curr_worker_shard, num_work_shards,
+ WORKER_SHARD_PREFIX);
+ if (shard >= 0 && shard < num_work_shards) {
+ d_curr_worker_shard = shard + 1;
+ return shard;
+ }
+ else {
+ return NULL_WORK_SHARD;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ md5_shard_t cluster::get_next_md5_shard_token(librados::IoCtx &ioctx,
+ md5_shard_t num_md5_shards)
+ {
+ int32_t shard = get_next_shard_token(ioctx, d_curr_md5_shard, num_md5_shards,
+ MD5_SHARD_PREFIX);
+ if (shard >= 0 && shard < num_md5_shards) {
+ d_curr_md5_shard = shard + 1;
+ return shard;
+ }
+ else {
+ return NULL_MD5_SHARD;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ bool cluster::all_shard_tokens_completed(librados::IoCtx &ioctx,
+ unsigned shards_count,
+ const char *prefix,
+ uint16_t *p_num_completed,
+ uint8_t completed_arr[],
+ uint64_t *p_total_ingressed)
+ {
+ unsigned count = 0;
+ shard_token_oid sto(prefix);
+ for (unsigned shard = 0; shard < shards_count; shard++) {
+ if (completed_arr[shard] != TOKEN_STATE_PENDING) {
+ count++;
+ continue;
+ }
+
+ sto.set_shard(shard);
+ std::string oid(sto.get_buff(), sto.get_buff_size());
+ ldpp_dout(dpp, 10) << __func__ << "::checking object: " << oid << dendl;
+ bufferlist bl;
+ int ret = ioctx.getxattr(oid, SHARD_PROGRESS_ATTR, bl);
+ if (unlikely(ret <= 0)) {
+ if (ret != -ENODATA) {
+ ldpp_dout(dpp, 10) << __func__ << "::failed ioctx.getxattr() ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ }
+ continue;
+ }
+
+ shard_progress_t sp;
+ try {
+ auto p = bl.cbegin();
+ decode(sp, p);
+ }
+ catch (const buffer::error&) {
+ ldpp_dout(dpp, 0) << __func__ << "::failed shard_progress_t decode!" << dendl;
+ return false;
+ }
+
+ if (sp.progress_b == SP_ALL_OBJECTS) {
+ ceph_assert(sp.completed);
+ utime_t duration = sp.completion_time - sp.creation_time;
+ // mark token completed;
+ (*p_num_completed)++;
+ completed_arr[shard] = TOKEN_STATE_COMPLETED;
+ d_total_ingressed_obj += sp.progress_a;
+ ldpp_dout(dpp, 20) << __func__ << "::" << oid
+ << "::completed! duration=" << duration << dendl;
+ count++;
+ }
+ else {
+ static const utime_t heartbeat_timeout(EPOCH_MAX_LOCK_DURATION_SEC, 0);
+ utime_t time_elapsed = sp.update_time - sp.creation_time;
+ if (time_elapsed > heartbeat_timeout) {
+ // lock expired -> try and break lock
+ ldpp_dout(dpp, 0) << __func__ << "::" << oid << "::expired lock, skipping" << dendl;
+ completed_arr[shard] = TOKEN_STATE_TIMED_OUT;
+ d_num_failed_workers++;
+ continue;
+ }
+ else {
+ return false;
+ }
+ // TBD: need to store copies and declare token with no progress for N seconds
+ // as failing and then skip it
+ return false;
+ }
+ } // loop
+
+ *p_total_ingressed = d_total_ingressed_obj;
+ if (count < shards_count) {
+ unsigned n = shards_count - count;
+ ldpp_dout(dpp, 10) << __func__ << "::waiting for " << n << " tokens" << dendl;
+ }
+ return (count == shards_count);
+ }
+
+ //---------------------------------------------------------------------------
+ static int collect_shard_stats(librados::IoCtx &ioctx,
+ const DoutPrefixProvider *dpp,
+ utime_t epoch_time,
+ unsigned shards_count,
+ const char *prefix,
+ bufferlist bl_arr[],
+ shard_progress_t *sp_arr)
+ {
+ unsigned count = 0;
+ cluster::shard_token_oid sto(prefix);
+ for (unsigned shard = 0; shard < shards_count; shard++) {
+ sto.set_shard(shard);
+ std::string oid(sto.get_buff(), sto.get_buff_size());
+ ldpp_dout(dpp, 20) << __func__ << "::checking object: " << oid << dendl;
+
+ uint64_t size;
+ struct timespec tspec;
+ if (ioctx.stat2(oid, &size, &tspec) != 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::failed ioctx.stat( " << oid << " )"
+ << "::shards_count=" << shards_count << dendl;
+ continue;
+ }
+ utime_t mtime(tspec);
+ if (epoch_time > mtime) {
+ ldpp_dout(dpp, 10) << __func__ << "::skipping old obj! "
+ << "::EPOCH={" << epoch_time.tv.tv_sec << ":" << epoch_time.tv.tv_nsec << "} "
+ << "::mtime={" << mtime.tv.tv_sec << ":" << mtime.tv.tv_nsec << "}" << dendl;
+ continue;
+ }
+
+ shard_progress_t sp;
+ bufferlist bl;
+ int ret = ioctx.getxattr(oid, SHARD_PROGRESS_ATTR, bl);
+ if (ret > 0) {
+ try {
+ auto p = bl.cbegin();
+ decode(sp, p);
+ sp_arr[shard] = sp;
+ count++;
+ }
+ catch (const buffer::error&) {
+ ldpp_dout(dpp, 10) << __func__ << "::(1)failed shard_progress_t decode!" << dendl;
+ return -EINVAL;
+ }
+ }
+ else if (ret != -ENODATA) {
+ ldpp_dout(dpp, 10) << __func__ << "::" << oid << "::failed getxattr() ret="
+ << ret << "::" << cpp_strerror(-ret) << dendl;
+ continue;
+ }
+ bl_arr[shard] = sp.stats_bl;
+ }
+
+ if (count != shards_count) {
+ ldpp_dout(dpp, 10) << __func__ << "::missing shards stats! we got "
+ << count << " / " << shards_count << dendl;
+ }
+
+ return count;
+ }
+
+ struct member_time_t {
+ utime_t start_time;
+ utime_t end_time;
+ utime_t aggregated_time;
+ };
+
+ //---------------------------------------------------------------------------
+ static void collect_single_shard_stats(const DoutPrefixProvider *dpp,
+ std::map<std::string, member_time_t> &owner_map,
+ const shard_progress_t sp_arr[],
+ unsigned shard,
+ bool *p_show_time,
+ const char *name)
+ {
+ const utime_t null_time;
+ const shard_progress_t &sp = sp_arr[shard];
+ if (sp.creation_time == null_time || sp.completion_time == null_time) {
+ *p_show_time = false;
+ return;
+ }
+
+ const std::string &owner = sp.owner;
+ utime_t duration = sp.completion_time - sp.creation_time;
+ if (owner_map.find(owner) != owner_map.end()) {
+ owner_map[owner].aggregated_time += duration;
+ owner_map[owner].end_time = sp.completion_time;
+ }
+ else {
+ owner_map[owner].start_time = sp.creation_time;
+ owner_map[owner].aggregated_time = duration;
+ owner_map[owner].end_time = sp.completion_time;
+ }
+ ldpp_dout(dpp, 10) << __func__ << "::Got " << name
+ << " stats for shard #" << shard << dendl;
+ }
+
+ //---------------------------------------------------------------------------
+ static void show_incomplete_shards_fmt(bool has_incomplete_shards,
+ unsigned num_shards,
+ const shard_progress_t sp_arr[],
+ Formatter *fmt)
+
+ {
+ if (!has_incomplete_shards) {
+ return;
+ }
+ Formatter::ArraySection array_section{*fmt, "incomplete_shards"};
+ for (unsigned shard = 0; shard < num_shards; shard++) {
+ if (sp_arr[shard].is_completed() ) {
+ continue;
+ }
+ Formatter::ObjectSection object_section{*fmt, "shard_progress"};
+ fmt->dump_unsigned("shard_id", shard);
+ fmt->dump_string("owner", sp_arr[shard].owner);
+ fmt->dump_unsigned("progress_a", sp_arr[shard].progress_a);
+ fmt->dump_unsigned("progress_b", sp_arr[shard].progress_b);
+ fmt->dump_stream("last updated") << sp_arr[shard].update_time;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ static utime_t show_time_func_fmt(const utime_t &start_time,
+ bool show_time,
+ const std::map<std::string, member_time_t> &owner_map,
+ Formatter *fmt)
+ {
+ member_time_t all_members_time;
+ all_members_time.start_time = start_time;
+ all_members_time.end_time = start_time;
+ all_members_time.aggregated_time = utime_t();
+
+ Formatter::ObjectSection section{*fmt, "time"};
+ {
+ Formatter::ArraySection array_section{*fmt, "per-shard time"};
+ for (const auto& [owner, value] : owner_map) {
+ uint32_t sec = value.end_time.tv.tv_sec - value.start_time.tv.tv_sec;
+ fmt->dump_stream("member time")
+ << owner << "::start time = [" << value.start_time.tv.tv_sec % 1000
+ << ":" << value.start_time.tv.tv_nsec / (1000*1000) << "] "
+ << "::aggregated time = " << value.aggregated_time.tv.tv_sec
+ << "(" << sec << ") seconds";
+ all_members_time.aggregated_time += value.aggregated_time;
+ if (all_members_time.end_time < value.end_time) {
+ all_members_time.end_time = value.end_time;
+ }
+ }
+ }
+
+ if (show_time) {
+ uint32_t sec = all_members_time.end_time.tv.tv_sec - all_members_time.start_time.tv.tv_sec;
+
+ Formatter::ObjectSection section{*fmt, "All shards time"};
+ fmt->dump_stream("start time") << all_members_time.start_time;
+ fmt->dump_stream("end time")
+ << all_members_time.end_time << " (" << sec << " seconds total)";
+ fmt->dump_unsigned("aggregated time (sec)", all_members_time.aggregated_time.tv.tv_sec);
+ }
+
+ return all_members_time.end_time;
+ }
+
+ //---------------------------------------------------------------------------
+ static void show_dedup_ratio_estimate_fmt(const worker_stats_t &wrk_stats_sum,
+ const md5_stats_t &md5_stats_sum,
+ Formatter *fmt)
+ {
+ uint64_t s3_bytes_before = wrk_stats_sum.ingress_obj_bytes;
+ uint64_t s3_dedup_bytes = md5_stats_sum.dedup_bytes_estimate;
+ uint64_t s3_bytes_after = s3_bytes_before - s3_dedup_bytes;
+
+ Formatter::ObjectSection section{*fmt, "dedup_ratio_estimate"};
+ fmt->dump_unsigned("s3_bytes_before", s3_bytes_before);
+ fmt->dump_unsigned("s3_bytes_after", s3_bytes_after);
+
+ if (s3_bytes_before > s3_bytes_after && s3_bytes_after) {
+ double dedup_ratio = (double)s3_bytes_before/s3_bytes_after;
+ fmt->dump_float("dedup_ratio", dedup_ratio);
+ }
+ else {
+ fmt->dump_float("dedup_ratio", 0);
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ static void show_dedup_ratio_actual_fmt(const worker_stats_t &wrk_stats_sum,
+ const md5_stats_t &md5_stats_sum,
+ Formatter *fmt)
+ {
+ uint64_t s3_bytes_before = wrk_stats_sum.ingress_obj_bytes;
+ uint64_t s3_dedup_bytes = (md5_stats_sum.deduped_objects_bytes +
+ md5_stats_sum.shared_manifest_dedup_bytes);
+ uint64_t s3_bytes_after = s3_bytes_before - s3_dedup_bytes;
+
+ Formatter::ObjectSection section{*fmt, "dedup_ratio_actual"};
+ fmt->dump_unsigned("s3_bytes_before", s3_bytes_before);
+ fmt->dump_unsigned("s3_bytes_after", s3_bytes_after);
+
+ if (s3_bytes_before > s3_bytes_after && s3_bytes_after) {
+ double dedup_ratio = (double)s3_bytes_before/s3_bytes_after;
+ fmt->dump_float("dedup_ratio", dedup_ratio);
+ }
+ else {
+ fmt->dump_float("dedup_ratio", 0);
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ // command-line called from radosgw-admin.cc
+ int cluster::collect_all_shard_stats(rgw::sal::RadosStore *store,
+ Formatter *fmt,
+ const DoutPrefixProvider *dpp)
+ {
+ librados::IoCtx ioctx;
+ int ret = init_dedup_pool_ioctx(store->getRados(), dpp, ioctx);
+ if (ret != 0) {
+ return ret;
+ }
+
+ dedup_epoch_t epoch;
+ ret = get_epoch(ioctx, dpp, &epoch, nullptr);
+ if (ret != 0) {
+ return ret;
+ }
+
+ Formatter::ObjectSection section{*fmt, "DEDUP STAT COUNTERS"};
+ work_shard_t num_work_shards = epoch.num_work_shards;
+ md5_shard_t num_md5_shards = epoch.num_md5_shards;
+
+ unsigned completed_work_shards_count = 0;
+ unsigned completed_md5_shards_count = 0;
+ utime_t md5_start_time;
+ worker_stats_t wrk_stats_sum;
+ {
+ std::map<std::string, member_time_t> owner_map;
+ bool show_time = true;
+ bufferlist bl_arr[num_work_shards];
+ shard_progress_t sp_arr[num_work_shards];
+ int cnt = collect_shard_stats(ioctx, dpp, epoch.time, num_work_shards,
+ WORKER_SHARD_PREFIX, bl_arr, sp_arr);
+ if (cnt != num_work_shards && 0) {
+ std::cerr << ">>>Partial work shard stats recived " << cnt << " / "
+ << num_work_shards << "\n" << std::endl;
+ }
+ bool has_incomplete_shards = false;
+ for (unsigned shard = 0; shard < num_work_shards; shard++) {
+ if (bl_arr[shard].length() == 0) {
+ has_incomplete_shards = true;
+ continue;
+ }
+ completed_work_shards_count++;
+ worker_stats_t stats;
+ try {
+ auto p = bl_arr[shard].cbegin();
+ decode(stats, p);
+ wrk_stats_sum += stats;
+ }catch (const buffer::error&) {
+ // TBD: can we use std::cerr or should we use formatter ??
+ std::cerr << __func__ << "::(2)failed worker_stats_t decode #" << shard << std::endl;
+ continue;
+ }
+ collect_single_shard_stats(dpp, owner_map, sp_arr, shard, &show_time, "WORKER");
+ }
+ Formatter::ObjectSection worker_stats(*fmt, "worker_stats");
+ wrk_stats_sum.dump(fmt);
+ show_incomplete_shards_fmt(has_incomplete_shards, num_work_shards, sp_arr, fmt);
+ md5_start_time = show_time_func_fmt(epoch.time, show_time, owner_map, fmt);
+ }
+
+ if (completed_work_shards_count == num_work_shards) {
+ std::map<std::string, member_time_t> owner_map;
+ bool show_time = true;
+ md5_stats_t md5_stats_sum;
+ bufferlist bl_arr[num_md5_shards];
+ shard_progress_t sp_arr[num_md5_shards];
+ int cnt = collect_shard_stats(ioctx, dpp, epoch.time, num_md5_shards,
+ MD5_SHARD_PREFIX, bl_arr, sp_arr);
+ if (cnt != num_md5_shards && 0) {
+ std::cerr << ">>>Partial MD5_SHARD stats recived " << cnt << " / "
+ << num_md5_shards << "\n" << std::endl;
+ }
+ bool has_incomplete_shards = false;
+ for (unsigned shard = 0; shard < num_md5_shards; shard++) {
+ if (bl_arr[shard].length() == 0) {
+ has_incomplete_shards = true;
+ continue;
+ }
+ completed_md5_shards_count++;
+ md5_stats_t stats;
+ try {
+ auto p = bl_arr[shard].cbegin();
+ decode(stats, p);
+ md5_stats_sum += stats;
+ }catch (const buffer::error&) {
+ // TBD: can we use std::cerr or should we use formatter ??
+ std::cerr << __func__ << "::failed md5_stats_t decode #" << shard << std::endl;
+ continue;
+ }
+ collect_single_shard_stats(dpp, owner_map, sp_arr, shard, &show_time, "MD5");
+ }
+ {
+ Formatter::ObjectSection outer(*fmt, "md5_stats");
+ md5_stats_sum.dump(fmt);
+ show_incomplete_shards_fmt(has_incomplete_shards, num_md5_shards, sp_arr, fmt);
+ show_time_func_fmt(md5_start_time, show_time, owner_map, fmt);
+ }
+ show_dedup_ratio_estimate_fmt(wrk_stats_sum, md5_stats_sum, fmt);
+ show_dedup_ratio_actual_fmt(wrk_stats_sum, md5_stats_sum, fmt);
+ }
+
+ fmt->dump_bool("completed", (completed_md5_shards_count == num_md5_shards));
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ // command-line called from radosgw-admin.cc
+ int cluster::dedup_control(rgw::sal::RadosStore *store,
+ const DoutPrefixProvider *dpp,
+ urgent_msg_t urgent_msg)
+ {
+ ldpp_dout(dpp, 20) << __func__ << "::dedup_control req = "
+ << get_urgent_msg_names(urgent_msg) << dendl;
+ if (urgent_msg != URGENT_MSG_RESUME &&
+ urgent_msg != URGENT_MSG_PASUE &&
+ urgent_msg != URGENT_MSG_RESTART &&
+ urgent_msg != URGENT_MSG_ABORT) {
+ ldpp_dout(dpp, 1) << __func__ << "::illegal urgent_msg="<< urgent_msg << dendl;
+ return -EINVAL;
+ }
+
+ librados::IoCtx ioctx;
+ int ret = init_dedup_pool_ioctx(store->getRados(), dpp, ioctx);
+ if (ret != 0) {
+ return ret;
+ }
+ // 10 seconds timeout
+ const uint64_t timeout_ms = 10*1000;
+ bufferlist reply_bl, urgent_msg_bl;
+ ceph::encode(urgent_msg, urgent_msg_bl);
+ ret = rgw_rados_notify(dpp, ioctx, DEDUP_WATCH_OBJ, urgent_msg_bl,
+ timeout_ms, &reply_bl, null_yield);
+ if (ret < 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::failed rgw_rados_notify("
+ << DEDUP_WATCH_OBJ << ")::err="<<cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+ std::vector<librados::notify_ack_t> acks;
+ std::vector<librados::notify_timeout_t> timeouts;
+ ioctx.decode_notify_response(reply_bl, &acks, &timeouts);
+ if (timeouts.size() > 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::failed rgw_rados_notify("
+ << DEDUP_WATCH_OBJ << ")::timeout error" << dendl;
+ return -EAGAIN;
+ }
+
+ for (auto& ack : acks) {
+ try {
+ ldpp_dout(dpp, 20) << __func__ << "::ACK: notifier_id=" << ack.notifier_id
+ << "::cookie=" << ack.cookie << dendl;
+ auto iter = ack.payload_bl.cbegin();
+ ceph::decode(ret, iter);
+ struct rgw::dedup::control_t ctl;
+ decode(ctl, iter);
+ ldpp_dout(dpp, 10) << __func__ << "::++ACK::ctl=" << ctl << "::ret=" << ret << dendl;
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__ << "::failed decoding notify acks" << dendl;
+ return -EINVAL;
+ }
+ if (ret != 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::Bad notify ack, ret=" << ret
+ << "::err=" << cpp_strerror(-ret) << dendl;
+ return ret;
+ }
+ }
+ ldpp_dout(dpp, 10) << __func__ << "::" << get_urgent_msg_names(urgent_msg)
+ << " finished successfully!" << dendl;
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ // command-line called from radosgw-admin.cc
+ int cluster::dedup_restart_scan(rgw::sal::RadosStore *store,
+ dedup_req_type_t dedup_type,
+ const DoutPrefixProvider *dpp)
+ {
+ librados::IoCtx ioctx;
+ int ret = init_dedup_pool_ioctx(store->getRados(), dpp, ioctx);
+ if (ret != 0) {
+ return ret;
+ }
+
+ dedup_epoch_t old_epoch;
+ // store the previous epoch for cmp-swap
+ ret = get_epoch(ioctx, dpp, &old_epoch, __func__);
+ if (ret != 0) {
+ return ret;
+ }
+
+ // first abort all dedup work!
+ ret = dedup_control(store, dpp, URGENT_MSG_ABORT);
+ if (ret != 0) {
+ return ret;
+ }
+
+ ldpp_dout(dpp, 10) << __func__ << dedup_type << dendl;
+#ifdef FULL_DEDUP_SUPPORT
+ ceph_assert(dedup_type == dedup_req_type_t::DEDUP_TYPE_ESTIMATE ||
+ dedup_type == dedup_req_type_t::DEDUP_TYPE_FULL);
+#else
+ ceph_assert(dedup_type == dedup_req_type_t::DEDUP_TYPE_ESTIMATE);
+#endif
+ ret = swap_epoch(dpp, ioctx, &old_epoch, dedup_type, 0, 0);
+ if (ret == 0) {
+ ldpp_dout(dpp, 10) << __func__ << "::Epoch object was reset" << dendl;
+ return dedup_control(store, dpp, URGENT_MSG_RESTART);
+ }
+ else {
+ return ret;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ bool cluster::can_start_new_scan(rgw::sal::RadosStore *store)
+ {
+ ldpp_dout(dpp, 10) << __func__ << "::epoch=" << d_epoch_time << dendl;
+ librados::IoCtx ioctx;
+ int ret = init_dedup_pool_ioctx(store->getRados(), dpp, ioctx);
+ if (ret != 0) {
+ return ret;
+ }
+
+ dedup_epoch_t new_epoch;
+ if (get_epoch(ioctx, dpp, &new_epoch, nullptr) != 0) {
+ ldpp_dout(dpp, 1) << __func__ << "::No Epoch Object::"
+ << "::scan can be restarted!\n\n\n" << dendl;
+ // no epoch object exists -> we should start a new scan
+ return true;
+ }
+
+ if (new_epoch.time <= d_epoch_time) {
+ if (new_epoch.time == d_epoch_time) {
+ ldpp_dout(dpp, 10) << __func__ << "::Epoch hasn't change - > Do not restart scan!!" << dendl;
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << " ::Do not restart scan!\n epoch="
+ << d_epoch_time << "\nnew_epoch="<< new_epoch.time <<dendl;
+ }
+ return false;
+ }
+ // allow members to join within a 30 sec limit
+ utime_t limit = {30, 0};
+ utime_t now = ceph_clock_now();
+ ldpp_dout(dpp, 1) << __func__ << "\n::new_epoch=" << new_epoch.time
+ << "\n::now =" << now << dendl;
+ if ((now > new_epoch.time) && ((now - new_epoch.time) < limit)) {
+ ldpp_dout(dpp, 1) << __func__ << "::Epoch is less than 30 seconds old!"
+ << " Restart scan\n\n\n" << dendl;
+ return true;
+ }
+ ldpp_dout(dpp, 1) << "\n::new_epoch - now = " << (new_epoch.time - now)
+ << "\n::limit = " << limit << dendl;
+
+ if (new_epoch.time > now) {
+ ldpp_dout(dpp, 1) << ":new_epoch > now = TRUE " << dendl;
+ }
+ return false;
+ }
+} // namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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.
+ *
+ */
+
+#pragma once
+#include "common/dout.h"
+#include "rgw_dedup_utils.h"
+#include "rgw_dedup_store.h"
+#include <string>
+
+namespace rgw::dedup {
+ static constexpr const char* DEDUP_POOL_NAME = "rgw_dedup_pool";
+ static constexpr const char* MD5_SHARD_PREFIX = "MD5.SHRD.TK.";
+ static constexpr const char* WORKER_SHARD_PREFIX = "WRK.SHRD.TK.";
+
+ struct dedup_epoch_t;
+ int init_dedup_pool_ioctx(RGWRados *rados,
+ const DoutPrefixProvider *dpp,
+ librados::IoCtx &ioctx);
+
+ class cluster{
+ public:
+ //==================================================================================
+ class shard_token_oid {
+ public:
+ //---------------------------------------------------------------------------
+ shard_token_oid(const char *prefix) {
+ this->prefix_len = snprintf(this->buff, BUFF_SIZE, "%s", prefix);
+ this->total_len = this->prefix_len;
+ }
+
+ //---------------------------------------------------------------------------
+ shard_token_oid(const char *prefix, uint16_t shard) {
+ this->prefix_len = snprintf(this->buff, BUFF_SIZE, "%s", prefix);
+ set_shard(shard);
+ }
+
+ //---------------------------------------------------------------------------
+ void set_shard(uint16_t shard) {
+ int n = snprintf(this->buff + this->prefix_len, BUFF_SIZE, "%03x", shard);
+ this->total_len = this->prefix_len + n;
+ }
+
+ inline const char* get_buff() { return this->buff; }
+ inline unsigned get_buff_size() { return this->total_len; }
+ private:
+ static const unsigned BUFF_SIZE = 15;
+ unsigned total_len = 0;
+ unsigned prefix_len = 0;
+ char buff[BUFF_SIZE];
+ };
+
+ //==================================================================================
+ cluster(const DoutPrefixProvider *_dpp,
+ CephContext* cct,
+ rgw::sal::Driver* driver);
+ int reset(rgw::sal::RadosStore *store,
+ librados::IoCtx &ioctx,
+ struct dedup_epoch_t*,
+ work_shard_t num_work_shards,
+ md5_shard_t num_md5_shards);
+
+ utime_t get_epoch_time() { return d_epoch_time; }
+ work_shard_t get_next_work_shard_token(librados::IoCtx &ioctx,
+ work_shard_t num_work_shards);
+ md5_shard_t get_next_md5_shard_token(librados::IoCtx &ioctx,
+ md5_shard_t num_md5_shards);
+ bool can_start_new_scan(rgw::sal::RadosStore *store);
+ static int collect_all_shard_stats(rgw::sal::RadosStore *store,
+ Formatter *p_formatter,
+ const DoutPrefixProvider *dpp);
+ static int dedup_control(rgw::sal::RadosStore *store,
+ const DoutPrefixProvider *dpp,
+ urgent_msg_t urgent_msg);
+ static int dedup_restart_scan(rgw::sal::RadosStore *store,
+ dedup_req_type_t dedup_type,
+ const DoutPrefixProvider *dpp);
+
+ //---------------------------------------------------------------------------
+ int mark_work_shard_token_completed(librados::IoCtx &ioctx,
+ work_shard_t work_shard,
+ const worker_stats_t *p_stats)
+ {
+ ceph::bufferlist bl;
+ encode(*p_stats, bl);
+ d_num_completed_workers++;
+ d_completed_workers[work_shard] = TOKEN_STATE_COMPLETED;
+ d_total_ingressed_obj += p_stats->ingress_obj;
+
+ return mark_shard_token_completed(ioctx, work_shard, p_stats->ingress_obj,
+ WORKER_SHARD_PREFIX, bl);
+ }
+
+ //---------------------------------------------------------------------------
+ int mark_md5_shard_token_completed(librados::IoCtx &ioctx,
+ md5_shard_t md5_shard,
+ const md5_stats_t *p_stats)
+ {
+ ceph::bufferlist bl;
+ encode(*p_stats, bl);
+ d_num_completed_md5++;
+ d_completed_md5[md5_shard] = TOKEN_STATE_COMPLETED;
+ return mark_shard_token_completed(ioctx, md5_shard, p_stats->loaded_objects,
+ MD5_SHARD_PREFIX, bl);
+ }
+
+ int update_shard_token_heartbeat(librados::IoCtx &ioctx,
+ unsigned shard,
+ uint64_t count_a,
+ uint64_t count_b,
+ const char *prefix);
+
+ //---------------------------------------------------------------------------
+ bool all_work_shard_tokens_completed(librados::IoCtx &ioctx,
+ work_shard_t num_work_shards,
+ uint64_t *p_total_ingressed)
+ {
+ return all_shard_tokens_completed(ioctx,
+ num_work_shards,
+ WORKER_SHARD_PREFIX,
+ &d_num_completed_workers,
+ d_completed_workers,
+ p_total_ingressed);
+ }
+
+ private:
+ static constexpr unsigned TOKEN_STATE_PENDING = 0x00;
+ static constexpr unsigned TOKEN_STATE_TIMED_OUT = 0xDD;
+ static constexpr unsigned TOKEN_STATE_COMPLETED = 0xFF;
+
+ void clear();
+ bool all_shard_tokens_completed(librados::IoCtx &ioctx,
+ unsigned shards_count,
+ const char *prefix,
+ uint16_t *p_num_completed,
+ uint8_t completed_arr[],
+ uint64_t *p_total_ingressed);
+ int cleanup_prev_run(librados::IoCtx &ioctx);
+ int32_t get_next_shard_token(librados::IoCtx &ioctx,
+ uint16_t start_shard,
+ uint16_t max_count,
+ const char *prefix);
+ int create_shard_tokens(librados::IoCtx &ioctx,
+ unsigned shards_count,
+ const char *prefix);
+ int verify_all_shard_tokens(librados::IoCtx &ioctx,
+ unsigned shards_count,
+ const char *prefix);
+ int mark_shard_token_completed(librados::IoCtx &ioctx,
+ unsigned shard,
+ uint64_t obj_count,
+ const char *prefix,
+ const bufferlist &bl);
+
+ const DoutPrefixProvider *dpp;
+ std::string d_lock_cookie;
+ std::string d_cluster_id;
+ md5_shard_t d_curr_md5_shard = 0;
+ work_shard_t d_curr_worker_shard = 0;
+ utime_t d_epoch_time;
+ utime_t d_token_creation_time;
+ uint64_t d_total_ingressed_obj = 0;
+ uint8_t d_completed_workers[MAX_WORK_SHARD];
+ uint8_t d_completed_md5[MAX_MD5_SHARD];
+ uint16_t d_num_completed_workers = 0;
+ uint16_t d_num_completed_md5 = 0;
+ uint16_t d_num_failed_workers = 0;
+ };
+
+} //namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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.
+ *
+ */
+
+#pragma once
+#include "common/dout.h"
+#include "rgw_dedup_utils.h"
+
+#include <string>
+
+namespace rgw::dedup {
+ constexpr const char* RGW_DEDUP_ATTR_EPOCH = "rgw.dedup.attr.epoch";
+ //===========================================================================
+
+ struct dedup_epoch_t {
+ uint32_t serial;
+ dedup_req_type_t dedup_type;
+ utime_t time;
+ uint32_t num_work_shards = 0;
+ uint32_t num_md5_shards = 0;
+ };
+
+ //---------------------------------------------------------------------------
+ inline void encode(const dedup_epoch_t& o, ceph::bufferlist& bl)
+ {
+ ENCODE_START(1, 1, bl);
+ encode(o.serial, bl);
+ encode(static_cast<int32_t>(o.dedup_type), bl);
+ encode(o.time, bl);
+ encode(o.num_work_shards, bl);
+ encode(o.num_md5_shards, bl);
+ ENCODE_FINISH(bl);
+ }
+
+ //---------------------------------------------------------------------------
+ inline void decode(dedup_epoch_t& o, ceph::bufferlist::const_iterator& bl)
+ {
+ DECODE_START(1, bl);
+ decode(o.serial, bl);
+ int32_t dedup_type;
+ decode(dedup_type, bl);
+ o.dedup_type = static_cast<dedup_req_type_t> (dedup_type);
+ decode(o.time, bl);
+ decode(o.num_work_shards, bl);
+ decode(o.num_md5_shards, bl);
+ DECODE_FINISH(bl);
+ }
+
+ //---------------------------------------------------------------------------
+ inline std::ostream& operator<<(std::ostream &out, const dedup_epoch_t &ep)
+ {
+ utime_t elapsed = ceph_clock_now() - ep.time;
+ out << "EPOCH::Time={" << ep.time.tv.tv_sec <<":"<< ep.time.tv.tv_nsec << "}::";
+ out << "Elapsed={" << elapsed.tv.tv_sec <<":"<< elapsed.tv.tv_nsec << "}::";
+ out << ep.dedup_type << "::serial=" << ep.serial;
+ out << "::num_work_shards=" << ep.num_work_shards;
+ out << "::num_md5_shards=" << ep.num_md5_shards;
+ return out;
+ }
+
+} //namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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.
+ *
+ */
+
+#pragma once
+#include "common/dout.h"
+#include <unordered_map>
+#include <cstring>
+#include <string>
+
+
+namespace rgw::dedup {
+ class remapper_t
+ {
+ public:
+ static inline constexpr uint8_t NULL_IDX = 0xFF;
+ remapper_t(uint32_t max_entries) : d_max_entries(max_entries) {}
+ uint8_t remap(const std::string &key,
+ const DoutPrefixProvider* dpp,
+ uint64_t *p_overflow_count) { // IN-OUT
+ uint8_t idx;
+
+ auto itr = d_map.find(key);
+ if (itr != d_map.end()) {
+ idx = itr->second;
+ ldpp_dout(dpp, 20) << __func__ << "::Existing key: " << key
+ << " is mapped to idx=" << (int)idx << dendl;
+ }
+ else if (d_num_entries < d_max_entries) {
+ // assign it the next entry
+ idx = d_num_entries++;
+ d_map[key] = idx;
+ ldpp_dout(dpp, 20) << __func__ << "::New key: " << key
+ << " was mapped to idx=" << (int)idx << dendl;
+ }
+ else {
+ (*p_overflow_count) ++;
+ ldpp_dout(dpp, 10) << __func__ << "::ERR: Failed adding key: "
+ << key << dendl;
+ idx = NULL_IDX;
+ }
+
+ return idx;
+ }
+
+ private:
+ uint32_t d_num_entries = 0;
+ const uint32_t d_max_entries;
+ std::unordered_map<std::string, uint8_t> d_map;
+ };
+
+} //namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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/rados/rados_types.hpp"
+#include "include/rados/buffer.h"
+#include "include/rados/librados.hpp"
+#include "svc_zone.h"
+#include "common/config.h"
+#include "common/Cond.h"
+#include "common/debug.h"
+#include "common/errno.h"
+#include "rgw_common.h"
+#include "include/denc.h"
+#include "rgw_sal.h"
+#include "driver/rados/rgw_sal_rados.h"
+#include "rgw_dedup_utils.h"
+#include "rgw_dedup.h"
+#include "rgw_dedup_store.h"
+#include "fmt/ranges.h"
+#include <span>
+
+namespace rgw::dedup {
+
+ rgw_pool pool(DEDUP_POOL_NAME);
+
+ //---------------------------------------------------------------------------
+ disk_record_t::disk_record_t(const rgw::sal::Bucket *p_bucket,
+ const std::string &obj_name,
+ const parsed_etag_t *p_parsed_etag,
+ uint64_t obj_size,
+ const std::string &storage_class)
+ {
+ this->s.rec_version = 0;
+ this->s.flags = 0;
+ this->s.num_parts = p_parsed_etag->num_parts;
+ this->obj_name = obj_name;
+ this->s.obj_name_len = this->obj_name.length();
+ this->bucket_name = p_bucket->get_name();
+ this->s.bucket_name_len = this->bucket_name.length();
+
+ this->s.md5_high = p_parsed_etag->md5_high;
+ this->s.md5_low = p_parsed_etag->md5_low;
+ this->s.obj_bytes_size = obj_size;
+ this->s.object_version = 0;
+
+ this->bucket_id = p_bucket->get_bucket_id();
+ this->s.bucket_id_len = this->bucket_id.length();
+ this->tenant_name = p_bucket->get_tenant();
+ this->s.tenant_name_len = this->tenant_name.length();
+ this->stor_class = storage_class;
+ this->s.stor_class_len = storage_class.length();
+
+ this->s.ref_tag_len = 0;
+ this->s.manifest_len = 0;
+
+ this->s.shared_manifest = 0;
+ memset(this->s.sha256, 0, sizeof(this->s.sha256));
+ this->ref_tag = "";
+ this->manifest_bl.clear();
+ }
+
+ //---------------------------------------------------------------------------
+ disk_record_t::disk_record_t(const char *buff)
+ {
+ disk_record_t *p_rec = (disk_record_t*)buff;
+ this->s.rec_version = p_rec->s.rec_version;
+ // wrong version, bail out
+ if (unlikely(p_rec->s.rec_version != 0)) {
+ return;
+ }
+
+ this->s.flags = p_rec->s.flags;
+ this->s.num_parts = CEPHTOH_16(p_rec->s.num_parts);
+ this->s.obj_name_len = CEPHTOH_16(p_rec->s.obj_name_len);
+ this->s.bucket_name_len = CEPHTOH_16(p_rec->s.bucket_name_len);
+
+ this->s.md5_high = CEPHTOH_64(p_rec->s.md5_high);
+ this->s.md5_low = CEPHTOH_64(p_rec->s.md5_low);
+ this->s.obj_bytes_size = CEPHTOH_64(p_rec->s.obj_bytes_size);
+ this->s.object_version = CEPHTOH_64(p_rec->s.object_version);
+
+ this->s.bucket_id_len = CEPHTOH_16(p_rec->s.bucket_id_len);
+ this->s.tenant_name_len = CEPHTOH_16(p_rec->s.tenant_name_len);
+ this->s.stor_class_len = CEPHTOH_16(p_rec->s.stor_class_len);
+ this->s.ref_tag_len = CEPHTOH_16(p_rec->s.ref_tag_len);
+ this->s.manifest_len = CEPHTOH_16(p_rec->s.manifest_len);
+
+ const char *p = buff + sizeof(this->s);
+ this->obj_name = std::string(p, this->s.obj_name_len);
+ p += p_rec->s.obj_name_len;
+
+ this->bucket_name = std::string(p, this->s.bucket_name_len);
+ p += p_rec->s.bucket_name_len;
+
+ this->bucket_id = std::string(p, this->s.bucket_id_len);
+ p += p_rec->s.bucket_id_len;
+
+ this->tenant_name = std::string(p, this->s.tenant_name_len);
+ p += p_rec->s.tenant_name_len;
+
+ this->stor_class = std::string(p, this->s.stor_class_len);
+ p += p_rec->s.stor_class_len;
+
+ if (p_rec->s.flags.is_fastlane()) {
+ // TBD:: remove asserts
+ ceph_assert(this->s.ref_tag_len == 0);
+ ceph_assert(this->s.manifest_len == 0);
+ }
+ else {
+ this->s.shared_manifest = CEPHTOH_64(p_rec->s.shared_manifest);
+ for (int i = 0; i < 4; i++) {
+ this->s.sha256[i] = CEPHTOH_64(p_rec->s.sha256[i]);
+ }
+ this->ref_tag = std::string(p, this->s.ref_tag_len);
+ p += p_rec->s.ref_tag_len;
+
+ this->manifest_bl.append(p, this->s.manifest_len);
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ size_t disk_record_t::serialize(char *buff) const
+ {
+ ceph_assert(this->s.rec_version == 0);
+ disk_record_t *p_rec = (disk_record_t*)buff;
+ p_rec->s.rec_version = 0;
+ p_rec->s.flags = this->s.flags;
+ p_rec->s.num_parts = HTOCEPH_16(this->s.num_parts);
+ p_rec->s.obj_name_len = HTOCEPH_16(this->obj_name.length());
+ p_rec->s.bucket_name_len = HTOCEPH_16(this->bucket_name.length());
+
+ p_rec->s.md5_high = HTOCEPH_64(this->s.md5_high);
+ p_rec->s.md5_low = HTOCEPH_64(this->s.md5_low);
+ p_rec->s.obj_bytes_size = HTOCEPH_64(this->s.obj_bytes_size);
+ p_rec->s.object_version = HTOCEPH_64(this->s.object_version);
+
+ p_rec->s.bucket_id_len = HTOCEPH_16(this->bucket_id.length());
+ p_rec->s.tenant_name_len = HTOCEPH_16(this->tenant_name.length());
+ p_rec->s.stor_class_len = HTOCEPH_16(this->stor_class.length());
+ p_rec->s.ref_tag_len = HTOCEPH_16(this->ref_tag.length());
+ p_rec->s.manifest_len = HTOCEPH_16(this->manifest_bl.length());
+ char *p = buff + sizeof(this->s);
+ unsigned len = this->obj_name.length();
+ std::memcpy(p, this->obj_name.data(), len);
+ p += len;
+
+ len = this->bucket_name.length();
+ std::memcpy(p, this->bucket_name.data(), len);
+ p += len;
+
+ len = this->bucket_id.length();
+ std::memcpy(p, this->bucket_id.data(), len);
+ p += len;
+
+ len = this->tenant_name.length();
+ std::memcpy(p, this->tenant_name.data(), len);
+ p += len;
+
+ len = this->stor_class.length();
+ std::memcpy(p, this->stor_class.data(), len);
+ p += len;
+
+ if (this->s.flags.is_fastlane()) {
+ // TBD:: remove asserts
+ ceph_assert(this->s.ref_tag_len == 0);
+ ceph_assert(this->s.manifest_len == 0);
+ }
+ else {
+ p_rec->s.shared_manifest = HTOCEPH_64(this->s.shared_manifest);
+ for (int i = 0; i < 4; i++) {
+ p_rec->s.sha256[i] = HTOCEPH_64(this->s.sha256[i]);
+ }
+ len = this->ref_tag.length();
+ std::memcpy(p, this->ref_tag.data(), len);
+ p += len;
+
+ len = this->manifest_bl.length();
+ const char *p_manifest = const_cast<disk_record_t*>(this)->manifest_bl.c_str();
+ std::memcpy(p, p_manifest, len);
+ p += len;
+ }
+ return (p - buff);
+ }
+
+ //---------------------------------------------------------------------------
+ size_t disk_record_t::length() const
+ {
+ return (sizeof(this->s) +
+ this->obj_name.length() +
+ this->bucket_name.length() +
+ this->bucket_id.length() +
+ this->tenant_name.length() +
+ this->stor_class.length() +
+ this->ref_tag.length() +
+ this->manifest_bl.length());
+ }
+
+ //---------------------------------------------------------------------------
+ int disk_record_t::validate(const char *caller,
+ const DoutPrefixProvider* dpp,
+ disk_block_id_t block_id,
+ record_id_t rec_id) const
+ {
+ // optimistic approach
+ if (likely((this->s.rec_version == 0) && (this->length() <= MAX_REC_SIZE))) {
+ ldpp_dout(dpp, 20) << __func__ << "::success" << dendl;
+ return 0;
+ }
+
+ // wrong version
+ if (this->s.rec_version != 0) {
+ // TBD
+ //p_stats->failed_wrong_ver++;
+ ldpp_dout(dpp, 5) << __func__ << "::" << caller << "::ERR: Bad record version: "
+ << this->s.rec_version
+ << "::block_id=" << block_id
+ << "::rec_id=" << rec_id
+ << dendl;
+ return -EPROTO; // Protocol error
+ }
+
+ // if arrived here record size is too large
+ // TBD
+ //p_stats->failed_rec_overflow++;
+ ldpp_dout(dpp, 5) << __func__ << "::" << caller << "::ERR: record size too big: "
+ << this->length()
+ << "::block_id=" << block_id
+ << "::rec_id=" << rec_id
+ << dendl;
+ return -EOVERFLOW; // maybe should use -E2BIG ??
+ }
+
+ //---------------------------------------------------------------------------
+ std::ostream &operator<<(std::ostream &stream, const disk_record_t & rec)
+ {
+ stream << rec.obj_name << "::" << rec.s.obj_name_len << "\n";
+ stream << rec.bucket_name << "::" << rec.s.bucket_name_len << "\n";
+ stream << rec.bucket_id << "::" << rec.s.bucket_id_len << "\n";
+ stream << rec.tenant_name << "::" << rec.s.tenant_name_len << "\n";
+ stream << rec.stor_class << "::" << rec.s.stor_class_len << "\n";
+ stream << rec.ref_tag << "::" << rec.s.ref_tag_len << "\n";
+ stream << "num_parts = " << rec.s.num_parts << "\n";
+ stream << "obj_size = " << rec.s.obj_bytes_size/1024 <<" KiB" << "\n";
+ stream << "MD5 = " << std::hex << rec.s.md5_high << rec.s.md5_low << "\n";
+ stream << "SHA256 = ";
+ for (int i =0; i < 4; i++) {
+ stream << rec.s.sha256[i];
+ }
+ stream << "\n";
+
+ if (rec.has_shared_manifest()) {
+ stream << "Shared Manifest Object\n";
+ }
+ else {
+ stream << "Dedicated Manifest Object\n";
+ }
+ stream << "Manifest len=" << rec.s.manifest_len << "\n";
+ return stream;
+ }
+
+ //---------------------------------------------------------------------------
+ void disk_block_t::init(work_shard_t worker_id, uint32_t seq_number)
+ {
+ disk_block_header_t *p_header = get_header();
+ p_header->offset = sizeof(disk_block_header_t);
+ p_header->rec_count = 0;
+ p_header->block_id = disk_block_id_t(worker_id, seq_number);
+ }
+
+ //---------------------------------------------------------------------------
+ int disk_block_header_t::verify(disk_block_id_t expected_block_id, const DoutPrefixProvider* dpp)
+ {
+ if (unlikely(offset != BLOCK_MAGIC && offset != LAST_BLOCK_MAGIC)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR::bad magic number (0x" << std::hex << offset << std::dec << ")" << dendl;
+ return -EINVAL;
+ }
+
+ if (unlikely(rec_count > MAX_REC_IN_BLOCK) ) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR::rec_count=" << rec_count << " > MAX_REC_IN_BLOCK" << dendl;
+ return -EINVAL;
+ }
+
+ if (unlikely(this->block_id != expected_block_id)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR::block_id=" << block_id
+ << "!= expected_block_id=" << expected_block_id << dendl;
+ return -EINVAL;
+ }
+
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ record_id_t disk_block_t::add_record(const disk_record_t *p_rec,
+ const DoutPrefixProvider *dpp)
+ {
+ disk_block_header_t *p_header = get_header();
+ if (unlikely(p_header->rec_count >= MAX_REC_IN_BLOCK)) {
+ ldpp_dout(dpp, 20) << __func__ << "::rec_count=" << p_header->rec_count
+ << ", MAX_REC_IN_BLOCK=" << MAX_REC_IN_BLOCK << dendl;
+ return MAX_REC_IN_BLOCK;
+ }
+
+ if ((DISK_BLOCK_SIZE - p_header->offset) >= p_rec->length()) {
+ p_header->rec_offsets[p_header->rec_count] = p_header->offset;
+ unsigned rec_id = p_header->rec_count;
+ p_header->rec_count ++;
+ p_rec->serialize(data+p_header->offset);
+ p_header->offset += p_rec->length();
+ return rec_id;
+ }
+ else {
+ return MAX_REC_IN_BLOCK;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ void disk_block_t::close_block(const DoutPrefixProvider* dpp, bool has_more)
+ {
+ disk_block_header_t *p_header = get_header();
+ ldpp_dout(dpp, 20) << __func__ << "::rec_count=" << p_header->rec_count
+ << ", has_more=" << (has_more? "TRUE" : "FALSE") << dendl;
+
+ memset(data + p_header->offset, 0, (DISK_BLOCK_SIZE - p_header->offset));
+ if (has_more) {
+ p_header->offset = HTOCEPH_16(BLOCK_MAGIC);
+ }
+ else {
+ p_header->offset = HTOCEPH_16(LAST_BLOCK_MAGIC);
+ }
+ for (unsigned i = 0; i < p_header->rec_count; i++) {
+ p_header->rec_offsets[i] = HTOCEPH_16(p_header->rec_offsets[i]);
+ }
+ p_header->rec_count = HTOCEPH_16(p_header->rec_count);
+ p_header->block_id = HTOCEPH_32((uint32_t)p_header->block_id);
+ // TBD: CRC
+ }
+
+ //---------------------------------------------------------------------------
+ void disk_block_header_t::deserialize()
+ {
+ this->offset = CEPHTOH_16(this->offset);
+ this->rec_count = CEPHTOH_16(this->rec_count);
+ this->block_id = CEPHTOH_32((uint32_t)this->block_id);
+ for (unsigned i = 0; i < this->rec_count; i++) {
+ this->rec_offsets[i] = CEPHTOH_16(this->rec_offsets[i]);
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ disk_block_seq_t::disk_block_seq_t(const DoutPrefixProvider* dpp_in,
+ disk_block_t *p_arr_in,
+ work_shard_t worker_id,
+ md5_shard_t md5_shard,
+ worker_stats_t *p_stats_in)
+ {
+ activate(dpp_in, p_arr_in, worker_id, md5_shard, p_stats_in);
+ }
+
+ //---------------------------------------------------------------------------
+ void disk_block_seq_t::activate(const DoutPrefixProvider* dpp_in,
+ disk_block_t *p_arr_in,
+ work_shard_t worker_id,
+ md5_shard_t md5_shard,
+ worker_stats_t *p_stats_in)
+ {
+ dpp = dpp_in;
+ p_arr = p_arr_in;
+ d_worker_id = worker_id;
+ d_md5_shard = md5_shard;
+ p_stats = p_stats_in;
+ p_curr_block = nullptr;
+ d_seq_number = 0;
+
+ memset(p_arr, 0, sizeof(disk_block_t));
+ slab_reset();
+ }
+
+ //---------------------------------------------------------------------------
+ [[maybe_unused]]static int print_manifest(const DoutPrefixProvider *dpp,
+ RGWRados *rados,
+ const bufferlist &manifest_bl)
+ {
+ RGWObjManifest manifest;
+ try {
+ auto bl_iter = manifest_bl.cbegin();
+ decode(manifest, bl_iter);
+ } catch (buffer::error& err) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERROR: unable to decode manifest" << dendl;
+ return -EINVAL;
+ }
+
+ unsigned idx = 0;
+ for (auto p = manifest.obj_begin(dpp); p != manifest.obj_end(dpp); ++p, ++idx) {
+ rgw_raw_obj raw_obj = p.get_location().get_raw_obj(rados);
+ ldpp_dout(dpp, 20) << idx << "] " << raw_obj.oid << dendl;
+ }
+ ldpp_dout(dpp, 20) << "==============================================" << dendl;
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ std::ostream& operator<<(std::ostream& out, const disk_block_id_t& block_id)
+ {
+ std::ios_base::fmtflags flags = out.flags();
+ out << std::hex << "0x"
+ << (uint32_t)block_id.get_work_shard_id() << "::"
+ << (uint32_t)block_id.get_slab_id() << "::"
+ << (uint32_t)block_id.get_block_offset();
+
+ if (flags & std::ios::dec) {
+ out << std::dec;
+ }
+ return out;
+ }
+
+ //---------------------------------------------------------------------------
+ std::string disk_block_id_t::get_slab_name(md5_shard_t md5_shard) const
+ {
+ // SLAB.MD5_ID.WORKER_ID.SLAB_SEQ_ID
+ const char *SLAB_NAME_FORMAT = "SLB.%03X.%02X.%04X";
+ static constexpr uint32_t SLAB_NAME_SIZE = 16;
+ char name_buf[SLAB_NAME_SIZE];
+ slab_id_t slab_id = get_slab_id();
+ work_shard_t work_id = get_work_shard_id();
+ unsigned n = snprintf(name_buf, sizeof(name_buf), SLAB_NAME_FORMAT,
+ md5_shard, work_id, slab_id);
+ std::string oid(name_buf, n);
+ return oid;
+ }
+
+ //---------------------------------------------------------------------------
+ int load_record(librados::IoCtx &ioctx,
+ const disk_record_t *p_tgt_rec,
+ disk_record_t *p_src_rec, /* OUT */
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ md5_shard_t md5_shard,
+ const DoutPrefixProvider *dpp)
+ {
+ std::string oid(block_id.get_slab_name(md5_shard));
+ int read_len = DISK_BLOCK_SIZE;
+ static_assert(sizeof(disk_block_t) == DISK_BLOCK_SIZE);
+ int byte_offset = block_id.get_block_offset() * DISK_BLOCK_SIZE;
+ bufferlist bl;
+ int ret = ioctx.read(oid, bl, read_len, byte_offset);
+ if (unlikely(ret != read_len)) {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: failed to read block from " << oid
+ << "::ret=" << ret << "::err=" << cpp_strerror(-ret)<<dendl;
+ return ret;
+ }
+ else {
+ ldpp_dout(dpp, 20) << __func__ << "::oid=" << oid << "::ret=" << ret
+ << "::len=" << bl.length() << dendl;
+ }
+
+ const char *p = bl.c_str();
+ disk_block_t *p_disk_block = (disk_block_t*)p;
+ disk_block_header_t *p_header = p_disk_block->get_header();
+ p_header->deserialize();
+ ret = p_header->verify(block_id, dpp);
+ if (ret != 0) {
+ return ret;
+ }
+
+ unsigned offset = p_header->rec_offsets[rec_id];
+ // We deserialize the record inside the CTOR
+ disk_record_t rec(p + offset);
+ ret = rec.validate(__func__, dpp, block_id, rec_id);
+ if (unlikely(ret != 0)) {
+ //p_stats->failed_rec_load++;
+ return ret;
+ }
+
+ if (rec.s.md5_high == p_tgt_rec->s.md5_high &&
+ rec.s.md5_low == p_tgt_rec->s.md5_low &&
+ rec.s.num_parts == p_tgt_rec->s.num_parts &&
+ rec.s.obj_bytes_size == p_tgt_rec->s.obj_bytes_size &&
+ rec.stor_class == p_tgt_rec->stor_class) {
+
+ *p_src_rec = rec;
+ return 0;
+ }
+ else {
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: Bad record in block=" << block_id
+ << ", rec_id=" << rec_id << dendl;
+ return -EIO;
+ }
+
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ [[maybe_unused]]static void
+ copy_bl_multi_parts(const bufferlist &bl_in, bufferlist &bl_out,
+ const DoutPrefixProvider* dpp)
+ {
+ const size_t MAX = 260*1024;
+ char buff[MAX];
+ std::srand(std::time({}));
+
+ std::vector<int> vec;
+ auto bl_itr = bl_in.cbegin();
+ size_t len = bl_in.length();
+ while (len) {
+ const int random_value = std::rand();
+ size_t req_len = std::min((random_value % MAX), len);
+ if (len < MAX) {
+ req_len = len;
+ }
+ vec.push_back(req_len);
+ const char *p = get_next_data_ptr(bl_itr, buff, req_len, dpp);
+ bufferptr ptr(p, req_len);
+ bl_out.append(ptr);
+ len -= req_len;
+ }
+ ldpp_dout(dpp, 20) << __func__ << "::req_len=" << vec << dendl;
+ }
+
+ //---------------------------------------------------------------------------
+ int load_slab(librados::IoCtx &ioctx,
+ bufferlist &bl_out,
+ md5_shard_t md5_shard,
+ work_shard_t worker_id,
+ uint32_t seq_number,
+ const DoutPrefixProvider* dpp)
+ {
+ disk_block_id_t block_id(worker_id, seq_number);
+ std::string oid(block_id.get_slab_name(md5_shard));
+ ldpp_dout(dpp, 20) << __func__ << "::worker_id=" << (uint32_t)worker_id
+ << ", md5_shard=" << (uint32_t)md5_shard
+ << ", seq_number=" << seq_number
+ << ":: oid=" << oid << dendl;
+#ifndef DEBUG_FRAGMENTED_BUFFERLIST
+ int ret = ioctx.read(oid, bl_out, 0, 0);
+ if (ret > 0) {
+ ldpp_dout(dpp, 20) << __func__ << "::oid=" << oid << ", len="
+ << bl_out.length() << dendl;
+ }
+#else
+ // DEBUG MODE to test with fragmented bufferlist
+ bufferlist bl_in;
+ // read full object
+ int ret = ioctx.read(oid, bl_in, 0, 0);
+ if (ret > 0) {
+ ldpp_dout(dpp, 20) << __func__ << "::oid=" << oid << ", len="
+ << bl_in.length() << dendl;
+ copy_bl_multi_parts(bl_in, bl_out, dpp);
+ }
+#endif
+ else {
+ if (ret == 0) {
+ // no error reported, but we read nothing which should never happen
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: Empty SLAB " << oid << dendl;
+ ret = -ENODATA;
+ }
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: failed to read " << oid
+ << ", error is " << cpp_strerror(-ret) << dendl;
+ }
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int store_slab(librados::IoCtx &ioctx,
+ bufferlist &bl,
+ md5_shard_t md5_shard,
+ work_shard_t worker_id,
+ uint32_t seq_number,
+ const DoutPrefixProvider* dpp)
+ {
+ disk_block_id_t block_id(worker_id, seq_number);
+ std::string oid(block_id.get_slab_name(md5_shard));
+ ldpp_dout(dpp, 20) << __func__ << "::oid=" << oid << ", len="
+ << bl.length() << dendl;
+ ceph_assert(bl.length());
+
+ int ret = ioctx.write_full(oid, bl);
+ if (ret == (int)bl.length()) {
+ ldpp_dout(dpp, 20) << __func__ << "::wrote " << bl.length() << " bytes to "
+ << oid << dendl;
+ }
+ else {
+ if (ret == 0) {
+ // no error reported, but we wrote nothing which should never happen
+ ldpp_dout(dpp, 5) << __func__ << "::ERR: No Data was written to " << oid
+ << ", bl.length()=" << bl.length() << dendl;
+ ret = -ENODATA;
+ }
+ ldpp_dout(dpp, 1) << "ERROR: failed to write " << oid
+ << " with: " << cpp_strerror(-ret) << dendl;
+ }
+
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int disk_block_seq_t::flush(librados::IoCtx &ioctx)
+ {
+ unsigned len = (p_curr_block + 1 - p_arr) * sizeof(disk_block_t);
+ bufferlist bl = bufferlist::static_from_mem((char*)p_arr, len);
+ int ret = store_slab(ioctx, bl, d_md5_shard, d_worker_id, d_seq_number, dpp);
+ // Need to make sure the call to rgw_put_system_obj was fully synchronous
+
+ // d_seq_number++ must be called **after** flush!!
+ d_seq_number++;
+ p_stats->egress_slabs++;
+ slab_reset();
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int disk_block_seq_t::flush_disk_records(librados::IoCtx &ioctx)
+ {
+ ceph_assert(p_arr);
+ ldpp_dout(dpp, 20) << __func__ << "::worker_id=" << (uint32_t)d_worker_id
+ << ", md5_shard=" << (uint32_t)d_md5_shard << dendl;
+
+ // we need to force flush at the end of a cycle even if there was no work done
+ // it is used as a signal to worker in the next step
+ if (p_curr_block == &p_arr[0] && p_curr_block->is_empty()) {
+ ldpp_dout(dpp, 20) << __func__ << "::Empty buffers, generate terminating block" << dendl;
+ }
+ p_stats->egress_blocks++;
+ p_curr_block->close_block(dpp, false);
+
+ int ret = flush(ioctx);
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ int disk_block_seq_t::add_record(librados::IoCtx &ioctx,
+ const disk_record_t *p_rec, // IN-OUT
+ record_info_t *p_rec_info) // OUT-PARAM
+ {
+ disk_block_id_t null_block_id;
+ int ret = p_rec->validate(__func__, dpp, null_block_id, MAX_REC_IN_BLOCK);
+ if (unlikely(ret != 0)) {
+ // TBD
+ //p_stats->failed_rec_store++;
+ return ret;
+ }
+
+ p_stats->egress_records ++;
+ // first, try and add the record to the current open block
+ p_rec_info->rec_id = p_curr_block->add_record(p_rec, dpp);
+ if (p_rec_info->rec_id < MAX_REC_IN_BLOCK) {
+ p_rec_info->block_id = p_curr_block->get_block_id();
+ return 0;
+ }
+ else {
+ // Not enough space left in current block, close it and open the next block
+ ldpp_dout(dpp, 20) << __func__ << "::Block is full-> close and move to next" << dendl;
+ p_stats->egress_blocks++;
+ p_curr_block->close_block(dpp, true);
+ }
+
+ // Do we have more Blocks in the block-array ?
+ if (p_curr_block < last_block()) {
+ p_curr_block ++;
+ d_seq_number ++;
+ p_curr_block->init(d_worker_id, d_seq_number);
+ p_rec_info->rec_id = p_curr_block->add_record(p_rec, dpp);
+ }
+ else {
+ ldpp_dout(dpp, 20) << __func__ << "::calling flush()" << dendl;
+ ret = flush(ioctx);
+ p_rec_info->rec_id = p_curr_block->add_record(p_rec, dpp);
+ }
+
+ p_rec_info->block_id = p_curr_block->get_block_id();
+ return ret;
+ }
+
+ //---------------------------------------------------------------------------
+ disk_block_array_t::disk_block_array_t(const DoutPrefixProvider* dpp,
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size,
+ work_shard_t worker_id,
+ worker_stats_t *p_stats,
+ md5_shard_t num_md5_shards)
+ {
+ d_num_md5_shards = num_md5_shards;
+ d_worker_id = worker_id;
+ disk_block_t *p = (disk_block_t *)raw_mem;
+ disk_block_t *p_end = (disk_block_t *)(raw_mem + raw_mem_size);
+
+ for (unsigned md5_shard = 0; md5_shard < d_num_md5_shards; md5_shard++) {
+ ldpp_dout(dpp, 20) << __func__ << "::p=" << p << "::p_end=" << p_end << dendl;
+ if (p + DISK_BLOCK_COUNT <= p_end) {
+ d_disk_arr[md5_shard].activate(dpp, p, d_worker_id, md5_shard, p_stats);
+ p += DISK_BLOCK_COUNT;
+ }
+ else {
+ ldpp_dout(dpp, 1) << __func__ << "::ERR: buffer overflow! "
+ << "::md5_shard=" << md5_shard << "/" << d_num_md5_shards
+ << "::raw_mem_size=" << raw_mem_size << dendl;
+ ldpp_dout(dpp, 1) << __func__
+ << "::sizeof(disk_block_t)=" << sizeof(disk_block_t)
+ << "::DISK_BLOCK_COUNT=" << DISK_BLOCK_COUNT << dendl;
+ ceph_abort();
+ }
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ void disk_block_array_t::flush_output_buffers(const DoutPrefixProvider* dpp,
+ librados::IoCtx &ioctx)
+ {
+ for (md5_shard_t md5_shard = 0; md5_shard < d_num_md5_shards; md5_shard++) {
+ ldpp_dout(dpp, 20) <<__func__ << "::flush buffers:: worker_id="
+ << d_worker_id<< ", md5_shard=" << md5_shard << dendl;
+ d_disk_arr[md5_shard].flush_disk_records(ioctx);
+ }
+ }
+} // namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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.
+ *
+ */
+
+#pragma once
+#include "common/dout.h"
+#include "rgw_common.h"
+#include "rgw_realm_reloader.h"
+#include <string>
+#include <unordered_map>
+#include <variant>
+#include <iostream>
+#include <ostream>
+#include <cstring>
+#include <string>
+#include "include/rados/rados_types.hpp"
+#include "include/rados/buffer.h"
+#include "include/rados/librados.hpp"
+#include "rgw_dedup_utils.h"
+
+namespace rgw::dedup {
+ struct key_t;
+#define CEPHTOH_16 le16toh
+#define CEPHTOH_32 le32toh
+#define CEPHTOH_64 le64toh
+#define HTOCEPH_16 htole16
+#define HTOCEPH_32 htole32
+#define HTOCEPH_64 htole64
+
+ static inline constexpr unsigned DISK_BLOCK_SIZE = 8*1024;
+ // we use 16 bit offset
+ static_assert(DISK_BLOCK_SIZE < 64*1024);
+ static constexpr unsigned DISK_BLOCK_COUNT = 256;
+ static_assert(DISK_BLOCK_COUNT <= (4*1024*1024/DISK_BLOCK_SIZE));
+ static constexpr unsigned MAX_REC_IN_BLOCK = 32;
+ // we use 8bit record indices
+ static_assert(MAX_REC_IN_BLOCK < 0xFF);
+ using slab_id_t = uint16_t;
+ using block_offset_t = uint8_t;
+ using record_id_t = uint8_t;
+
+ // disk_block_id_t is a 32 bits concataion of shard_id, slab_id and block_off
+ // ---8---- | -------16------- | ---8----
+ // shard_id | slab_id | block_off
+ struct __attribute__ ((packed)) disk_block_id_t
+ {
+ public:
+ disk_block_id_t() {
+ block_id = 0;
+ }
+
+ disk_block_id_t(work_shard_t shard_id, uint32_t seq_number) {
+ ceph_assert((seq_number & SEQ_NUMBER_MASK) == seq_number);
+ ceph_assert(shard_id <= MAX_WORK_SHARD);
+ block_id = (uint32_t)shard_id << OBJ_SHARD_SHIFT | seq_number;
+ }
+
+ disk_block_id_t& operator =(const disk_block_id_t &other) {
+ this->block_id = other.block_id;
+ return *this;
+ }
+
+ inline disk_block_id_t& operator =(uint32_t val) {
+ this->block_id = val;
+ return *this;
+ }
+
+ inline bool operator ==(const disk_block_id_t &other) const {
+ return (this->block_id == other.block_id);
+ }
+
+ inline explicit operator uint32_t() const {
+ return this->block_id;
+ }
+
+ friend std::ostream& operator<<(std::ostream& os, const disk_block_id_t& block_id);
+
+ std::string get_slab_name(md5_shard_t md5_shard) const;
+
+ static inline slab_id_t seq_num_to_slab_id(uint32_t seq_number) {
+ return (seq_number & SLAB_ID_MASK) >> SLAB_ID_SHIFT;
+ }
+
+ static inline uint32_t slab_id_to_seq_num(uint32_t slab_id) {
+ return (slab_id << SLAB_ID_SHIFT);
+ }
+
+ inline block_offset_t get_block_offset() const {
+ return get_block_offset(get_seq_num());
+ }
+
+ inline work_shard_t get_work_shard_id() const {
+ return (block_id & OBJ_SHARD_MASK) >> OBJ_SHARD_SHIFT;
+ }
+
+ private:
+ inline uint32_t get_seq_num() const {
+ return (block_id & SEQ_NUMBER_MASK);
+ }
+
+ inline slab_id_t get_slab_id() const {
+ return seq_num_to_slab_id(get_seq_num());
+ }
+
+ inline block_offset_t get_block_offset(uint32_t seq_number) const {
+ return (seq_number & BLOCK_OFF_MASK);
+ }
+
+ static constexpr uint32_t OBJ_SHARD_SHIFT = 24;
+ static constexpr uint32_t OBJ_SHARD_MASK = 0xFF000000;
+
+ static constexpr uint32_t SEQ_NUMBER_SHIFT = 0;
+ static constexpr uint32_t SEQ_NUMBER_MASK = 0x00FFFFFF;
+
+ static constexpr uint32_t SLAB_ID_SHIFT = 8;
+ static constexpr uint32_t SLAB_ID_MASK = 0x00FFFF00;
+
+ static constexpr uint32_t BLOCK_OFF_SHIFT = 0;
+ static constexpr uint32_t BLOCK_OFF_MASK = 0x000000FF;
+
+ uint32_t block_id;
+ };
+
+ struct disk_record_t
+ {
+ disk_record_t(const char *buff);
+ disk_record_t(const rgw::sal::Bucket *p_bucket,
+ const std::string &obj_name,
+ const parsed_etag_t *p_parsed_etag,
+ uint64_t obj_size,
+ const std::string &storage_class);
+ disk_record_t() {}
+ size_t serialize(char *buff) const;
+ size_t length() const;
+ int validate(const char *caller,
+ const DoutPrefixProvider* dpp,
+ disk_block_id_t block_id,
+ record_id_t rec_id) const;
+ inline bool has_shared_manifest() const { return s.flags.has_shared_manifest(); }
+ inline void set_shared_manifest() { s.flags.set_shared_manifest(); }
+
+ struct __attribute__ ((packed)) packed_rec_t
+ {
+ uint8_t rec_version; // allows changing record format
+ dedup_flags_t flags; // 1 Byte flags
+ uint16_t num_parts; // For multipart upload (AWS MAX-PART is 10,000)
+ uint16_t obj_name_len;
+ uint16_t bucket_name_len;
+
+ uint64_t md5_high; // High Bytes of the Object Data MD5
+ uint64_t md5_low; // Low Bytes of the Object Data MD5
+ uint64_t obj_bytes_size;
+ uint64_t object_version;
+
+ uint16_t bucket_id_len;
+ uint16_t tenant_name_len;
+ uint16_t stor_class_len;
+ uint16_t ref_tag_len;
+
+ uint16_t manifest_len;
+ uint8_t pad[6];
+
+ uint64_t shared_manifest; // 64bit hash of the SRC object manifest
+ uint64_t sha256[4]; // 4 * 8 Bytes of SHA256
+ }s;
+ std::string obj_name;
+ // TBD: find pool name making it easier to get ioctx
+ std::string bucket_name;
+ std::string bucket_id;
+ std::string tenant_name;
+ std::string ref_tag;
+ std::string stor_class;
+ bufferlist manifest_bl;
+ };
+
+ std::ostream &operator<<(std::ostream &stream, const disk_record_t & rec);
+
+ static constexpr unsigned BLOCK_MAGIC = 0xFACE;
+ static constexpr unsigned LAST_BLOCK_MAGIC = 0xCAD7;
+ struct __attribute__ ((packed)) disk_block_header_t {
+ void deserialize();
+ int verify(disk_block_id_t block_id, const DoutPrefixProvider* dpp);
+ uint16_t offset;
+ uint16_t rec_count;
+ disk_block_id_t block_id;
+ uint16_t rec_offsets[MAX_REC_IN_BLOCK];
+ };
+ static constexpr unsigned MAX_REC_SIZE = (DISK_BLOCK_SIZE - sizeof(disk_block_header_t));
+
+ struct __attribute__ ((packed)) disk_block_t
+ {
+ const disk_block_header_t* get_header() const { return (disk_block_header_t*)data; }
+ disk_block_header_t* get_header() { return (disk_block_header_t*)data; }
+ bool is_empty() const { return (get_header()->rec_count == 0); }
+
+ void init(work_shard_t worker_id, uint32_t seq_number);
+ record_id_t add_record(const disk_record_t *p_rec, const DoutPrefixProvider *dpp);
+ void close_block(const DoutPrefixProvider* dpp, bool has_more);
+ disk_block_id_t get_block_id() {
+ disk_block_header_t *p_header = get_header();
+ return p_header->block_id;
+ }
+ char data[DISK_BLOCK_SIZE];
+ };
+
+ int load_record(librados::IoCtx &ioctx,
+ const disk_record_t *p_tgt_rec,
+ disk_record_t *p_src_rec, /* OUT */
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ md5_shard_t md5_shard,
+ const DoutPrefixProvider *dpp);
+
+ int load_slab(librados::IoCtx &ioctx,
+ bufferlist &bl,
+ md5_shard_t md5_shard,
+ work_shard_t worker_id,
+ uint32_t seq_number,
+ const DoutPrefixProvider* dpp);
+
+ int store_slab(librados::IoCtx &ioctx,
+ bufferlist &bl,
+ md5_shard_t md5_shard,
+ work_shard_t worker_id,
+ uint32_t seq_number,
+ const DoutPrefixProvider* dpp);
+
+ class disk_block_array_t;
+ class disk_block_seq_t
+ {
+ friend class disk_block_array_t;
+ public:
+ struct record_info_t {
+ disk_block_id_t block_id;
+ record_id_t rec_id;
+ };
+
+ disk_block_seq_t(const DoutPrefixProvider* dpp_in,
+ disk_block_t *p_arr_in,
+ work_shard_t worker_id,
+ md5_shard_t md5_shard,
+ worker_stats_t *p_stats_in);
+ int flush_disk_records(librados::IoCtx &ioctx);
+ md5_shard_t get_md5_shard() { return d_md5_shard; }
+ int add_record(librados::IoCtx &ioctx,
+ const disk_record_t *p_rec, // IN-OUT
+ record_info_t *p_rec_info); // OUT-PARAM
+
+ private:
+ disk_block_seq_t() {;}
+ void activate(const DoutPrefixProvider* _dpp,
+ disk_block_t *_p_arr,
+ work_shard_t worker_id,
+ md5_shard_t md5_shard,
+ worker_stats_t *p_stats);
+ inline const disk_block_t* last_block() { return &p_arr[DISK_BLOCK_COUNT-1]; }
+ int flush(librados::IoCtx &ioctx);
+ void slab_reset() {
+ p_curr_block = p_arr;
+ p_curr_block->init(d_worker_id, d_seq_number);
+ }
+
+ disk_block_t *p_arr = nullptr;
+ disk_block_t *p_curr_block = nullptr;
+ worker_stats_t *p_stats = nullptr;
+ const DoutPrefixProvider *dpp = nullptr;
+ uint32_t d_seq_number = 0;
+ work_shard_t d_worker_id = NULL_WORK_SHARD;
+ md5_shard_t d_md5_shard = NULL_MD5_SHARD;
+ };
+
+ class disk_block_array_t
+ {
+ public:
+ disk_block_array_t(const DoutPrefixProvider* _dpp,
+ uint8_t *raw_mem,
+ uint64_t raw_mem_size,
+ work_shard_t worker_id,
+ worker_stats_t *p_worker_stats,
+ md5_shard_t num_md5_shards);
+ void flush_output_buffers(const DoutPrefixProvider* dpp,
+ librados::IoCtx &ioctx);
+ disk_block_seq_t* get_shard_block_seq(uint64_t md5_low) {
+ md5_shard_t md5_shard = md5_low % d_num_md5_shards;
+ return d_disk_arr + md5_shard;
+ }
+
+ //private:
+ disk_block_seq_t d_disk_arr[MAX_MD5_SHARD];
+ work_shard_t d_worker_id;
+ md5_shard_t d_num_md5_shards;
+ };
+} //namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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 "rgw_dedup_table.h"
+#include "include/ceph_assert.h"
+#include <cstring>
+#include <iostream>
+
+namespace rgw::dedup {
+
+ //---------------------------------------------------------------------------
+ dedup_table_t::dedup_table_t(const DoutPrefixProvider* _dpp,
+ uint32_t _head_object_size,
+ uint8_t *p_slab,
+ uint64_t slab_size)
+ {
+ dpp = _dpp;
+ head_object_size = _head_object_size;
+ memset(p_slab, 0, slab_size);
+ hash_tab = (table_entry_t*)p_slab;
+ entries_count = slab_size/sizeof(table_entry_t);
+ values_count = 0;
+ occupied_count = 0;
+ }
+
+ //---------------------------------------------------------------------------
+ void dedup_table_t::remove_singletons_and_redistribute_keys()
+ {
+ for (uint32_t tab_idx = 0; tab_idx < entries_count; tab_idx++) {
+ if (!hash_tab[tab_idx].val.is_occupied()) {
+ continue;
+ }
+
+ if (hash_tab[tab_idx].val.is_singleton()) {
+ hash_tab[tab_idx].val.clear_flags();
+ redistributed_clear++;
+ continue;
+ }
+
+ const key_t &key = hash_tab[tab_idx].key;
+ uint32_t key_idx = key.hash() % entries_count;
+ if (key_idx != tab_idx) {
+ uint64_t count = 1;
+ redistributed_count++;
+ uint32_t idx = key_idx;
+ while (hash_tab[idx].val.is_occupied() &&
+ !hash_tab[idx].val.is_singleton() &&
+ (hash_tab[idx].key != key)) {
+ count++;
+ idx = (idx + 1) % entries_count;
+ }
+
+ if (idx != tab_idx) {
+ if (hash_tab[idx].val.is_occupied() && hash_tab[idx].val.is_singleton() ) {
+ redistributed_clear++;
+ }
+ if (idx == key_idx) {
+ redistributed_perfect++;
+ }
+ hash_tab[idx] = hash_tab[tab_idx];
+ hash_tab[tab_idx].val.clear_flags();
+ }
+ else {
+ redistributed_loopback++;
+ }
+
+ redistributed_search_max = std::max(redistributed_search_max, count);
+ redistributed_search_total += count;
+ }
+ else {
+ redistributed_not_needed++;
+ }
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ uint32_t dedup_table_t::find_entry(const key_t *p_key) const
+ {
+ uint32_t idx = p_key->hash() % entries_count;
+
+ // search until we either find the key, or find an empty slot.
+ while (hash_tab[idx].val.is_occupied() && (hash_tab[idx].key != *p_key)) {
+ idx = (idx + 1) % entries_count;
+ }
+ return idx;
+ }
+
+ //---------------------------------------------------------------------------
+ int dedup_table_t::add_entry(key_t *p_key,
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ bool shared_manifest)
+ {
+ value_t new_val(block_id, rec_id, shared_manifest);
+ uint32_t idx = find_entry(p_key);
+ value_t &val = hash_tab[idx].val;
+ if (!val.is_occupied()) {
+ if (occupied_count < entries_count) {
+ occupied_count++;
+ }
+ else {
+ return -EOVERFLOW;
+ }
+
+ hash_tab[idx].key = *p_key;
+ hash_tab[idx].val = new_val;
+ ldpp_dout(dpp, 20) << __func__ << "::add new entry" << dendl;
+ ceph_assert(val.count == 1);
+ }
+ else {
+ ceph_assert(hash_tab[idx].key == *p_key);
+ val.count ++;
+ if (!val.has_shared_manifest() && shared_manifest) {
+ // replace value!
+ ldpp_dout(dpp, 20) << __func__ << "::Replace with shared_manifest::["
+ << val.block_idx << "/" << (int)val.rec_id << "] -> ["
+ << block_id << "/" << (int)rec_id << "]" << dendl;
+ new_val.count = val.count;
+ hash_tab[idx].val = new_val;
+ }
+ ceph_assert(val.count > 1);
+ }
+ values_count++;
+ ldpp_dout(dpp, 20) << __func__ << "::COUNT="<< val.count << dendl;
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ void dedup_table_t::update_entry(key_t *p_key,
+ disk_block_id_t block_id,
+ record_id_t rec_id,
+ bool shared_manifest)
+ {
+ uint32_t idx = find_entry(p_key);
+ ceph_assert(hash_tab[idx].key == *p_key);
+ value_t &val = hash_tab[idx].val;
+ ceph_assert(val.is_occupied());
+ // we only update non-singletons since we purge singletons after the first pass
+ ceph_assert(val.count > 1);
+
+ // need to overwrite the block_idx/rec_id from the first pass
+ // unless already set with shared_manifest with the correct block-id/rec-id
+ // We only set the shared_manifest flag on the second pass where we
+ // got valid block-id/rec-id
+ if (!val.has_shared_manifest()) {
+ // replace value!
+ value_t new_val(block_id, rec_id, shared_manifest);
+ new_val.count = val.count;
+ hash_tab[idx].val = new_val;
+ ldpp_dout(dpp, 20) << __func__ << "::Replaced table entry::["
+ << val.block_idx << "/" << (int)val.rec_id << "] -> ["
+ << block_id << "/" << (int)rec_id << "]" << dendl;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ int dedup_table_t::set_shared_manifest_src_mode(const key_t *p_key,
+ disk_block_id_t block_id,
+ record_id_t rec_id)
+ {
+ uint32_t idx = find_entry(p_key);
+ value_t &val = hash_tab[idx].val;
+ if (val.is_occupied()) {
+ if (val.block_idx == block_id && val.rec_id == rec_id) {
+ val.set_shared_manifest_src();
+ return 0;
+ }
+ }
+
+ return -ENOENT;
+ }
+
+ //---------------------------------------------------------------------------
+ int dedup_table_t::get_val(const key_t *p_key, struct value_t *p_val /*OUT*/)
+ {
+ uint32_t idx = find_entry(p_key);
+ const value_t &val = hash_tab[idx].val;
+ if (!val.is_occupied()) {
+ return -ENOENT;
+ }
+
+ *p_val = val;
+ return 0;
+ }
+
+ //---------------------------------------------------------------------------
+ void dedup_table_t::count_duplicates(uint64_t *p_singleton_count,
+ uint64_t *p_unique_count,
+ uint64_t *p_duplicate_count,
+ uint64_t *p_duplicate_bytes_approx)
+ {
+ for (uint32_t tab_idx = 0; tab_idx < entries_count; tab_idx++) {
+ if (!hash_tab[tab_idx].val.is_occupied()) {
+ continue;
+ }
+
+ if (hash_tab[tab_idx].val.is_singleton()) {
+ (*p_singleton_count)++;
+ }
+ else {
+ ceph_assert(hash_tab[tab_idx].val.count > 1);
+ uint32_t duplicate_count = (hash_tab[tab_idx].val.count -1);
+ key_t &key = hash_tab[tab_idx].key;
+ // This is an approximation only since size is stored in 4KB resolution
+ uint64_t byte_size_approx = disk_blocks_to_byte_size(key.size_4k_units);
+ uint64_t dup_bytes_approx = calc_deduped_bytes(head_object_size,
+ key.num_parts,
+ byte_size_approx);
+ (*p_duplicate_bytes_approx) += (duplicate_count * dup_bytes_approx);
+ (*p_duplicate_count) += duplicate_count;
+ (*p_unique_count) ++;
+ }
+ }
+ }
+
+} // namespace rgw::dedup
+
+#if 0
+#include <climits>
+#include <cstdlib>
+#include <iostream>
+#include <cmath>
+#include <iomanip>
+#include <random>
+
+//---------------------------------------------------------------------------
+int main()
+{
+ static constexpr unsigned MAX_ENTRIES = 1024;
+ rgw::dedup::key_t *key_tab = new rgw::dedup::key_t[MAX_ENTRIES];
+ if (!key_tab) {
+ std::cerr << "faild alloc!" << std::endl;
+ return 1;
+ }
+ rgw::dedup::key_t *p_key = key_tab;
+ //rgw::dedup::dedup_table_t tab(MAX_ENTRIES + MAX_ENTRIES/5);
+ rgw::dedup::dedup_table_t tab(MAX_ENTRIES);
+
+ std::cout << "sizeof(key)=" << sizeof(rgw::dedup::key_t) << std::endl;
+ // Seed with a real random value, if available
+ std::random_device r;
+ // Choose a random mean between 1 ULLONG_MAX
+ std::default_random_engine e1(r());
+ std::uniform_int_distribution<uint64_t> uniform_dist(1, std::numeric_limits<uint64_t>::max());
+
+ for (unsigned i = 0; i < MAX_ENTRIES; i++) {
+ uint64_t md5_high = uniform_dist(e1);
+ uint64_t md5_low = uniform_dist(e1);
+ uint32_t size_4k_units = std::rand();
+ uint16_t num_parts = std::rand();
+ //std::cout << std::hex << md5_high << "::" << md5_low << "::" << block_id << std::endl;
+ rgw::dedup::key_t key(md5_high, md5_low, size_4k_units, num_parts);
+ *p_key = key;
+ p_key++;
+ }
+ work_shard_t work_shard = 3;
+ for (unsigned i = 0; i < MAX_ENTRIES; i++) {
+ disk_block_id_t block_id(worker_id, std::rand());
+ tab.add_entry(key_tab+i, block_id, 0, false, false);
+ }
+ double avg = (double)total / MAX_ENTRIES;
+ std::cout << "Insert::num entries=" << MAX_ENTRIES << ", total=" << total
+ << ", avg=" << avg << ", max=" << max << std::endl;
+ std::cout << "==========================================\n";
+
+ total = 0;
+ max = 0;
+ for (unsigned i = 0; i < MAX_ENTRIES; i++) {
+ tab.find_entry(key_tab+i);
+ }
+ avg = (double)total / MAX_ENTRIES;
+ std::cout << "Find::num entries=" << MAX_ENTRIES << ", total=" << total
+ << ", avg=" << avg << ", max=" << max << std::endl;
+ std::cout << "==========================================\n";
+ tab.remove_singletons_and_redistribute_keys();
+ tab.print_redistribute_stats();
+ tab.stat_counters_reset();
+ std::cout << "==========================================\n";
+ total = 0;
+ max = 0;
+ uint32_t cnt = 0;
+ for (unsigned i = 0; i < MAX_ENTRIES; i++) {
+ rgw::dedup::key_t *p_key = key_tab+i;
+ tab.find_entry(p_key);
+ cnt++;
+#if 0
+ if (p_key->md5_high % 5 == 0) {
+ tab.find_entry(p_key);
+ cnt++;
+ }
+#endif
+ }
+ avg = (double)total / cnt;
+ std::cout << "num entries=" << cnt << ", total=" << total
+ << ", avg=" << avg << ", max=" << max << std::endl;
+}
+#endif
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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.
+ *
+ */
+
+#pragma once
+#include <cstdint>
+#include <cstddef>
+#include <iterator>
+#include "common/dout.h"
+#include "rgw_dedup_store.h"
+namespace rgw::dedup {
+
+ // 24 Bytes key
+ struct key_t {
+ key_t() { ;}
+ key_t(uint64_t _md5_high,
+ uint64_t _md5_low,
+ uint32_t _size_4k_units,
+ uint16_t _num_parts,
+ uint8_t _stor_class_idx) {
+ md5_high = _md5_high;
+ md5_low = _md5_low;
+ size_4k_units = _size_4k_units;
+ num_parts = _num_parts;
+ stor_class_idx = _stor_class_idx;
+ pad8 = 0;
+ }
+
+ bool operator==(const struct key_t& other) const {
+ return (memcmp(this, &other, sizeof(other)) == 0);
+ }
+
+ bool operator!=(const struct key_t& other) const {
+ return !operator==(other);
+ }
+
+ uint64_t hash() const {
+ // The MD5 is already a hashing function so no need for another hash
+ return this->md5_low;
+ }
+
+ uint64_t md5_high; // High Bytes of the Object Data MD5
+ uint64_t md5_low; // Low Bytes of the Object Data MD5
+ uint32_t size_4k_units; // Object size in 4KB units max out at 16TB (AWS MAX-SIZE is 5TB)
+ uint16_t num_parts; // How many parts were used in multipart upload (AWS MAX-PART is 10,000)
+ uint8_t stor_class_idx;// storage class id
+ uint8_t pad8;
+ } __attribute__((__packed__));
+ static_assert(sizeof(key_t) == 24);
+
+ class dedup_table_t {
+ public:
+ // 8 Bytes Value
+ struct value_t {
+ value_t() {
+ this->block_idx = 0xFFFFFFFF;
+ this->count = 0;
+ this->rec_id = 0xFF;
+ this->flags.clear();
+ }
+
+ value_t(disk_block_id_t block_id, record_id_t rec_id, bool shared_manifest) {
+ this->block_idx = block_id;
+ this->count = 1;
+ this->rec_id = rec_id;
+ this->flags.clear();
+ this->flags.set_occupied();
+ if (shared_manifest) {
+ flags.set_shared_manifest();
+ }
+ }
+
+ inline void clear_flags() { flags.clear(); }
+ inline bool has_shared_manifest() const {return flags.has_shared_manifest(); }
+ inline void set_shared_manifest_src() { this->flags.set_shared_manifest(); }
+ inline bool is_singleton() const { return (count == 1); }
+ inline bool is_occupied() const { return flags.is_occupied(); }
+ inline void set_occupied() { this->flags.set_occupied(); }
+ inline void clear_occupied() { this->flags.clear_occupied(); }
+
+ disk_block_id_t block_idx; // 32 bits
+ uint16_t count; // 16 bits
+ record_id_t rec_id; // 8 bits
+ dedup_flags_t flags; // 8 bits
+ } __attribute__((__packed__));
+ static_assert(sizeof(value_t) == 8);
+
+ dedup_table_t(const DoutPrefixProvider* _dpp,
+ uint32_t _head_object_size,
+ uint8_t *p_slab,
+ uint64_t slab_size);
+ int add_entry(key_t *p_key, disk_block_id_t block_id, record_id_t rec_id,
+ bool shared_manifest);
+ void update_entry(key_t *p_key, disk_block_id_t block_id, record_id_t rec_id,
+ bool shared_manifest);
+
+ int get_val(const key_t *p_key, struct value_t *p_val /*OUT*/);
+
+ int set_shared_manifest_src_mode(const key_t *p_key,
+ disk_block_id_t block_id,
+ record_id_t rec_id);
+
+ void count_duplicates(uint64_t *p_singleton_count,
+ uint64_t *p_unique_count,
+ uint64_t *p_duplicate_count,
+ uint64_t *p_duplicate_bytes_approx);
+ void remove_singletons_and_redistribute_keys();
+ private:
+ // 32 Bytes unified entries
+ struct table_entry_t {
+ key_t key;
+ value_t val;
+ } __attribute__((__packed__));
+ static_assert(sizeof(table_entry_t) == 32);
+
+ uint32_t find_entry(const key_t *p_key) const;
+ uint32_t values_count = 0;
+ uint32_t entries_count = 0;
+ uint32_t occupied_count = 0;
+ uint32_t head_object_size = (4ULL * 1024 * 1024);
+ table_entry_t *hash_tab = nullptr;
+
+ // stat counters
+ uint64_t redistributed_count = 0;
+ uint64_t redistributed_search_total = 0;
+ uint64_t redistributed_search_max = 0;
+ uint64_t redistributed_loopback = 0;
+ uint64_t redistributed_perfect = 0;
+ uint64_t redistributed_clear = 0;
+ uint64_t redistributed_not_needed = 0;
+ const DoutPrefixProvider* dpp;
+ };
+
+} //namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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 "rgw_dedup_utils.h"
+#include "common/ceph_crypto.h"
+
+namespace rgw::dedup {
+ //---------------------------------------------------------------------------
+ std::ostream& operator<<(std::ostream &out, const dedup_req_type_t& dedup_type)
+ {
+ if (dedup_type == dedup_req_type_t::DEDUP_TYPE_NONE) {
+ out << "DEDUP_TYPE_NONE";
+ }
+ else if (dedup_type == dedup_req_type_t::DEDUP_TYPE_ESTIMATE) {
+ out << "DEDUP_TYPE_ESTIMATE";
+ }
+ else if (dedup_type == dedup_req_type_t::DEDUP_TYPE_FULL) {
+ out << "DEDUP_TYPE_FULL";
+ }
+ else {
+ out << "\n*** unexpected dedup_type ***\n";
+ }
+
+ return out;
+ }
+
+ // convert a hex-string to a 64bit integer (max 16 hex digits)
+ //---------------------------------------------------------------------------
+ bool hex2int(const char *p, const char *p_end, uint64_t *p_val)
+ {
+ if (p_end - p <= (int)(sizeof(uint64_t) * 2)) {
+ uint64_t val = 0;
+ while (p < p_end) {
+ // get current character then increment
+ uint8_t byte = *p++;
+ // transform hex character to the 4bit equivalent number, using the ASCII table indexes
+ if (byte >= '0' && byte <= '9') {
+ byte = byte - '0';
+ }
+ else if (byte >= 'a' && byte <='f') {
+ byte = byte - 'a' + 10;
+ }
+ else if (byte >= 'A' && byte <='F') {
+ byte = byte - 'A' + 10;
+ }
+ else {
+ // terminate on the first non hex char
+ return false;
+ }
+ // shift 4 to make space for new digit, and add the 4 bits of the new digit
+ val = (val << 4) | (byte & 0xF);
+ }
+ *p_val = val;
+ return true;
+ }
+ else {
+ return false;
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ bool dec2int(const char *p, const char* p_end, uint16_t *p_val)
+ {
+ uint16_t val = 0;
+ while (p < p_end) {
+ uint8_t byte = *p++;
+ if (byte >= '0' && byte <= '9') {
+ val = val * 10 + (byte - '0');
+ }
+ else {
+ // terminate on the first non hex char
+ return false;
+ }
+ }
+ *p_val = val;
+ return true;
+ }
+
+ // 16Bytes MD5 takes 32 chars
+ const unsigned MD5_LENGTH = 32;
+
+ //---------------------------------------------------------------------------
+ static bool get_num_parts(const std::string & etag, uint16_t *p_num_parts)
+ {
+ // Amazon S3 multipart upload Maximum number = 10,000
+ const unsigned MAX_PARTS = 10000;
+ if (etag.length() <= MD5_LENGTH) {
+ // i.e. no multipart
+ *p_num_parts = 0;
+ return true;
+ }
+
+ // Amazon S3 multipart upload Maximum number = 10,000 (5 decimal digits)
+ // We need 1 extra byte for the '-' delimiter and 1 extra byte for '"' at the end
+ // 7 Bytes should suffice, but we roundup to 8 Bytes
+ const unsigned MAX_PART_LEN = 8;
+ if (unlikely(etag.length() > MD5_LENGTH + MAX_PART_LEN)) {
+ // illegal ETAG
+ return false;
+ }
+
+ std::string::size_type n = etag.find('-', etag.length() - MAX_PART_LEN);
+ if (n != std::string::npos) {
+ char buff[MAX_PART_LEN];
+ // again, 1 extra byte for the '-' delimiter
+ unsigned copy_size = etag.length() - (n + 1);
+ if (copy_size <= MAX_PART_LEN) {
+ unsigned nbytes = etag.copy(buff, copy_size, n+1);
+ uint16_t num_parts;
+ const unsigned MAX_UINT16_DIGITS = 5; // 65536
+ if (nbytes <= MAX_UINT16_DIGITS) {
+ if (dec2int(buff, buff+nbytes, &num_parts) && num_parts <= MAX_PARTS) {
+ *p_num_parts = num_parts;
+ return true;
+ } // else, not all digits are legal
+ } // else, more than 5 digits
+ } // else, copy len too large
+ } // else, '-' delimiter was not found
+
+ // illegal number of parts
+ return false;
+ }
+
+ //---------------------------------------------------------------------------
+ bool parse_etag_string(const std::string& etag, parsed_etag_t *parsed_etag)
+ {
+ char buff[MD5_LENGTH*2];
+ uint16_t num_parts = 0;
+ if (get_num_parts(etag, &num_parts)) {
+ etag.copy(buff, MD5_LENGTH, 0);
+ uint64_t high, low;
+ if (hex2int(buff, buff+16, &high)) {
+ if (hex2int(buff+16, buff+32, &low)) {
+ parsed_etag->md5_high = high; // High Bytes of the Object Data MD5
+ parsed_etag->md5_low = low; // Low Bytes of the Object Data MD5
+ parsed_etag->num_parts = num_parts; // How many parts were used in multipart upload
+ return true;
+ }
+ }
+ }
+
+ // an illegal etag string
+ return false;
+ }
+
+ //---------------------------------------------------------------------------
+ void etag_to_bufferlist(uint64_t md5_high, uint64_t md5_low, uint16_t num_parts,
+ ceph::bufferlist *bl)
+ {
+ char buff[64];
+ int n = snprintf(buff, sizeof(buff), "%016lx%016lx", md5_high, md5_low);
+ if (num_parts >= 1) {
+ n += snprintf(buff + n, sizeof(buff) - n, "-%u", num_parts);
+ }
+ bl->append(buff, n);
+ }
+
+ //---------------------------------------------------------------------------
+ const char* get_next_data_ptr(bufferlist::const_iterator &bl_itr,
+ char data_buff[],
+ size_t len,
+ const DoutPrefixProvider* dpp)
+ {
+ const char *p = nullptr;
+ size_t n = bl_itr.get_ptr_and_advance(len, &p);
+ if (n == len) {
+ // we got a zero-copy raw pointer to contiguous data on the buffer-list
+ return p;
+ }
+
+ std::vector<int> vec;
+ // otherwise - copy the data to the @data_buff
+ char *p_buff = data_buff;
+ do {
+ vec.push_back(n);
+ std::memcpy(p_buff, p, n);
+ p_buff += n;
+ len -= n;
+ if (len > 0) {
+ n = bl_itr.get_ptr_and_advance(len, &p);
+ }
+ } while (len > 0);
+
+ ldpp_dout(dpp, 20) << __func__ << "::vec=" << vec << dendl;
+ return data_buff;
+ }
+
+ static const char* s_urgent_msg_names[] = {
+ "URGENT_MSG_NONE",
+ "URGENT_MSG_ABORT",
+ "URGENT_MSG_PASUE",
+ "URGENT_MSG_RESUME",
+ "URGENT_MSG_RESTART",
+ "URGENT_MSG_INVALID"
+ };
+
+ //---------------------------------------------------------------------------
+ const char* get_urgent_msg_names(int msg) {
+ if (msg <= URGENT_MSG_INVALID && msg >= URGENT_MSG_NONE) {
+ return s_urgent_msg_names[msg];
+ }
+ else {
+ return s_urgent_msg_names[URGENT_MSG_INVALID];
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ std::ostream& operator<<(std::ostream &out, const worker_stats_t &s)
+ {
+ JSONFormatter formatter(false);
+ s.dump(&formatter);
+ std::stringstream sstream;
+ formatter.flush(sstream);
+ out << sstream.str();
+ return out;
+ }
+
+ //---------------------------------------------------------------------------
+ void worker_stats_t::dump(Formatter *f) const
+ {
+ // main section
+ {
+ Formatter::ObjectSection notify(*f, "main");
+
+ f->dump_unsigned("Ingress Objs count", this->ingress_obj);
+ f->dump_unsigned("Accum byte size Ingress Objs", this->ingress_obj_bytes);
+ f->dump_unsigned("Egress Records count", this->egress_records);
+ f->dump_unsigned("Egress Blocks count", this->egress_blocks);
+ f->dump_unsigned("Egress Slabs count", this->egress_slabs);
+ f->dump_unsigned("Single part obj count", this->single_part_objs);
+ f->dump_unsigned("Multipart obj count", this->multipart_objs);
+ if (this->small_multipart_obj) {
+ f->dump_unsigned("Small Multipart obj count", this->small_multipart_obj);
+ }
+ }
+
+ {
+ Formatter::ObjectSection notify(*f, "notify");
+
+ if(this->non_default_storage_class_objs) {
+ f->dump_unsigned("non default storage class objs",
+ this->non_default_storage_class_objs);
+ f->dump_unsigned("non default storage class objs bytes",
+ this->non_default_storage_class_objs_bytes);
+ }
+ else {
+ ceph_assert(this->default_storage_class_objs == this->ingress_obj);
+ ceph_assert(this->default_storage_class_objs_bytes == this->ingress_obj_bytes);
+ }
+ }
+
+ {
+ Formatter::ObjectSection skipped(*f, "skipped");
+ if(this->ingress_skip_too_small) {
+ f->dump_unsigned("Ingress skip: too small objs",
+ this->ingress_skip_too_small);
+ f->dump_unsigned("Ingress skip: too small bytes",
+ this->ingress_skip_too_small_bytes);
+
+ if(this->ingress_skip_too_small_64KB) {
+ f->dump_unsigned("Ingress skip: 64KB<=size<=4MB Obj",
+ this->ingress_skip_too_small_64KB);
+ f->dump_unsigned("Ingress skip: 64KB<=size<=4MB Bytes",
+ this->ingress_skip_too_small_64KB_bytes);
+ }
+ }
+ }
+
+ {
+ Formatter::ObjectSection failed(*f, "failed");
+ if(this->ingress_corrupted_etag) {
+ f->dump_unsigned("Corrupted ETAG", this->ingress_corrupted_etag);
+ }
+ }
+ }
+
+ //---------------------------------------------------------------------------
+ std::ostream& operator<<(std::ostream &out, const md5_stats_t &s)
+ {
+ JSONFormatter formatter(false);
+ s.dump(&formatter);
+ std::stringstream sstream;
+ formatter.flush(sstream);
+ out << sstream.str();
+ return out;
+ }
+
+ //---------------------------------------------------------------------------
+ void md5_stats_t::dump(Formatter *f) const
+ {
+ // main section
+ {
+ Formatter::ObjectSection notify(*f, "main");
+
+ f->dump_unsigned("Total processed objects", this->processed_objects);
+ f->dump_unsigned("Loaded objects", this->loaded_objects);
+ f->dump_unsigned("Set Shared-Manifest SRC", this->set_shared_manifest_src);
+ f->dump_unsigned("Deduped Obj (this cycle)", this->deduped_objects);
+ f->dump_unsigned("Deduped Bytes(this cycle)", this->deduped_objects_bytes);
+ f->dump_unsigned("Already Deduped bytes (prev cycles)",
+ this->shared_manifest_dedup_bytes);
+ f->dump_unsigned("Singleton Obj", this->singleton_count);
+ f->dump_unsigned("Unique Obj", this->unique_count);
+ f->dump_unsigned("Duplicate Obj", this->duplicate_count);
+ f->dump_unsigned("Dedup Bytes Estimate", this->dedup_bytes_estimate);
+ }
+
+ {
+ Formatter::ObjectSection notify(*f, "notify");
+ if (this->failed_table_load) {
+ f->dump_unsigned("Failed Table Load", this->failed_table_load);
+ }
+ if (this->failed_map_overflow) {
+ f->dump_unsigned("Failed Remap Overflow", this->failed_map_overflow);
+ }
+
+ f->dump_unsigned("Valid SHA256 attrs", this->valid_sha256_attrs);
+ f->dump_unsigned("Invalid SHA256 attrs", this->invalid_sha256_attrs);
+
+ if (this->set_sha256_attrs) {
+ f->dump_unsigned("Set SHA256", this->set_sha256_attrs);
+ }
+
+ if (this->skip_sha256_cmp) {
+ f->dump_unsigned("Can't run SHA256 compare", this->skip_sha256_cmp);
+ }
+ }
+
+ {
+ Formatter::ObjectSection skipped(*f, "skipped");
+ f->dump_unsigned("Skipped shared_manifest", this->skipped_shared_manifest);
+ f->dump_unsigned("Skipped singleton objs", this->skipped_singleton);
+ if (this->skipped_singleton) {
+ f->dump_unsigned("Skipped singleton Bytes", this->skipped_singleton_bytes);
+ }
+ f->dump_unsigned("Skipped source record", this->skipped_source_record);
+
+ if (this->ingress_skip_encrypted) {
+ f->dump_unsigned("Skipped Encrypted objs", this->ingress_skip_encrypted);
+ f->dump_unsigned("Skipped Encrypted Bytes",this->ingress_skip_encrypted_bytes);
+ }
+ if (this->ingress_skip_compressed) {
+ f->dump_unsigned("Skipped Compressed objs", this->ingress_skip_compressed);
+ f->dump_unsigned("Skipped Compressed Bytes", this->ingress_skip_compressed_bytes);
+ }
+ if (this->ingress_skip_changed_objs) {
+ f->dump_unsigned("Skipped Changed Object", this->ingress_skip_changed_objs);
+ }
+ }
+
+ {
+ Formatter::ObjectSection sys_failures(*f, "system failures");
+ if (this->ingress_failed_load_bucket) {
+ f->dump_unsigned("Failed load_bucket()", this->ingress_failed_load_bucket);
+ }
+ if (this->ingress_failed_get_object) {
+ f->dump_unsigned("Failed get_object()", this->ingress_failed_get_object);
+ }
+ if (this->ingress_failed_get_obj_attrs) {
+ f->dump_unsigned("Failed get_obj_attrs", this->ingress_failed_get_obj_attrs);
+ }
+ if (this->ingress_corrupted_etag) {
+ f->dump_unsigned("Corrupted ETAG", this->ingress_corrupted_etag);
+ }
+ if (this->ingress_corrupted_obj_attrs) {
+ f->dump_unsigned("Corrupted obj attributes", this->ingress_corrupted_obj_attrs);
+ }
+ if (this->failed_src_load) {
+ f->dump_unsigned("Failed SRC-Load ", this->failed_src_load);
+ }
+ if (this->failed_rec_load) {
+ f->dump_unsigned("Failed Record-Load ", this->failed_rec_load);
+ }
+ if (this->failed_block_load) {
+ f->dump_unsigned("Failed Block-Load ", this->failed_block_load);
+ }
+ if (this->failed_dedup) {
+ f->dump_unsigned("Failed Dedup", this->failed_dedup);
+ }
+ }
+
+ {
+ Formatter::ObjectSection logical_failures(*f, "logical failures");
+ if (this->sha256_mismatch) {
+ f->dump_unsigned("SHA256 mismatch", this->sha256_mismatch);
+ }
+ if (this->duplicate_records) {
+ f->dump_unsigned("Duplicate SRC/TGT", this->duplicate_records);
+ }
+ if (this->size_mismatch) {
+ f->dump_unsigned("Size mismatch SRC/TGT", this->size_mismatch);
+ }
+ }
+ }
+} //namespace rgw::dedup
--- /dev/null
+// -*- mode:C++; tab-width:8; c-basic-offset:2;
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Author: Gabriel BenHanokh <gbenhano@redhat.com>
+ *
+ * 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.
+ *
+ */
+
+#pragma once
+#include <string>
+#include "include/rados/buffer.h"
+#include "include/encoding.h"
+#include "common/Formatter.h"
+#include "common/ceph_json.h"
+#include <time.h>
+#include "include/utime.h"
+#include "include/encoding.h"
+#include "common/dout.h"
+
+//#define FULL_DEDUP_SUPPORT
+namespace rgw::dedup {
+ static constexpr const char* DEDUP_WATCH_OBJ = "DEDUP_WATCH_OBJ";
+ using work_shard_t = uint16_t;
+ using md5_shard_t = uint16_t;
+
+ // settings to help debug small systems
+ const work_shard_t MIN_WORK_SHARD = 2;
+ const md5_shard_t MIN_MD5_SHARD = 4;
+
+ // Those are the correct values for production system
+ const work_shard_t MAX_WORK_SHARD = 255;
+ const md5_shard_t MAX_MD5_SHARD = 512;
+
+ const work_shard_t NULL_WORK_SHARD = 0xFFFF;
+ const md5_shard_t NULL_MD5_SHARD = 0xFFFF;
+ const unsigned NULL_SHARD = 0xFFFF;
+
+ // work_shard is an 8 bits int with 255 legal values for the first iteration
+ // and one value (0xFF) reserved for second iteration
+ const unsigned WORK_SHARD_HARD_LIMIT = 0x0FF;
+ // md5_shard_t is a 12 bits int with 4096 possible values
+ const unsigned MD5_SHARD_HARD_LIMIT = 0xFFF;
+
+ static_assert(MAX_WORK_SHARD < NULL_WORK_SHARD);
+ static_assert(MAX_WORK_SHARD < NULL_SHARD);
+ static_assert(MAX_WORK_SHARD <= WORK_SHARD_HARD_LIMIT);
+ static_assert(MAX_MD5_SHARD < NULL_MD5_SHARD);
+ static_assert(MAX_MD5_SHARD < NULL_SHARD);
+ static_assert(MAX_MD5_SHARD <= MD5_SHARD_HARD_LIMIT);
+
+ //---------------------------------------------------------------------------
+ enum dedup_req_type_t {
+ DEDUP_TYPE_NONE = 0,
+ DEDUP_TYPE_ESTIMATE = 1,
+ DEDUP_TYPE_FULL = 2
+ };
+
+ std::ostream& operator<<(std::ostream &out, const dedup_req_type_t& dedup_type);
+ struct __attribute__ ((packed)) dedup_flags_t {
+ private:
+ static constexpr uint8_t RGW_DEDUP_FLAG_SHA256_CALCULATED = 0x01; // REC
+ static constexpr uint8_t RGW_DEDUP_FLAG_SHARED_MANIFEST = 0x02; // REC + TAB
+ static constexpr uint8_t RGW_DEDUP_FLAG_OCCUPIED = 0x04; // TAB
+ static constexpr uint8_t RGW_DEDUP_FLAG_FASTLANE = 0x08; // REC
+
+ public:
+ dedup_flags_t() : flags(0) {}
+ dedup_flags_t(uint8_t _flags) : flags(_flags) {}
+ inline void clear() { this->flags = 0; }
+ inline bool sha256_calculated() const { return ((flags & RGW_DEDUP_FLAG_SHA256_CALCULATED) != 0); }
+ inline void set_sha256_calculated() { flags |= RGW_DEDUP_FLAG_SHA256_CALCULATED; }
+ inline bool has_shared_manifest() const { return ((flags & RGW_DEDUP_FLAG_SHARED_MANIFEST) != 0); }
+ inline void set_shared_manifest() { flags |= RGW_DEDUP_FLAG_SHARED_MANIFEST; }
+ inline bool is_occupied() const {return ((this->flags & RGW_DEDUP_FLAG_OCCUPIED) != 0); }
+ inline void set_occupied() {this->flags |= RGW_DEDUP_FLAG_OCCUPIED; }
+ inline void clear_occupied() { this->flags &= ~RGW_DEDUP_FLAG_OCCUPIED; }
+ inline bool is_fastlane() const { return ((flags & RGW_DEDUP_FLAG_FASTLANE) != 0); }
+ inline void set_fastlane() { flags |= RGW_DEDUP_FLAG_FASTLANE; }
+ private:
+ uint8_t flags;
+ };
+
+ struct worker_stats_t {
+ worker_stats_t& operator +=(const worker_stats_t& other) {
+ this->ingress_obj += other.ingress_obj;
+ this->ingress_obj_bytes += other.ingress_obj_bytes;
+ this->egress_records += other.egress_records;
+ this->egress_blocks += other.egress_blocks;
+ this->egress_slabs += other.egress_slabs;
+ this->single_part_objs += other.single_part_objs;
+ this->multipart_objs += other.multipart_objs;
+ this->small_multipart_obj += other.small_multipart_obj;
+ this->default_storage_class_objs += other.default_storage_class_objs;
+ this->default_storage_class_objs_bytes += other.default_storage_class_objs_bytes;
+ this->non_default_storage_class_objs += other.non_default_storage_class_objs;
+ this->non_default_storage_class_objs_bytes += other.non_default_storage_class_objs_bytes;
+ this->ingress_corrupted_etag += other.ingress_corrupted_etag;
+ this->ingress_skip_too_small_bytes += other.ingress_skip_too_small_bytes;
+ this->ingress_skip_too_small += other.ingress_skip_too_small;
+ this->ingress_skip_too_small_64KB_bytes += other.ingress_skip_too_small_64KB_bytes;
+ this->ingress_skip_too_small_64KB += other.ingress_skip_too_small_64KB;
+
+ return *this;
+ }
+
+ void dump(Formatter *f) const;
+
+ uint64_t ingress_obj = 0;
+ uint64_t ingress_obj_bytes = 0;
+ uint64_t egress_records = 0;
+ uint64_t egress_blocks = 0;
+ uint64_t egress_slabs = 0;
+
+ uint64_t single_part_objs = 0;
+ uint64_t multipart_objs = 0;
+ uint64_t small_multipart_obj = 0;
+
+ uint64_t default_storage_class_objs = 0;
+ uint64_t default_storage_class_objs_bytes = 0;
+
+ uint64_t non_default_storage_class_objs = 0;
+ uint64_t non_default_storage_class_objs_bytes = 0;
+
+ uint64_t ingress_corrupted_etag = 0;
+
+ uint64_t ingress_skip_too_small_bytes = 0;
+ uint64_t ingress_skip_too_small = 0;
+
+ uint64_t ingress_skip_too_small_64KB_bytes = 0;
+ uint64_t ingress_skip_too_small_64KB = 0;
+
+ utime_t duration = {0, 0};
+ };
+ std::ostream& operator<<(std::ostream &out, const worker_stats_t &s);
+
+ inline void encode(const worker_stats_t& w, ceph::bufferlist& bl)
+ {
+ ENCODE_START(1, 1, bl);
+ encode(w.ingress_obj, bl);
+ encode(w.ingress_obj_bytes, bl);
+ encode(w.egress_records, bl);
+ encode(w.egress_blocks, bl);
+ encode(w.egress_slabs, bl);
+
+ encode(w.single_part_objs, bl);
+ encode(w.multipart_objs, bl);
+ encode(w.small_multipart_obj, bl);
+
+ encode(w.default_storage_class_objs, bl);
+ encode(w.default_storage_class_objs_bytes, bl);
+ encode(w.non_default_storage_class_objs, bl);
+ encode(w.non_default_storage_class_objs_bytes, bl);
+
+ encode(w.ingress_corrupted_etag, bl);
+
+ encode(w.ingress_skip_too_small_bytes, bl);
+ encode(w.ingress_skip_too_small, bl);
+
+ encode(w.ingress_skip_too_small_64KB_bytes, bl);
+ encode(w.ingress_skip_too_small_64KB, bl);
+
+ encode(w.duration, bl);
+ ENCODE_FINISH(bl);
+ }
+
+ inline void decode(worker_stats_t& w, ceph::bufferlist::const_iterator& bl)
+ {
+ DECODE_START(1, bl);
+ decode(w.ingress_obj, bl);
+ decode(w.ingress_obj_bytes, bl);
+ decode(w.egress_records, bl);
+ decode(w.egress_blocks, bl);
+ decode(w.egress_slabs, bl);
+ decode(w.single_part_objs, bl);
+ decode(w.multipart_objs, bl);
+ decode(w.small_multipart_obj, bl);
+ decode(w.default_storage_class_objs, bl);
+ decode(w.default_storage_class_objs_bytes, bl);
+ decode(w.non_default_storage_class_objs, bl);
+ decode(w.non_default_storage_class_objs_bytes, bl);
+ decode(w.ingress_corrupted_etag, bl);
+ decode(w.ingress_skip_too_small_bytes, bl);
+ decode(w.ingress_skip_too_small, bl);
+ decode(w.ingress_skip_too_small_64KB_bytes, bl);
+ decode(w.ingress_skip_too_small_64KB, bl);
+
+ decode(w.duration, bl);
+ DECODE_FINISH(bl);
+ }
+
+ struct md5_stats_t {
+ md5_stats_t& operator +=(const md5_stats_t& other) {
+ this->ingress_failed_load_bucket += other.ingress_failed_load_bucket;
+ this->ingress_failed_get_object += other.ingress_failed_get_object;
+ this->ingress_failed_get_obj_attrs += other.ingress_failed_get_obj_attrs;
+ this->ingress_corrupted_etag += other.ingress_corrupted_etag;
+ this->ingress_corrupted_obj_attrs += other.ingress_corrupted_obj_attrs;
+ this->ingress_skip_encrypted += other.ingress_skip_encrypted;
+ this->ingress_skip_encrypted_bytes += other.ingress_skip_encrypted_bytes;
+ this->ingress_skip_compressed += other.ingress_skip_compressed;
+ this->ingress_skip_compressed_bytes += other.ingress_skip_compressed_bytes;
+ this->ingress_skip_changed_objs += other.ingress_skip_changed_objs;
+ this->shared_manifest_dedup_bytes += other.shared_manifest_dedup_bytes;
+
+ this->skipped_shared_manifest += other.skipped_shared_manifest;
+ this->skipped_singleton += other.skipped_singleton;
+ this->skipped_singleton_bytes += other.skipped_singleton_bytes;
+ this->skipped_source_record += other.skipped_source_record;
+ this->duplicate_records += other.duplicate_records;
+ this->size_mismatch += other.size_mismatch;
+ this->sha256_mismatch += other.sha256_mismatch;
+ this->failed_src_load += other.failed_src_load;
+ this->failed_rec_load += other.failed_rec_load;
+ this->failed_block_load += other.failed_block_load;
+
+ this->valid_sha256_attrs += other.valid_sha256_attrs;
+ this->invalid_sha256_attrs += other.invalid_sha256_attrs;
+ this->set_sha256_attrs += other.set_sha256_attrs;
+ this->skip_sha256_cmp += other.skip_sha256_cmp;
+
+ this->set_shared_manifest_src += other.set_shared_manifest_src;
+ this->loaded_objects += other.loaded_objects;
+ this->processed_objects += other.processed_objects;
+ this->singleton_count += other.singleton_count;
+ this->duplicate_count += other.duplicate_count;
+ this->dedup_bytes_estimate += other.dedup_bytes_estimate;
+ this->unique_count += other.unique_count;
+ this->deduped_objects += other.deduped_objects;
+ this->deduped_objects_bytes += other.deduped_objects_bytes;
+
+ this->failed_dedup += other.failed_dedup;
+ this->failed_table_load += other.failed_table_load;
+ this->failed_map_overflow += other.failed_map_overflow;
+ return *this;
+ }
+ void dump(Formatter *f) const;
+
+ uint64_t ingress_failed_load_bucket = 0;
+ uint64_t ingress_failed_get_object = 0;
+ uint64_t ingress_failed_get_obj_attrs = 0;
+ uint64_t ingress_corrupted_etag = 0;
+ uint64_t ingress_corrupted_obj_attrs = 0;
+ uint64_t ingress_skip_encrypted = 0;
+ uint64_t ingress_skip_encrypted_bytes = 0;
+ uint64_t ingress_skip_compressed = 0;
+ uint64_t ingress_skip_compressed_bytes = 0;
+ uint64_t ingress_skip_changed_objs = 0;
+
+ uint64_t shared_manifest_dedup_bytes = 0;
+ uint64_t skipped_shared_manifest = 0;
+ uint64_t skipped_singleton = 0;
+ uint64_t skipped_singleton_bytes = 0;
+ uint64_t skipped_source_record = 0;
+ uint64_t duplicate_records = 0;
+ uint64_t size_mismatch = 0;
+ uint64_t sha256_mismatch = 0;
+ uint64_t failed_src_load = 0;
+ uint64_t failed_rec_load = 0;
+ uint64_t failed_block_load = 0;
+
+ uint64_t valid_sha256_attrs = 0;
+ uint64_t invalid_sha256_attrs = 0;
+ uint64_t set_sha256_attrs = 0;
+ uint64_t skip_sha256_cmp = 0;
+
+ uint64_t set_shared_manifest_src = 0;
+ uint64_t loaded_objects = 0;
+ uint64_t processed_objects = 0;
+ uint64_t singleton_count = 0;
+ uint64_t duplicate_count = 0;
+ // counter is using on-disk size affected by block-size
+ uint64_t dedup_bytes_estimate = 0;
+ uint64_t unique_count = 0;
+ uint64_t deduped_objects = 0;
+ // counter is using s3 byte size disregarding the on-disk size affected by block-size
+ uint64_t deduped_objects_bytes = 0;
+ uint64_t failed_dedup = 0;
+ uint64_t failed_table_load = 0;
+ uint64_t failed_map_overflow = 0;
+ utime_t duration = {0, 0};
+ };
+ std::ostream &operator<<(std::ostream &out, const md5_stats_t &s);
+ inline void encode(const md5_stats_t& m, ceph::bufferlist& bl)
+ {
+ ENCODE_START(1, 1, bl);
+
+ encode(m.ingress_failed_load_bucket, bl);
+ encode(m.ingress_failed_get_object, bl);
+ encode(m.ingress_failed_get_obj_attrs, bl);
+ encode(m.ingress_corrupted_etag, bl);
+ encode(m.ingress_corrupted_obj_attrs, bl);
+ encode(m.ingress_skip_encrypted, bl);
+ encode(m.ingress_skip_encrypted_bytes, bl);
+ encode(m.ingress_skip_compressed, bl);
+ encode(m.ingress_skip_compressed_bytes, bl);
+ encode(m.ingress_skip_changed_objs, bl);
+ encode(m.shared_manifest_dedup_bytes, bl);
+
+ encode(m.skipped_shared_manifest, bl);
+ encode(m.skipped_singleton, bl);
+ encode(m.skipped_singleton_bytes, bl);
+ encode(m.skipped_source_record, bl);
+ encode(m.duplicate_records, bl);
+ encode(m.size_mismatch, bl);
+ encode(m.sha256_mismatch, bl);
+ encode(m.failed_src_load, bl);
+ encode(m.failed_rec_load, bl);
+ encode(m.failed_block_load, bl);
+
+ encode(m.valid_sha256_attrs, bl);
+ encode(m.invalid_sha256_attrs, bl);
+ encode(m.set_sha256_attrs, bl);
+ encode(m.skip_sha256_cmp, bl);
+ encode(m.set_shared_manifest_src, bl);
+
+ encode(m.loaded_objects, bl);
+ encode(m.processed_objects, bl);
+ encode(m.singleton_count, bl);
+ encode(m.duplicate_count, bl);
+ encode(m.dedup_bytes_estimate, bl);
+ encode(m.unique_count, bl);
+ encode(m.deduped_objects, bl);
+ encode(m.deduped_objects_bytes, bl);
+ encode(m.failed_dedup, bl);
+ encode(m.failed_table_load, bl);
+ encode(m.failed_map_overflow, bl);
+
+ encode(m.duration, bl);
+ ENCODE_FINISH(bl);
+ }
+
+ inline void decode(md5_stats_t& m, ceph::bufferlist::const_iterator& bl)
+ {
+ DECODE_START(1, bl);
+ decode(m.ingress_failed_load_bucket, bl);
+ decode(m.ingress_failed_get_object, bl);
+ decode(m.ingress_failed_get_obj_attrs, bl);
+ decode(m.ingress_corrupted_etag, bl);
+ decode(m.ingress_corrupted_obj_attrs, bl);
+ decode(m.ingress_skip_encrypted, bl);
+ decode(m.ingress_skip_encrypted_bytes, bl);
+ decode(m.ingress_skip_compressed, bl);
+ decode(m.ingress_skip_compressed_bytes, bl);
+ decode(m.ingress_skip_changed_objs, bl);
+ decode(m.shared_manifest_dedup_bytes, bl);
+
+ decode(m.skipped_shared_manifest, bl);
+ decode(m.skipped_singleton, bl);
+ decode(m.skipped_singleton_bytes, bl);
+ decode(m.skipped_source_record, bl);
+ decode(m.duplicate_records, bl);
+ decode(m.size_mismatch, bl);
+ decode(m.sha256_mismatch, bl);
+ decode(m.failed_src_load, bl);
+ decode(m.failed_rec_load, bl);
+ decode(m.failed_block_load, bl);
+
+ decode(m.valid_sha256_attrs, bl);
+ decode(m.invalid_sha256_attrs, bl);
+ decode(m.set_sha256_attrs, bl);
+ decode(m.skip_sha256_cmp, bl);
+ decode(m.set_shared_manifest_src, bl);
+
+ decode(m.loaded_objects, bl);
+ decode(m.processed_objects, bl);
+ decode(m.singleton_count, bl);
+ decode(m.duplicate_count, bl);
+ decode(m.dedup_bytes_estimate, bl);
+ decode(m.unique_count, bl);
+ decode(m.deduped_objects, bl);
+ decode(m.deduped_objects_bytes, bl);
+ decode(m.failed_dedup, bl);
+ decode(m.failed_table_load, bl);
+ decode(m.failed_map_overflow, bl);
+
+ decode(m.duration, bl);
+ DECODE_FINISH(bl);
+ }
+
+ struct parsed_etag_t {
+ uint64_t md5_high; // High Bytes of the Object Data MD5
+ uint64_t md5_low; // Low Bytes of the Object Data MD5
+ uint16_t num_parts; // How many parts were used in multipart upload
+ // Setting num_parts to zero when multipart is not used
+ };
+
+#define DIV_UP(a, b) ( ((a)+(b-1)) / b)
+ // CEPH min allocation unit on disk is 4KB
+ // TBD: take from config
+ static constexpr uint64_t DISK_ALLOC_SIZE = 4*1024;
+ // 16 bytes hexstring -> 8 Byte uint64_t
+ static inline constexpr unsigned HEX_UNIT_SIZE = 16;
+
+ //---------------------------------------------------------------------------
+ static inline uint64_t byte_size_to_disk_blocks(uint64_t byte_size) {
+ return DIV_UP(byte_size, DISK_ALLOC_SIZE);
+ }
+
+ //---------------------------------------------------------------------------
+ static inline uint64_t disk_blocks_to_byte_size(uint64_t disk_blocks) {
+ return disk_blocks * DISK_ALLOC_SIZE;
+ }
+
+ //---------------------------------------------------------------------------
+ // ceph store full blocks so need to round up and multiply by block_size
+ static inline uint64_t calc_on_disk_byte_size(uint64_t byte_size) {
+ uint64_t size_4k_units = byte_size_to_disk_blocks(byte_size);
+ return disk_blocks_to_byte_size(size_4k_units);
+ }
+
+ enum urgent_msg_t {
+ URGENT_MSG_NONE = 0,
+ URGENT_MSG_ABORT = 1,
+ URGENT_MSG_PASUE = 2,
+ URGENT_MSG_RESUME = 3,
+ URGENT_MSG_RESTART = 4,
+ URGENT_MSG_INVALID = 5
+ };
+
+ const char* get_urgent_msg_names(int msg);
+ bool hex2int(const char *p, const char *p_end, uint64_t *p_val);
+ bool parse_etag_string(const std::string& etag, parsed_etag_t *parsed_etag);
+ void etag_to_bufferlist(uint64_t md5_high, uint64_t md5_low, uint16_t num_parts,
+ ceph::bufferlist *bl);
+ const char* get_next_data_ptr(bufferlist::const_iterator &bl_itr,
+ char data_buff[],
+ size_t len,
+ const DoutPrefixProvider* dpp);
+
+ //---------------------------------------------------------------------------
+ static inline void build_oid(const std::string &bucket_id,
+ const std::string &obj_name,
+ std::string *oid)
+ {
+ *oid = bucket_id + "_" + obj_name;
+ }
+
+ //---------------------------------------------------------------------------
+ static inline uint64_t calc_deduped_bytes(uint64_t head_obj_size,
+ uint16_t num_parts,
+ uint64_t size_bytes)
+ {
+ if (num_parts > 0) {
+ // multipart objects with an empty head i.e. we achive full dedup
+ return size_bytes;
+ }
+ else {
+ // reduce the head size
+ if (size_bytes > head_obj_size) {
+ return size_bytes - head_obj_size;
+ }
+ else {
+ return 0;
+ }
+ }
+ }
+
+} //namespace rgw::dedup
}
main.init_lua();
+ main.init_dedup();
return 0;
} /* RGWLib::init() */
main.init_opslog();
main.init_tracepoints();
main.init_lua();
+ main.init_dedup();
r = main.init_frontends2(nullptr /* RGWLib */);
if (r != 0) {
derr << "ERROR: initialize frontend fail, r = " << r << dendl;
#include "rgw_realm_reloader.h"
#include "rgw_ldap.h"
#include "rgw_lua.h"
+#include "rgw_dedup.h"
#include "rgw_dmclock_scheduler_ctx.h"
#include "rgw_ratelimit.h"
namespace rgw {
namespace lua { class Background; }
+namespace dedup{ class Background; }
namespace sal { class ConfigStore; }
class RGWLib;
std::unique_ptr<rgw::LDAPHelper> ldh;
RGWREST rest;
std::unique_ptr<rgw::lua::Background> lua_background;
+ std::unique_ptr<rgw::dedup::Background> dedup_background;
std::unique_ptr<rgw::auth::ImplicitTenants> implicit_tenant_context;
std::unique_ptr<rgw::dmclock::SchedulerCtx> sched_ctx;
std::unique_ptr<ActiveRateLimiter> ratelimiter;
int init_frontends2(RGWLib* rgwlib = nullptr);
void init_tracepoints();
void init_lua();
+ void init_dedup();
bool have_http() {
return have_http_frontend;
namespace rgw::lua {
class Background;
}
+namespace rgw::dedup {
+ class Background;
+}
namespace rgw::sal {
class ConfigStore;
class Driver;
user policy list attached list attached managed policies
caps add add user capabilities
caps rm remove user capabilities
+ dedup stats Display dedup statistics from the last run
+ dedup estimate Runs dedup in estimate mode (no changes will be made)
+ dedup restart Restart dedup
+ dedup abort Abort dedup
+ dedup pause Pause dedup
+ dedup resume Resume paused dedup
subuser create create a new subuser
subuser modify modify subuser
subuser rm remove subuser
--- /dev/null
+===================
+Dedup Scripting Tests
+===================
+
+* Start the cluster using the `vstart.sh` script
+* Run the test from within the `src/test/rgw/dedup` directory:
+ `DEDUPTESTS_CONF=deduptests.conf.SAMPLE tox`
+
--- /dev/null
+import configparser
+import os
+import pytest
+
+def setup():
+ cfg = configparser.RawConfigParser()
+ try:
+ path = os.environ['DEDUPTESTS_CONF']
+ except KeyError:
+ raise RuntimeError(
+ 'To run tests, point environment '
+ + 'variable DEDUPTESTS_CONF to a config file.',
+ )
+ cfg.read(path)
+
+ if not cfg.defaults():
+ raise RuntimeError('Your config file is missing the DEFAULT section!')
+ if not cfg.has_section("s3 main"):
+ raise RuntimeError('Your config file is missing the "s3 main" section!')
+
+ defaults = cfg.defaults()
+
+ # vars from the DEFAULT section
+ global default_host
+ default_host = defaults.get("host")
+
+ global default_port
+ default_port = int(defaults.get("port"))
+ # vars from the main section
+ global main_access_key
+ main_access_key = cfg.get('s3 main',"access_key")
+
+ global main_secret_key
+ main_secret_key = cfg.get('s3 main',"secret_key")
+
+
+def get_config_host():
+ global default_host
+ return default_host
+
+
+def get_config_port():
+ global default_port
+ return default_port
+
+
+def get_access_key():
+ global main_access_key
+ return main_access_key
+
+
+def get_secret_key():
+ global main_secret_key
+ return main_secret_key
+
+
+@pytest.fixture(autouse=True, scope="package")
+def configfile():
+ setup()
+
--- /dev/null
+[DEFAULT]
+port = 8000
+host = localhost
+
+[s3 main]
+access_key = 0555b35654ad1656d804
+secret_key = h7GhxuBLTrlhVUyxSPUKUV8r/2EI4ngqJxD7iBdBYLhwluN30JaT3Q==
+display_name = M. Tester
+user_id = testid
+email = tester@ceph.com
--- /dev/null
+[pytest]
+markers =
+ basic_test
+
+log_cli=true
+log_cli_level=WARNING
+#log_cli_level=INFO
--- /dev/null
+boto3 >=1.0.0
+pytest
--- /dev/null
+import threading
+import logging
+import random
+import math
+import time
+import subprocess
+import urllib.request
+import hashlib
+from multiprocessing import Process
+import os
+import string
+import shutil
+import pytest
+import json
+from collections import namedtuple
+import boto3
+from boto3.s3.transfer import TransferConfig
+from dataclasses import dataclass
+
+from . import(
+ configfile,
+ get_config_host,
+ get_config_port,
+ get_access_key,
+ get_secret_key
+)
+
+@dataclass
+class Dedup_Stats:
+ skip_too_small: int = 0
+ skip_too_small_bytes: int = 0
+ skip_shared_manifest: int = 0
+ skip_singleton: int = 0
+ skip_singleton_bytes: int = 0
+ skip_src_record: int = 0
+ skip_changed_object: int = 0
+ corrupted_etag: int = 0
+ sha256_mismatch: int = 0
+ valid_sha256: int = 0
+ invalid_sha256: int = 0
+ set_sha256: int = 0
+ total_processed_objects: int = 0
+ size_before_dedup: int = 0
+ loaded_objects: int = 0
+ set_shared_manifest_src : int = 0
+ deduped_obj: int = 0
+ singleton_obj : int = 0
+ unique_obj : int = 0
+ dedup_bytes_estimate : int = 0
+ duplicate_obj : int = 0
+ deduped_obj_bytes : int = 0
+ non_default_storage_class_objs_bytes : int = 0
+
+@dataclass
+class Dedup_Ratio:
+ s3_bytes_before: int = 0
+ s3_bytes_after: int = 0
+ ratio: float = 0.0
+
+full_dedup_state_was_checked = False
+full_dedup_state_disabled = True
+
+# configure logging for the tests module
+log = logging.getLogger(__name__)
+num_buckets = 0
+num_files = 0
+num_users = 0
+num_conns = 0
+run_prefix=''.join(random.choice(string.ascii_lowercase) for _ in range(16))
+test_path = os.path.normpath(os.path.dirname(os.path.realpath(__file__))) + '/../'
+
+#-----------------------------------------------
+def bash(cmd, **kwargs):
+ #log.info('running command: %s', ' '.join(cmd))
+ kwargs['stdout'] = subprocess.PIPE
+ process = subprocess.Popen(cmd, **kwargs)
+ s = process.communicate()[0].decode('utf-8')
+ return (s, process.returncode)
+
+#-----------------------------------------------
+def admin(args, **kwargs):
+ """ radosgw-admin command """
+ cmd = [test_path + 'test-rgw-call.sh', 'call_rgw_admin', 'noname'] + args
+ return bash(cmd, **kwargs)
+
+#-----------------------------------------------
+def rados(args, **kwargs):
+ """ rados command """
+ cmd = [test_path + 'test-rgw-call.sh', 'call_rgw_rados', 'noname'] + args
+ return bash(cmd, **kwargs)
+
+#-----------------------------------------------
+def gen_bucket_name():
+ global num_buckets
+
+ num_buckets += 1
+ bucket_name = run_prefix + '-' + str(num_buckets)
+ log.info("bucket_name=%s", bucket_name);
+ return bucket_name
+
+#-----------------------------------------------
+def get_buckets(num_buckets):
+ bucket_names=[]
+ for i in range(num_buckets):
+ bucket_name=gen_bucket_name()
+ bucket_names.append(bucket_name)
+
+ return bucket_names
+
+
+#==============================================
+g_tenant_connections=[]
+g_tenants=[]
+g_simple_connection=[]
+
+#-----------------------------------------------
+def close_all_connections():
+ global g_simple_connection
+
+ for conn in g_simple_connection:
+ log.info("close simple connection")
+ conn.close()
+
+ for conn in g_tenant_connections:
+ log.info("close tenant connection")
+ conn.close()
+
+#-----------------------------------------------
+def get_connections(req_count):
+ global g_simple_connection
+ conns=[]
+
+ for i in range(min(req_count, len(g_simple_connection))):
+ log.info("recycle existing connection")
+ conns.append(g_simple_connection[i])
+
+ if len(conns) < req_count:
+ hostname = get_config_host()
+ port_no = get_config_port()
+ access_key = get_access_key()
+ secret_key = get_secret_key()
+ if port_no == 443 or port_no == 8443:
+ scheme = 'https://'
+ else:
+ scheme = 'http://'
+
+ for i in range(req_count - len(conns)):
+ log.info("generate new connection")
+ client = boto3.client('s3',
+ endpoint_url=scheme+hostname+':'+str(port_no),
+ aws_access_key_id=access_key,
+ aws_secret_access_key=secret_key)
+ g_simple_connection.append(client);
+ conns.append(client);
+
+ return conns
+
+#-----------------------------------------------
+def get_single_connection():
+ conns=get_connections(1)
+ return conns[0]
+
+
+#-----------------------------------------------
+def another_user(uid, tenant, display_name):
+ global num_users
+ num_users += 1
+
+ access_key = run_prefix + "_" +str(num_users) + "_" + str(time.time())
+ secret_key = run_prefix + "_" +str(num_users) + "_" + str(time.time())
+
+ cmd = ['user', 'create', '--uid', uid, '--tenant', tenant, '--access-key', access_key, '--secret-key', secret_key, '--display-name', display_name]
+ result = admin(cmd)
+ assert result[1] == 0
+
+ hostname = get_config_host()
+ port_no = get_config_port()
+ if port_no == 443 or port_no == 8443:
+ scheme = 'https://'
+ else:
+ scheme = 'http://'
+
+ endpoint=scheme+hostname+':'+str(port_no)
+ client = boto3.client('s3',
+ endpoint_url=endpoint,
+ aws_access_key_id=access_key,
+ aws_secret_access_key=secret_key)
+
+ return client
+
+#-------------------------------------------------------------------------------
+def gen_connections_multi2(req_count):
+ g_tenant_connections=[]
+ g_tenants=[]
+ global num_conns
+
+ log.info("gen_connections_multi: Create connection and buckets ...")
+ suffix=run_prefix
+
+ tenants=[]
+ bucket_names=[]
+ conns=[]
+
+ for i in range(min(req_count, len(g_tenants))):
+ log.info("recycle existing tenants connection")
+ conns.append(g_tenants_connection[i])
+ tenants.append(g_tenants[i])
+ # we need to create a new bucket as we remove existing buckets at cleanup
+ bucket_name=gen_bucket_name()
+ bucket_names.append(bucket_name)
+ bucket=conns[i].create_bucket(Bucket=bucket_name)
+
+ if len(conns) < req_count:
+ for i in range(req_count - len(conns)):
+ num_conns += 1
+ user=gen_object_name("user", num_conns) + suffix
+ display_name=gen_object_name("display", num_conns) + suffix
+ tenant=gen_object_name("tenant", num_conns) + suffix
+ g_tenants.append(tenant)
+ tenants.append(tenant)
+ bucket_name=gen_bucket_name()
+ bucket_names.append(bucket_name)
+ log.debug("U=%s, T=%s, B=%s", user, tenant, bucket_name);
+
+ conn=another_user(user, tenant, display_name)
+ bucket=conn.create_bucket(Bucket=bucket_name)
+ g_tenant_connections.append(conn)
+ conns.append(conn)
+
+ log.info("gen_connections_multi: All connection and buckets are set")
+ return (tenants, bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+def gen_connections_multi(num_tenants):
+ global num_conns
+
+ tenants=[]
+ bucket_names=[]
+ conns=[]
+ log.info("gen_connections_multi: Create connection and buckets ...")
+ suffix=run_prefix
+ for i in range(0, num_tenants):
+ num_conns += 1
+ user=gen_object_name("user", num_conns) + suffix
+ display_name=gen_object_name("display", num_conns) + suffix
+ tenant=gen_object_name("tenant", num_conns) + suffix
+ tenants.append(tenant)
+ bucket_name=gen_bucket_name()
+ bucket_names.append(bucket_name)
+ log.debug("U=%s, T=%s, B=%s", user, tenant, bucket_name);
+
+ conn=another_user(user, tenant, display_name)
+ bucket=conn.create_bucket(Bucket=bucket_name)
+ conns.append(conn)
+
+ log.info("gen_connections_multi: All connection and buckets are set")
+ return (tenants, bucket_names, conns)
+
+
+#####################
+# dedup tests
+#####################
+OUT_DIR="/tmp/dedup/"
+KB=(1024)
+MB=(1024*KB)
+RADOS_OBJ_SIZE=(4*MB)
+MULTIPART_SIZE=(16*MB)
+default_config = TransferConfig(multipart_threshold=MULTIPART_SIZE, multipart_chunksize=MULTIPART_SIZE)
+ETAG_ATTR="user.rgw.etag"
+POOLNAME="default.rgw.buckets.data"
+
+#-------------------------------------------------------------------------------
+def write_file(filename, size):
+ full_filename = OUT_DIR + filename
+
+ fout = open(full_filename, "wb")
+ fout.write(os.urandom(size))
+ fout.close()
+
+
+#-------------------------------------------------------------------------------
+def print_size(caller, size):
+ if (size < MB):
+ log.info("%s::size=%.2f KiB (%d Bytes)", caller, size/KB, size)
+ else:
+ log.info("%s::size=%.2f MiB", caller, size/MB)
+
+
+#-------------------------------------------------------------------------------
+def write_random(files, size, min_copies_count=1, max_copies_count=4):
+ global num_files
+ assert(max_copies_count <= 4)
+ num_files += 1
+ filename = "OBJ_" + str(num_files) + str(size)
+ copies_count=random.randint(min_copies_count, max_copies_count)
+ files.append((filename, size, copies_count))
+ write_file(filename, size)
+
+
+#-------------------------------------------------------------------------------
+def gen_files_fixed_copies(files, count, size, copies_count):
+ global num_files
+
+ for i in range(0, count):
+ num_files += 1
+ filename = "OBJ_" + str(num_files)
+
+ files.append((filename, size, copies_count))
+ log.debug("gen_files_fixed_size:%s, %d, %d", filename, size, copies_count)
+ write_file(filename, size)
+
+
+#-------------------------------------------------------------------------------
+def gen_files_fixed_size(files, count, size, max_copies_count=4):
+ global num_files
+
+ for i in range(0, count):
+ copies_count=random.randint(1, max_copies_count)
+ num_files += 1
+ filename = "OBJ_" + str(num_files)
+
+ files.append((filename, size, copies_count))
+ log.debug("gen_files_fixed_size:%s, %d, %d", filename, size, copies_count)
+ write_file(filename, size)
+
+
+#-------------------------------------------------------------------------------
+def gen_files_in_range(files, count, min_size, max_size, alignment=RADOS_OBJ_SIZE):
+ assert(min_size <= max_size)
+ if min_size == 0:
+ assert max_size > 1024
+
+ size_range = max_size - min_size
+ size=0
+ for i in range(0, count):
+ size = min_size + random.randint(1, size_range-1)
+ if size == 0:
+ size = 1024 * random.randint(1, 1024)
+
+ log.debug("gen_files_in_range::size=%.2f MiB", size/MB)
+ size_aligned = size - (size % alignment)
+ if size_aligned == 0:
+ size_aligned += alignment
+ if size_aligned == 0:
+ size_aligned = 4*MB
+
+ assert(size)
+ assert(size_aligned)
+ # force dedup by setting min_copies_count to 2
+ write_random(files, size_aligned, 2, 3)
+ write_random(files, size, 1, 3)
+
+
+#-------------------------------------------------------------------------------
+def gen_files(files, start_size, factor, max_copies_count=4):
+ size = start_size
+ for i in range(1, factor+1):
+ size2 = size + random.randint(1, size-1)
+ write_random(files, size, 1, max_copies_count)
+ write_random(files, size2, 1, max_copies_count)
+ size = size * 2;
+
+
+#-------------------------------------------------------------------------------
+def count_space_in_all_buckets():
+ result = rados(['df'])
+ assert result[1] == 0
+ log.info("=============================================")
+ for line in result[0].splitlines():
+ if line.startswith(POOLNAME):
+ log.info(line[:45])
+ elif line.startswith("POOL_NAME"):
+ log.info(line[:45])
+ log.info("=============================================")
+
+
+#-------------------------------------------------------------------------------
+def count_objects_in_bucket(bucket_name, conn):
+ max_keys=1000
+ marker=""
+ obj_count=0
+ while True:
+ log.info("bucket_name=%s", bucket_name)
+ listing=conn.list_objects(Bucket=bucket_name, Marker=marker, MaxKeys=max_keys)
+ if 'Contents' not in listing or len(listing['Contents'])== 0:
+ return 0
+
+ obj_count += len(listing['Contents'])
+
+ if listing['IsTruncated']:
+ marker=listing['NextMarker']
+ log.info("marker=%s, obj_count=%d", marker, obj_count)
+ continue
+ else:
+ return obj_count
+
+
+#-------------------------------------------------------------------------------
+def count_object_parts_in_all_buckets(verbose=False):
+ result = rados(['lspools'])
+ assert result[1] == 0
+ found=False
+ for pool in result[0].split():
+ if pool == POOLNAME:
+ found=True
+ log.debug("Pool %s was found", POOLNAME)
+ break
+
+ if found == False:
+ log.debug("Pool %s doesn't exists!", POOLNAME)
+ return 0
+
+ result = rados(['ls', '-p ', POOLNAME])
+ assert result[1] == 0
+
+ names=result[0].split()
+ count = 0
+ for name in names:
+ #log.info(name)
+ count = count + 1
+
+ if verbose:
+ log.info("Pool has %d rados objects", count)
+
+ return count
+
+
+#-------------------------------------------------------------------------------
+def cleanup_local():
+ if os.path.isdir(OUT_DIR):
+ log.debug("Removing old directory " + OUT_DIR)
+ shutil.rmtree(OUT_DIR)
+ return True
+ else:
+ return False
+
+
+#-------------------------------------------------------------------------------
+def delete_bucket_with_all_objects(bucket_name, conn):
+ max_keys=1000
+ marker=""
+ obj_count=0
+ while True:
+ listing=conn.list_objects(Bucket=bucket_name, Marker=marker, MaxKeys=max_keys)
+ if 'Contents' not in listing or len(listing['Contents'])== 0:
+ log.info("Bucket '%s' is empty, skipping...", bucket_name)
+ return
+
+ objects=[]
+ for obj in listing['Contents']:
+ log.debug(obj['Key'])
+ objects.append({'Key': obj['Key']})
+
+ obj_count += len(objects)
+ # delete objects from the bucket
+ conn.delete_objects(Bucket=bucket_name, Delete={'Objects':objects})
+ if listing['IsTruncated']:
+ marker=listing['NextMarker']
+ log.info("marker=%s, obj_count=%d", marker, obj_count)
+ continue
+ else:
+ break
+
+ log.debug("Removing Bucket '%s', obj_count=%d", bucket_name, obj_count)
+ conn.delete_bucket(Bucket=bucket_name)
+
+#-------------------------------------------------------------------------------
+def verify_pool_is_empty():
+ result = admin(['gc', 'process', '--include-all'])
+ assert result[1] == 0
+ assert count_object_parts_in_all_buckets() == 0
+
+
+#-------------------------------------------------------------------------------
+def cleanup(bucket_name, conn):
+ if cleanup_local():
+ log.debug("delete_all_objects for bucket <%s>",bucket_name)
+ delete_bucket_with_all_objects(bucket_name, conn)
+
+ verify_pool_is_empty()
+
+
+#-------------------------------------------------------------------------------
+def cleanup_all_buckets(bucket_names, conns):
+ if cleanup_local():
+ for (bucket_name, conn) in zip(bucket_names, conns):
+ log.debug("delete_all_objects for bucket <%s>",bucket_name)
+ delete_bucket_with_all_objects(bucket_name, conn)
+
+ verify_pool_is_empty()
+
+
+#-------------------------------------------------------------------------------
+def gen_object_name(filename, index):
+ return filename + "_" + str(index)
+
+
+#-------------------------------------------------------------------------------
+def calc_rados_obj_count(num_copies, obj_size, config):
+ rados_obj_count = 0
+ threshold = config.multipart_threshold
+ if threshold < RADOS_OBJ_SIZE:
+ if obj_size < 1*MB:
+ rados_obj_count = 2
+ else:
+ rados_obj_count = math.ceil(obj_size/threshold)
+ if obj_size >= threshold:
+ rados_obj_count += 1
+
+ log.debug(">>>obj_size=%.2f MiB, rados_obj_count=%d, num_copies=%d",
+ float(obj_size)/MB, rados_obj_count, num_copies)
+ return rados_obj_count
+
+ # split the object into parts
+ full_parts_count = (obj_size // threshold)
+ if full_parts_count :
+ # each part is written separately so the last part can be incomplete
+ rados_objs_per_full_part = math.ceil(threshold/RADOS_OBJ_SIZE)
+ rados_obj_count = (full_parts_count * rados_objs_per_full_part)
+ # add one part for an empty head-object
+ rados_obj_count += 1
+
+ partial_part = (obj_size % threshold)
+ if partial_part:
+ count = math.ceil(partial_part/RADOS_OBJ_SIZE)
+ rados_obj_count += count
+
+ log.debug("obj_size=%d/%.2f MiB, rados_obj_count=%d, num_copies=%d",
+ obj_size, float(obj_size)/MB, rados_obj_count, num_copies)
+ return rados_obj_count
+
+
+#-------------------------------------------------------------------------------
+def calc_dedupable_space(obj_size, config):
+ threshold = config.multipart_threshold
+ # Objects with size bigger than MULTIPART_SIZE are uploaded as multi-part
+ # multi-part objects got a zero size Head objects
+ if obj_size >= threshold:
+ dedupable_space = obj_size
+ elif obj_size > RADOS_OBJ_SIZE:
+ dedupable_space = obj_size - RADOS_OBJ_SIZE
+ else:
+ dedupable_space = 0
+
+ log.debug("obj_size=%.2f MiB, dedupable_space=%.2f MiB",
+ float(obj_size)/MB, float(dedupable_space)/MB)
+ return dedupable_space
+
+BLOCK_SIZE=4096
+#-------------------------------------------------------------------------------
+def calc_on_disk_byte_size(byte_size):
+ return (((byte_size+BLOCK_SIZE-1)//BLOCK_SIZE)*BLOCK_SIZE)
+
+
+#-------------------------------------------------------------------------------
+def calc_expected_stats(dedup_stats, obj_size, num_copies, config):
+ on_disk_byte_size = calc_on_disk_byte_size(obj_size)
+ log.debug("obj_size=%d, on_disk_byte_size=%d", obj_size, on_disk_byte_size)
+ threshold = config.multipart_threshold
+ dedup_stats.skip_shared_manifest = 0
+ dedup_stats.size_before_dedup += (on_disk_byte_size * num_copies)
+ if on_disk_byte_size <= RADOS_OBJ_SIZE and threshold > RADOS_OBJ_SIZE:
+ dedup_stats.skip_too_small += num_copies
+ dedup_stats.skip_too_small_bytes += (on_disk_byte_size * num_copies)
+ return
+
+ dedup_stats.total_processed_objects += num_copies
+ dedup_stats.loaded_objects += num_copies
+
+ if num_copies == 1:
+ dedup_stats.singleton_obj += 1
+ dedup_stats.skip_singleton += 1
+ dedup_stats.skip_singleton_bytes += on_disk_byte_size
+ else:
+ dedup_stats.skip_src_record += 1
+ dedup_stats.set_shared_manifest_src += 1
+ dedup_stats.set_sha256 += num_copies
+ dedup_stats.invalid_sha256 += num_copies
+ dedup_stats.unique_obj += 1
+ dups_count = (num_copies - 1)
+ dedup_stats.duplicate_obj += dups_count
+ dedup_stats.deduped_obj += dups_count
+ deduped_obj_bytes=calc_dedupable_space(on_disk_byte_size, config)
+ dedup_stats.deduped_obj_bytes += (deduped_obj_bytes * dups_count)
+ deduped_block_bytes=((deduped_obj_bytes+BLOCK_SIZE-1)//BLOCK_SIZE)*BLOCK_SIZE
+ dedup_stats.dedup_bytes_estimate += (deduped_block_bytes * dups_count)
+
+
+#-------------------------------------------------------------------------------
+def calc_expected_results(files, config):
+ duplicated_tail_objs=0
+ rados_objects_total=0
+
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ assert(obj_size)
+
+ if num_copies > 0:
+ log.debug("calc_expected_results::%s::size=%d, num_copies=%d", filename, obj_size, num_copies);
+ rados_obj_count=calc_rados_obj_count(num_copies, obj_size, config)
+ rados_objects_total += (rados_obj_count * num_copies)
+ duplicated_tail_objs += ((num_copies-1) * (rados_obj_count-1))
+
+ expected_rados_obj_count_post_dedup=(rados_objects_total-duplicated_tail_objs)
+ log.debug("Post dedup expcted rados obj count = %d", expected_rados_obj_count_post_dedup)
+
+ return expected_rados_obj_count_post_dedup
+
+
+#-------------------------------------------------------------------------------
+def upload_objects(bucket_name, files, indices, conn, config, check_obj_count=True):
+ dedup_stats = Dedup_Stats()
+ total_space=0
+ duplicated_space=0
+ duplicated_tail_objs=0
+ rados_objects_total=0
+ s3_objects_total=0
+
+ for (f, idx) in zip(files, indices):
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ assert(obj_size)
+ calc_expected_stats(dedup_stats, obj_size, num_copies, config)
+ total_space += (obj_size * num_copies)
+ dedupable_space=calc_dedupable_space(obj_size, config)
+ duplicated_space += ((num_copies-1) * dedupable_space)
+ rados_obj_count=calc_rados_obj_count(num_copies, obj_size, config)
+ rados_objects_total += (rados_obj_count * num_copies)
+ duplicated_tail_objs += ((num_copies-1) * (rados_obj_count-1))
+ log.debug("upload_objects::%s::size=%d, num_copies=%d", filename, obj_size, num_copies);
+ s3_objects_total += num_copies
+ if s3_objects_total and (s3_objects_total % 1000 == 0):
+ log.info("%d S3 objects were uploaded (%d rados objects), total size = %.2f MiB",
+ s3_objects_total, rados_objects_total, total_space/MB)
+ for i in range(idx, num_copies):
+ key = gen_object_name(filename, i)
+ #log.info("upload_file %s/%s with crc32", bucket_name, key)
+ conn.upload_file(OUT_DIR + filename, bucket_name, key, Config=config, ExtraArgs={'ChecksumAlgorithm': 'crc32'})
+
+ log.debug("==========================================")
+ log.info("Summery:\n%d S3 objects were uploaded (%d rados objects), total size = %.2f MiB",
+ s3_objects_total, rados_objects_total, total_space/MB)
+ log.debug("Based on calculation we should have %d rados objects", rados_objects_total)
+ log.debug("Based on calculation we should have %d duplicated tail objs", duplicated_tail_objs)
+ log.info("Based on calculation we should have %.2f MiB total in pool", total_space/MB)
+ log.info("Based on calculation we should have %.2f MiB duplicated space in pool", duplicated_space/MB)
+
+ expected_rados_obj_count_post_dedup=(rados_objects_total-duplicated_tail_objs)
+ log.debug("Post dedup expcted rados obj count = %d", expected_rados_obj_count_post_dedup)
+ expcted_space_post_dedup=(total_space-duplicated_space)
+ log.info("Post dedup expcted data in pool = %.2f MiB", expcted_space_post_dedup/MB)
+ if check_obj_count:
+ assert rados_objects_total == count_object_parts_in_all_buckets()
+
+ expected_results=(expected_rados_obj_count_post_dedup, expcted_space_post_dedup)
+ return (expected_rados_obj_count_post_dedup, dedup_stats, s3_objects_total)
+
+
+#-------------------------------------------------------------------------------
+def upload_objects_multi(files, conns, bucket_names, indices, config, check_obj_count=True):
+ max_tenants=len(conns)
+ dedup_stats = Dedup_Stats()
+ total_space=0
+ duplicated_space=0
+ duplicated_tail_objs=0
+ rados_objects_total=0
+ s3_objects_total=0
+ for (f, idx) in zip(files, indices):
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ assert(obj_size)
+ calc_expected_stats(dedup_stats, obj_size, num_copies, config)
+ total_space += (obj_size * num_copies)
+ dedupable_space=calc_dedupable_space(obj_size, config)
+ duplicated_space += ((num_copies-1) * dedupable_space)
+ rados_obj_count=calc_rados_obj_count(num_copies, obj_size, config)
+ rados_objects_total += (rados_obj_count * num_copies)
+ duplicated_tail_objs += ((num_copies-1) * (rados_obj_count-1))
+ log.debug("upload_objects::%s::size=%d, num_copies=%d", filename, obj_size, num_copies);
+ s3_objects_total += num_copies
+ if s3_objects_total and (s3_objects_total % 1000 == 0):
+ log.info("%d S3 objects were uploaded (%d rados objects), total size = %.2f MiB",
+ s3_objects_total, rados_objects_total, total_space/MB)
+ for i in range(idx, num_copies):
+ ten_id = i % max_tenants
+ key = gen_object_name(filename, i)
+ conns[ten_id].upload_file(OUT_DIR + filename, bucket_names[ten_id], key, Config=config)
+ log.debug("upload_objects::<%s/%s>", bucket_names[ten_id], key)
+
+ log.debug("==========================================")
+ log.info("Summery:%d S3 objects were uploaded (%d rados objects), total size = %.2f MiB",
+ s3_objects_total, rados_objects_total, total_space/MB)
+ log.debug("Based on calculation we should have %d rados objects", rados_objects_total)
+ log.debug("Based on calculation we should have %d duplicated tail objs", duplicated_tail_objs)
+ log.debug("Based on calculation we should have %.2f MiB total in pool", total_space/MB)
+ log.debug("Based on calculation we should have %.2f MiB duplicated space in pool", duplicated_space/MB)
+
+ s3_object_count=0
+ for (bucket_name, conn) in zip(bucket_names, conns):
+ s3_object_count += count_objects_in_bucket(bucket_name, conn)
+
+ log.info("bucket listings reported a total of %d s3 objects", s3_object_count)
+ expected_rados_obj_count_post_dedup=(rados_objects_total-duplicated_tail_objs)
+ log.debug("Post dedup expcted rados obj count = %d", expected_rados_obj_count_post_dedup)
+ expcted_space_post_dedup=(total_space-duplicated_space)
+ log.debug("Post dedup expcted data in pool = %.2f MiB", expcted_space_post_dedup/MB)
+ if check_obj_count:
+ assert rados_objects_total == count_object_parts_in_all_buckets()
+ assert (s3_object_count == s3_objects_total)
+
+ expected_results=(expected_rados_obj_count_post_dedup, expcted_space_post_dedup)
+ return (expected_rados_obj_count_post_dedup, dedup_stats, s3_objects_total)
+
+
+#---------------------------------------------------------------------------
+def proc_upload(proc_id, num_procs, files, conn, bucket_name, indices, config):
+ count=0
+ for (f, idx) in zip(files, indices):
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ assert(obj_size)
+ log.debug("upload_objects::%s::size=%d, num_copies=%d", filename, obj_size, num_copies);
+ for i in range(idx, num_copies):
+ target_proc = (count % num_procs)
+ count += 1
+ if (proc_id == target_proc):
+ key = gen_object_name(filename, i)
+ conn.upload_file(OUT_DIR+filename, bucket_name, key, Config=config)
+ log.info("[%d]upload_objects::<%s/%s>", proc_id, bucket_name, key)
+
+
+#---------------------------------------------------------------------------
+def procs_upload_objects(files, conns, bucket_names, indices, config, check_obj_count=True):
+ num_procs=len(conns)
+ proc_list=list()
+ for idx in range(num_procs):
+ # Seems the processes are much faster than threads (probably due to python gil)
+ p=Process(target=proc_upload,
+ args=(idx, num_procs, files, conns[idx], bucket_names[idx], indices, config))
+ proc_list.append(p)
+ proc_list[idx].start()
+
+ dedup_stats = Dedup_Stats()
+ total_space=0
+ duplicated_space=0
+ duplicated_tail_objs=0
+ rados_objects_total=0
+ s3_objects_total=0
+ for (f, idx) in zip(files, indices):
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ assert(obj_size)
+ calc_expected_stats(dedup_stats, obj_size, num_copies, config)
+ total_space += (obj_size * num_copies)
+ dedupable_space=calc_dedupable_space(obj_size, config)
+ duplicated_space += ((num_copies-1) * dedupable_space)
+ rados_obj_count=calc_rados_obj_count(num_copies, obj_size, config)
+ rados_objects_total += (rados_obj_count * num_copies)
+ duplicated_tail_objs += ((num_copies-1) * (rados_obj_count-1))
+ log.debug("upload_objects::%s::size=%d, num_copies=%d", filename, obj_size, num_copies);
+ s3_objects_total += num_copies
+
+ # wait for all worker proc to join
+ for idx in range(num_procs):
+ proc_list[idx].join()
+
+ log.debug("==========================================")
+ log.info("Summery:%d S3 objects were uploaded (%d rados objects), total size = %.2f MiB",
+ s3_objects_total, rados_objects_total, total_space/MB)
+ log.debug("Based on calculation we should have %d rados objects", rados_objects_total)
+ log.debug("Based on calculation we should have %d duplicated tail objs", duplicated_tail_objs)
+ log.debug("Based on calculation we should have %.2f MiB total in pool", total_space/MB)
+ log.debug("Based on calculation we should have %.2f MiB duplicated space in pool", duplicated_space/MB)
+
+ s3_object_count=0
+ for (bucket_name, conn) in zip(bucket_names, conns):
+ s3_object_count += count_objects_in_bucket(bucket_name, conn)
+
+ log.info("bucket listings reported a total of %d s3 objects", s3_object_count)
+ expected_rados_obj_count_post_dedup=(rados_objects_total-duplicated_tail_objs)
+ log.debug("Post dedup expcted rados obj count = %d", expected_rados_obj_count_post_dedup)
+ expcted_space_post_dedup=(total_space-duplicated_space)
+ log.debug("Post dedup expcted data in pool = %.2f MiB", expcted_space_post_dedup/MB)
+ if check_obj_count:
+ assert rados_objects_total == count_object_parts_in_all_buckets()
+ assert (s3_object_count == s3_objects_total)
+
+ expected_results=(expected_rados_obj_count_post_dedup, expcted_space_post_dedup)
+ return (expected_rados_obj_count_post_dedup, dedup_stats, s3_objects_total)
+
+
+#-------------------------------------------------------------------------------
+def verify_objects(bucket_name, files, conn, expected_results, config):
+ tempfile = OUT_DIR + "temp"
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ log.debug("comparing file=%s, size=%d, copies=%d", filename, obj_size, num_copies)
+ for i in range(0, num_copies):
+ key = gen_object_name(filename, i)
+ #log.info("download_file(%s) with crc32", key)
+ conn.download_file(bucket_name, key, tempfile, Config=config, ExtraArgs={'ChecksumMode': 'crc32'})
+ #conn.download_file(bucket_name, key, tempfile, Config=config)
+ result = bash(['cmp', tempfile, OUT_DIR + filename])
+ assert result[1] == 0 ,"Files %s and %s differ!!" % (key, tempfile)
+ os.remove(tempfile)
+
+ assert expected_results == count_object_parts_in_all_buckets(True)
+ log.info("verify_objects::completed successfully!!")
+
+
+#-------------------------------------------------------------------------------
+def verify_objects_multi(files, conns, bucket_names, expected_results, config):
+ max_tenants=len(conns)
+ tempfile = OUT_DIR + "temp"
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ log.debug("comparing file=%s, size=%d, copies=%d", filename, obj_size, num_copies)
+ for i in range(0, num_copies):
+ key = gen_object_name(filename, i)
+ log.debug("comparing object %s with file %s", key, filename)
+ ten_id = i % max_tenants
+ conns[ten_id].download_file(bucket_names[ten_id], key, tempfile, Config=config)
+ result = bash(['cmp', tempfile, OUT_DIR + filename])
+ assert result[1] == 0 ,"Files %s and %s differ!!" % (key, tempfile)
+ os.remove(tempfile)
+
+ assert expected_results == count_object_parts_in_all_buckets(True)
+ log.info("verify_objects::completed successfully!!")
+
+
+#-------------------------------------------------------------------------------
+def thread_verify(thread_id, num_threads, files, conn, bucket, config):
+ tempfile = OUT_DIR + "temp" + str(thread_id)
+ count = 0
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ log.info("comparing file=%s, size=%d, copies=%d", filename, obj_size, num_copies)
+ for i in range(0, num_copies):
+ target_thread = count % num_threads
+ count += 1
+ if thread_id == target_thread:
+ key = gen_object_name(filename, i)
+ log.info("comparing object %s with file %s", key, filename)
+ conn.download_file(bucket, key, tempfile, Config=config)
+ result = bash(['cmp', tempfile, OUT_DIR + filename])
+ assert result[1] == 0 ,"Files %s and %s differ!!" % (key, tempfile)
+ os.remove(tempfile)
+
+
+#-------------------------------------------------------------------------------
+def threads_verify_objects(files, conns, bucket_names, expected_results, config):
+ num_threads=len(conns)
+ thread_list=list()
+
+ for idx in range(num_threads):
+ t=threading.Thread(target=thread_verify,
+ args=(idx, num_threads, files, conns[idx], bucket_names[idx], config))
+ thread_list.append(t)
+ thread_list[idx].start()
+
+ # wait for all worker thread to join
+ for idx in range(num_threads):
+ thread_list[idx].join()
+
+ assert expected_results == count_object_parts_in_all_buckets(True)
+ log.info("verify_objects::completed successfully!!")
+
+
+#-------------------------------------------------------------------------------
+def get_stats_line_val(line):
+ return int(line.rsplit("=", maxsplit=1)[1].strip())
+
+
+#-------------------------------------------------------------------------------
+def print_dedup_stats(dedup_stats):
+ for key in dedup_stats.__dict__:
+ log.warning("dedup_stats[%s] = %d", key, dedup_stats.__dict__[key])
+
+
+#-------------------------------------------------------------------------------
+def print_dedup_stats_diff(actual, expected):
+ for (key1, key2) in zip(actual.__dict__, expected.__dict__):
+ if (actual.__dict__[key1] != expected.__dict__[key2]):
+ log.error("actual[%s] = %d != expected[%s] = %d",
+ key1, actual.__dict__[key1], key2, expected.__dict__[key2])
+
+
+#-------------------------------------------------------------------------------
+def reset_full_dedup_stats(dedup_stats):
+ dedup_stats.total_processed_objects = 0
+ dedup_stats.set_shared_manifest_src = 0
+ dedup_stats.deduped_obj = 0
+ dedup_stats.deduped_obj_bytes = 0
+ dedup_stats.skip_shared_manifest = 0
+ dedup_stats.skip_src_record = 0
+ dedup_stats.skip_singleton = 0
+ dedup_stats.skip_singleton_bytes = 0
+ dedup_stats.skip_changed_object = 0
+ dedup_stats.corrupted_etag = 0
+ dedup_stats.sha256_mismatch = 0
+ dedup_stats.valid_sha256 = 0
+ dedup_stats.invalid_sha256 = 0
+ dedup_stats.set_sha256 = 0
+
+
+#-------------------------------------------------------------------------------
+def read_full_dedup_stats(dedup_stats, md5_stats):
+ main = md5_stats['main']
+ dedup_stats.total_processed_objects = main['Total processed objects']
+ dedup_stats.set_shared_manifest_src = main['Set Shared-Manifest SRC']
+ dedup_stats.deduped_obj = main['Deduped Obj (this cycle)']
+ dedup_stats.deduped_obj_bytes = main['Deduped Bytes(this cycle)']
+
+ skipped = md5_stats['skipped']
+ dedup_stats.skip_shared_manifest = skipped['Skipped shared_manifest']
+ dedup_stats.skip_src_record = skipped['Skipped source record']
+ dedup_stats.skip_singleton = skipped['Skipped singleton objs']
+ if dedup_stats.skip_singleton:
+ dedup_stats.skip_singleton_bytes = skipped['Skipped singleton Bytes']
+ key='Skipped Changed Object'
+ if key in skipped:
+ dedup_stats.skip_changed_object = skipped[key]
+
+ notify=md5_stats['notify']
+ dedup_stats.valid_sha256 = notify['Valid SHA256 attrs']
+ dedup_stats.invalid_sha256 = notify['Invalid SHA256 attrs']
+ key='Set SHA256'
+ if key in notify:
+ dedup_stats.set_sha256 = notify[key]
+
+ sys_failures = md5_stats['system failures']
+ key='Corrupted ETAG'
+ if key in sys_failures:
+ dedup_stats.corrupted_etag = sys_failures[key]
+
+ log_failures = md5_stats['logical failures']
+ key='SHA256 mismatch'
+ if key in log_failures:
+ dedup_stats.sha256_mismatch = log_failures[key]
+
+
+#-------------------------------------------------------------------------------
+def read_dedup_ratio(json):
+ dedup_ratio=Dedup_Ratio()
+ dedup_ratio.s3_bytes_before=json['s3_bytes_before']
+ dedup_ratio.s3_bytes_after=json['s3_bytes_after']
+ dedup_ratio.ratio=json['dedup_ratio']
+
+ log.info("Completed! ::ratio=%f", dedup_ratio.ratio)
+ return dedup_ratio
+
+#-------------------------------------------------------------------------------
+def verify_dedup_ratio(expected_dedup_stats, dedup_ratio):
+ s3_bytes_before = expected_dedup_stats.size_before_dedup
+ s3_dedup_bytes = expected_dedup_stats.dedup_bytes_estimate
+ s3_bytes_after = s3_bytes_before - s3_dedup_bytes
+ skipped_bytes = (expected_dedup_stats.skip_too_small_bytes +
+ expected_dedup_stats.non_default_storage_class_objs_bytes)
+ #s3_bytes_after -= skipped_bytes
+ if (s3_bytes_before > s3_bytes_after) and s3_bytes_after:
+ ratio = s3_bytes_before/s3_bytes_after
+ else:
+ ratio = 0
+
+ log.info("s3_bytes_before = %d/%d", s3_bytes_before, dedup_ratio.s3_bytes_before)
+ log.info("s3_dedup_bytes = %d", expected_dedup_stats.dedup_bytes_estimate);
+ log.info("s3_bytes_after = %d/%d", s3_bytes_after, dedup_ratio.s3_bytes_after)
+ log.info("ratio = %f/%f", ratio, dedup_ratio.ratio)
+
+ assert s3_bytes_before == dedup_ratio.s3_bytes_before
+ assert s3_bytes_after == dedup_ratio.s3_bytes_after
+ assert ratio == dedup_ratio.ratio
+
+#-------------------------------------------------------------------------------
+def read_dedup_stats(dry_run):
+ dedup_work_was_completed = False
+ dedup_stats=Dedup_Stats()
+ dedup_ratio_estimate=Dedup_Ratio()
+ dedup_ratio_actual=Dedup_Ratio()
+
+ result = admin(['dedup', 'stats'])
+ assert result[1] == 0
+
+ jstats=json.loads(result[0])
+ worker_stats=jstats['worker_stats']
+ main=worker_stats['main']
+ skipped=worker_stats['skipped']
+ notify=worker_stats['notify']
+ dedup_stats.size_before_dedup = main['Accum byte size Ingress Objs']
+ key='Ingress skip: too small objs'
+ if key in skipped:
+ dedup_stats.skip_too_small = skipped[key]
+ dedup_stats.skip_too_small_bytes = skipped['Ingress skip: too small bytes']
+
+ key='non default storage class objs bytes'
+ if key in notify:
+ dedup_stats.non_default_storage_class_objs_bytes = notify[key]
+
+ key='md5_stats'
+ if key in jstats:
+ md5_stats=jstats[key]
+ main=md5_stats['main']
+ dedup_stats.loaded_objects = main['Loaded objects']
+ if dry_run == False:
+ read_full_dedup_stats(dedup_stats, md5_stats)
+
+ dedup_stats.singleton_obj = main['Singleton Obj']
+ dedup_stats.unique_obj = main['Unique Obj']
+ dedup_stats.duplicate_obj = main['Duplicate Obj']
+ dedup_stats.dedup_bytes_estimate = main['Dedup Bytes Estimate']
+
+ dedup_work_was_completed=jstats['completed']
+ if dedup_work_was_completed:
+ dedup_ratio_estimate=read_dedup_ratio(jstats['dedup_ratio_estimate'])
+ dedup_ratio_actual=read_dedup_ratio(jstats['dedup_ratio_actual'])
+ else:
+ log.info("Uncompleted!")
+
+ return (dedup_work_was_completed, dedup_stats, dedup_ratio_estimate, dedup_ratio_actual)
+
+
+#-------------------------------------------------------------------------------
+def exec_dedup_internal(expected_dedup_stats, dry_run, max_dedup_time):
+ log.info("sending exec_dedup request: dry_run=%d", dry_run)
+ if dry_run:
+ result = admin(['dedup', 'estimate'])
+ reset_full_dedup_stats(expected_dedup_stats)
+ else:
+ result = admin(['dedup', 'restart'])
+
+ assert result[1] == 0
+ log.info("wait for dedup to complete")
+
+ dedup_time = 0
+ dedup_timeout = 5
+ dedup_stats = Dedup_Stats()
+ dedup_ratio=Dedup_Ratio()
+ wait_for_completion = True
+ while wait_for_completion:
+ assert dedup_time < max_dedup_time
+ time.sleep(dedup_timeout)
+ dedup_time += dedup_timeout
+ ret = read_dedup_stats(dry_run)
+ if ret[0]:
+ wait_for_completion = False
+ log.info("dedup completed in %d seconds", dedup_time)
+ return (dedup_time, ret[1], ret[2], ret[3])
+
+
+#-------------------------------------------------------------------------------
+def exec_dedup(expected_dedup_stats, dry_run, verify_stats=True):
+ # dedup should complete in less than 5 minutes
+ max_dedup_time = 5*60
+ if expected_dedup_stats.deduped_obj > 10000:
+ max_dedup_time = 20 * 60
+ elif expected_dedup_stats.deduped_obj > 5000:
+ max_dedup_time = 10 * 60
+ elif expected_dedup_stats.deduped_obj > 1000:
+ max_dedup_time = 5 * 60
+
+ ret=exec_dedup_internal(expected_dedup_stats, dry_run, max_dedup_time)
+ dedup_time = ret[0]
+ dedup_stats = ret[1]
+ dedup_ratio_estimate = ret[2]
+ dedup_ratio_actual = ret[3]
+
+ if verify_stats == False:
+ return ret
+
+ #dedup_stats.set_sha256 = dedup_stats.invalid_sha256
+ if dedup_stats != expected_dedup_stats:
+ log.info("==================================================")
+ print_dedup_stats_diff(dedup_stats, expected_dedup_stats)
+ print_dedup_stats(dedup_stats)
+ log.info("==================================================\n")
+ assert dedup_stats == expected_dedup_stats
+
+ verify_dedup_ratio(expected_dedup_stats, dedup_ratio_estimate)
+ log.info("expcted_dedup::stats check completed successfully!!")
+ return ret
+
+
+#-------------------------------------------------------------------------------
+def prepare_test():
+ cleanup_local()
+ #make sure we are starting with all buckets empty
+ if count_object_parts_in_all_buckets() != 0:
+ log.warning("The system was left dirty from previous run");
+ log.warning("Make sure to remove all objects before starting");
+ assert(0)
+
+ os.mkdir(OUT_DIR)
+
+
+#-------------------------------------------------------------------------------
+def small_single_part_objs_dedup(conn, bucket_name, dry_run):
+ # 1) generate small random files and store them on disk
+ # 2) upload a random number of copies from each file to bucket
+ # 3) execute DEDUP!!
+ # Read dedup stat-counters:
+ # 5.a verify that objects smaller than RADOS_OBJ_SIZE were skipped
+ prepare_test()
+ try:
+ files=[]
+ num_files = 10
+ base_size = 4*KB
+ log.info("generate files: base size=%d KiB, max_size=%d KiB",
+ base_size/KB, (pow(2, num_files) * base_size)/KB)
+ gen_files(files, base_size, num_files)
+ bucket = conn.create_bucket(Bucket=bucket_name)
+ log.info("upload objects to bucket <%s> ...", bucket_name)
+ indices = [0] * len(files)
+ ret = upload_objects(bucket_name, files, indices, conn, default_config)
+ expected_results = ret[0]
+ dedup_stats = ret[1]
+ s3_objects_total = ret[2]
+
+ # expected stats for small objects - all zeros except for skip_too_small
+ small_objs_dedup_stats = Dedup_Stats()
+ small_objs_dedup_stats.size_before_dedup = dedup_stats.size_before_dedup
+ small_objs_dedup_stats.skip_too_small_bytes=dedup_stats.size_before_dedup
+ small_objs_dedup_stats.skip_too_small = s3_objects_total
+ assert small_objs_dedup_stats == dedup_stats
+
+ exec_dedup(dedup_stats, dry_run)
+ if dry_run == False:
+ log.info("Verify all objects")
+ verify_objects(bucket_name, files, conn, expected_results, default_config)
+
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conn)
+
+
+#-------------------------------------------------------------------------------
+def simple_dedup(conn, files, bucket_name, run_cleanup_after, config, dry_run):
+ # 1) generate random files and store them on disk
+ # 2) upload a random number of copies of each file to bucket
+ # 3) calculate current count of rados objects and pool size
+ # 4) calculate expected count of rados objects and pool size *post dedup*
+
+ # 5) execute DEDUP!!
+ # Read dedup stat-counters:
+ # 5.a verify that objects smaller than RADOS_OBJ_SIZE were skipped
+ # 5.b verify that all objects larger than RADOS_OBJ_SIZE were processed/loaded
+ # 5.c verify that objects with single occurrence were skipped
+ # 5.d verify that we calculate the correct number of unique and dedup objects
+ #
+ # 6) Read all objects from bucket and compare them to their stored copy *before dedup*
+ # This step is used to make sure objects were not corrupted by dedup
+ # 7) count number and size of in-pool rados objects and compare with expected
+ # This step is used to make sure dedup removed *all* duplications
+ # 8) delete all objects from bucket using s3 API
+ # 9) call GC to make sure everything was removed
+ #10) verify that there is nothing left on pool (i.e. ref-count is working)
+ try:
+ log.info("conn.create_bucket(%s)", bucket_name)
+ bucket = conn.create_bucket(Bucket=bucket_name)
+ indices = [0] * len(files)
+ log.info("upload objects to bucket <%s> ...", bucket_name)
+ ret = upload_objects(bucket_name, files, indices, conn, config)
+ expected_results = ret[0]
+ dedup_stats = ret[1]
+ exec_dedup(dedup_stats, dry_run)
+ if dry_run == False:
+ log.info("Verify all objects")
+ verify_objects(bucket_name, files, conn, expected_results, config)
+
+ return ret
+ finally:
+ if run_cleanup_after:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conn)
+
+
+#-------------------------------------------------------------------------------
+def simple_dedup_with_tenants(files, conns, bucket_names, config, dry_run=False):
+ indices=[0] * len(files)
+ ret=upload_objects_multi(files, conns, bucket_names, indices, config)
+ expected_results = ret[0]
+ dedup_stats = ret[1]
+ exec_dedup(dedup_stats, dry_run)
+ if dry_run == False:
+ log.info("Verify all objects")
+ verify_objects_multi(files, conns, bucket_names, expected_results, config)
+
+ return ret
+
+
+#-------------------------------------------------------------------------------
+def dedup_basic_with_tenants_common(files, max_copies_count, config, dry_run):
+ try:
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+ simple_dedup_with_tenants(files, conns, bucket_names, config, dry_run)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+def threads_simple_dedup_with_tenants(files, conns, bucket_names, config, dry_run=False):
+ indices=[0] * len(files)
+
+ start = time.time_ns()
+ upload_ret=procs_upload_objects(files, conns, bucket_names, indices, config)
+ upload_time_sec = (time.time_ns() - start) / (1000*1000*1000)
+ expected_results = upload_ret[0]
+ dedup_stats = upload_ret[1]
+ s3_objects_total = upload_ret[2]
+
+ exec_ret=exec_dedup(dedup_stats, dry_run)
+ exec_time_sec=exec_ret[0]
+ verify_time_sec=0
+ if dry_run == False:
+ log.info("Verify all objects")
+ start = time.time_ns()
+ threads_verify_objects(files, conns, bucket_names,
+ expected_results, config)
+ verify_time_sec = (time.time_ns() - start) / (1000*1000*1000)
+
+ log.info("[%d] obj_count=%d, upload=%d(sec), exec=%d(sec), verify=%d(sec)",
+ len(conns), s3_objects_total, upload_time_sec, exec_time_sec, verify_time_sec);
+ return upload_ret
+
+
+#-------------------------------------------------------------------------------
+def threads_dedup_basic_with_tenants_common(files, num_conns, config, dry_run):
+ try:
+ ret=gen_connections_multi2(num_conns)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+ threads_simple_dedup_with_tenants(files, conns, bucket_names, config, dry_run)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+def check_full_dedup_state():
+ global full_dedup_state_was_checked
+ global full_dedup_state_disabled
+ log.info("check_full_dedup_state:: sending FULL Dedup request")
+ result = admin(['dedup', 'restart'])
+ if result[1] == 0:
+ log.info("full dedup is enabled!")
+ full_dedup_state_disabled = False
+ result = admin(['dedup', 'abort'])
+ assert result[1] == 0
+ else:
+ log.info("full dedup is disabled, skip all full dedup tests")
+ full_dedup_state_disabled = True
+
+ full_dedup_state_was_checked = True
+ return full_dedup_state_disabled
+
+
+#-------------------------------------------------------------------------------
+def full_dedup_is_disabled():
+ global full_dedup_state_was_checked
+ global full_dedup_state_disabled
+
+ if not full_dedup_state_was_checked:
+ full_dedup_state_disabled = check_full_dedup_state()
+
+ if full_dedup_state_disabled:
+ log.info("Full Dedup is DISABLED, skipping test...")
+
+ return full_dedup_state_disabled
+
+
+CORRUPTIONS = ("no corruption", "change_etag", "illegal_hex_value",
+ "change_num_parts", "illegal_separator",
+ "illegal_dec_val_num_parts", "illegal_num_parts_overflow")
+
+#------------------------------------------------------------------------------
+def change_object_etag(rados_name, new_etag):
+ result = rados(['-p ', POOLNAME, 'setxattr', rados_name, ETAG_ATTR, new_etag])
+ assert result[1] == 0
+
+#------------------------------------------------------------------------------
+def gen_new_etag(etag, corruption, expected_dedup_stats):
+ expected_dedup_stats.skip_changed_object = 0
+ expected_dedup_stats.corrupted_etag = 0
+
+ if corruption == "change_etag":
+ # replace one character in the ETAG (will report changed ETAG)
+ expected_dedup_stats.skip_changed_object = 1
+ ch="a"
+ if etag[0] == ch:
+ ch="b"
+
+ return etag.replace(etag[0], ch, 1)
+
+ elif corruption == "illegal_hex_value":
+ # set an illegal hex value (will report corrupted ETAG)
+ expected_dedup_stats.corrupted_etag = 1
+ ch="Z"
+ return etag.replace(etag[0], ch, 1)
+
+ elif corruption == "change_num_parts":
+ # change num_parts (will report changed ETAG)
+ expected_dedup_stats.skip_changed_object = 1
+ return etag + "1"
+
+ elif corruption == "illegal_separator":
+ # change the num_parts separtor (will report corrupted ETAG)
+ expected_dedup_stats.corrupted_etag = 1
+ idx=len(etag) - 2
+ ch="a"
+ return etag.replace(etag[idx], ch, 1)
+
+ elif corruption == "illegal_dec_val_num_parts":
+ # set an illegal decimal val in num_parts (will report corrupted ETAG)
+ expected_dedup_stats.corrupted_etag = 1
+ return etag + "a"
+
+ elif corruption == "illegal_num_parts_overflow":
+ # expand num_part beyond the legal 10,000 (will report corrupted ETAG)
+ expected_dedup_stats.corrupted_etag = 1
+ return etag + "1111"
+
+#------------------------------------------------------------------------------
+def corrupt_etag(key, corruption, expected_dedup_stats):
+ log.info("key=%s, corruption=%s", key, corruption);
+ result = rados(['ls', '-p ', POOLNAME])
+ assert result[1] == 0
+
+ names=result[0].split()
+ for name in names:
+ log.info("name=%s", name)
+ if key in name:
+ log.info("key=%s is a substring of name=%s", key, name);
+ rados_name = name
+ break;
+
+ result = rados(['-p ', POOLNAME, 'getxattr', rados_name, ETAG_ATTR])
+ assert result[1] == 0
+ old_etag = result[0]
+
+ new_etag=gen_new_etag(old_etag, corruption, expected_dedup_stats)
+
+ log.info("Corruption:: %s\nold_etag=%s\nnew_etag=%s",
+ corruption, old_etag, new_etag)
+ change_object_etag(rados_name, new_etag)
+ return (rados_name, old_etag)
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_etag_corruption():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_etag_corruption: connect to AWS ...")
+ conn=get_single_connection()
+ prepare_test()
+ try:
+ files=[]
+ num_files = 1
+ # generate a single object of MULTIPART_SIZE with 2 identical copies
+ gen_files_fixed_copies(files, num_files, MULTIPART_SIZE, 2)
+
+ bucket = conn.create_bucket(Bucket=bucket_name)
+ indices = [0] * len(files)
+ ret = upload_objects(bucket_name, files, indices, conn, default_config)
+ expected_results = ret[0]
+ expected_dedup_stats = ret[1]
+ s3_objects_total = ret[2]
+ f=files[0]
+ filename=f[0]
+ key=gen_object_name(filename, 0)
+
+ for corruption in CORRUPTIONS:
+ if corruption != "no corruption":
+ corrupted=corrupt_etag(key, corruption, expected_dedup_stats)
+ # no dedup will happen because of the inserted corruption
+ expected_dedup_stats.deduped_obj=0
+ expected_dedup_stats.deduped_obj_bytes=0
+ expected_dedup_stats.set_shared_manifest_src=0
+
+ dry_run=False
+ ret=exec_dedup(expected_dedup_stats, dry_run)
+ #dedup_stats=ret[1]
+ dedup_ratio_estimate=ret[2]
+ dedup_ratio_actual=ret[3]
+
+ if corruption == "no corruption":
+ expected_dedup_stats.valid_sha256=1
+ expected_dedup_stats.invalid_sha256=0
+ expected_dedup_stats.set_sha256=0
+
+ s3_bytes_before=expected_dedup_stats.size_before_dedup
+ expected_ratio_actual=Dedup_Ratio()
+ expected_ratio_actual.s3_bytes_before=s3_bytes_before
+ expected_ratio_actual.s3_bytes_after=s3_bytes_before
+ expected_ratio_actual.ratio=0
+ if corruption != "no corruption":
+ assert expected_ratio_actual == dedup_ratio_actual
+ change_object_etag(corrupted[0], corrupted[1])
+
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conn)
+
+#-------------------------------------------------------------------------------
+def write_bin_file(files, bin_arr, filename):
+ full_filename = OUT_DIR + filename
+ fout = open(full_filename, "wb")
+ fout.write(bin_arr)
+ fout.close()
+ files.append((filename, len(bin_arr), 1))
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_md5_collisions():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ s1="d131dd02c5e6eec4693d9a0698aff95c2fcab58712467eab4004583eb8fb7f8955ad340609f4b30283e488832571415a085125e8f7cdc99fd91dbdf280373c5bd8823e3156348f5bae6dacd436c919c6dd53e2b487da03fd02396306d248cda0e99f33420f577ee8ce54b67080a80d1ec69821bcb6a8839396f9652b6ff72a70"
+ s2="d131dd02c5e6eec4693d9a0698aff95c2fcab50712467eab4004583eb8fb7f8955ad340609f4b30283e4888325f1415a085125e8f7cdc99fd91dbd7280373c5bd8823e3156348f5bae6dacd436c919c6dd53e23487da03fd02396306d248cda0e99f33420f577ee8ce54b67080280d1ec69821bcb6a8839396f965ab6ff72a70"
+
+ s1_bin=bytes.fromhex(s1)
+ s2_bin=bytes.fromhex(s2)
+
+ s1_hash=hashlib.md5(s1_bin).hexdigest()
+ s2_hash=hashlib.md5(s2_bin).hexdigest()
+ # data is different
+ assert s1 != s2
+ # but MD5 is identical
+ assert s1_hash == s2_hash
+
+ prepare_test()
+ files=[]
+ try:
+ write_bin_file(files, s1_bin, "s1")
+ write_bin_file(files, s2_bin, "s2")
+
+ bucket_name = gen_bucket_name()
+ log.info("test_md5_collisions: connect to AWS ...")
+ config2=TransferConfig(multipart_threshold=64, multipart_chunksize=1*MB)
+ conn=get_single_connection()
+ bucket = conn.create_bucket(Bucket=bucket_name)
+ indices = [0] * len(files)
+ upload_objects(bucket_name, files, indices, conn, config2)
+
+ dedup_stats = Dedup_Stats()
+ # we wrote 2 different small objects (BLOCK_SIZE) with the same md5
+ dedup_stats.total_processed_objects=2
+ dedup_stats.loaded_objects=dedup_stats.total_processed_objects
+ # the objects will seem like a duplications with 1 unique and 1 duplicate
+ dedup_stats.unique_obj=1
+ dedup_stats.duplicate_obj=1
+ dedup_stats.skip_src_record=1
+ # the objects are 128 Bytes long so will take the min of BLOCK_SIZE each
+ dedup_stats.size_before_dedup=2*BLOCK_SIZE
+ # the md5 collision confuses the estimate
+ dedup_stats.dedup_bytes_estimate=BLOCK_SIZE
+ # SHA256 check will expose the problem
+ dedup_stats.invalid_sha256=dedup_stats.total_processed_objects
+ dedup_stats.set_sha256=dedup_stats.total_processed_objects
+ dedup_stats.sha256_mismatch=1
+ s3_bytes_before=dedup_stats.size_before_dedup
+ expected_ratio_actual=Dedup_Ratio()
+ expected_ratio_actual.s3_bytes_before=s3_bytes_before
+ expected_ratio_actual.s3_bytes_after=s3_bytes_before
+ expected_ratio_actual.ratio=0
+
+ dry_run=False
+ log.info("test_md5_collisions: first call to exec_dedup")
+ ret=exec_dedup(dedup_stats, dry_run)
+ dedup_ratio_actual=ret[3]
+
+ assert expected_ratio_actual == dedup_ratio_actual
+
+ dedup_stats.valid_sha256=dedup_stats.total_processed_objects
+ dedup_stats.invalid_sha256=0
+ dedup_stats.set_sha256=0
+
+ log.info("test_md5_collisions: second call to exec_dedup")
+ ret=exec_dedup(dedup_stats, dry_run)
+ dedup_ratio_actual=ret[3]
+
+ assert expected_ratio_actual == dedup_ratio_actual
+
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conn)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_small():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_small: connect to AWS ...")
+ conn=get_single_connection()
+ small_single_part_objs_dedup(conn, bucket_name, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_small_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ max_copies_count=3
+ files=[]
+ num_files=10 # [4KB-4MB]
+ base_size = 4*KB
+ log.info("generate files: base size=%d KiB, max_size=%d KiB",
+ base_size/KB, (pow(2, num_files) * base_size)/KB)
+ try:
+ gen_files(files, base_size, num_files, max_copies_count)
+ indices=[0] * len(files)
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+
+ ret=upload_objects_multi(files, conns, bucket_names, indices, default_config)
+ expected_results = ret[0]
+ dedup_stats = ret[1]
+ s3_objects_total = ret[2]
+
+ # expected stats for small objects - all zeros except for skip_too_small
+ small_objs_dedup_stats = Dedup_Stats()
+ small_objs_dedup_stats.size_before_dedup=dedup_stats.size_before_dedup
+ small_objs_dedup_stats.skip_too_small_bytes=dedup_stats.size_before_dedup
+ small_objs_dedup_stats.skip_too_small=s3_objects_total
+ assert small_objs_dedup_stats == dedup_stats
+
+ dry_run=False
+ exec_dedup(dedup_stats, dry_run)
+ log.info("Verify all objects")
+ verify_objects_multi(files, conns, bucket_names, expected_results, default_config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#------------------------------------------------------------------------------
+# Trivial incremental dedup:
+# 1) Run the @simple_dedup test above without cleanup post dedup
+# 2) Run a second dedup *without making any change*
+# 3) The stats-counters should show the same dedup ratio, but no change
+# should be made to the system
+@pytest.mark.basic_test
+def test_dedup_inc_0_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ log.info("test_dedup_inc_0: connect to AWS ...")
+ max_copies_count=3
+ config=default_config
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+ try:
+ files=[]
+ num_files=11
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret=simple_dedup_with_tenants(files, conns, bucket_names, config)
+ expected_results = ret[0]
+ dedup_stats = ret[1]
+ s3_objects_total = ret[2]
+
+ dedup_stats2 = dedup_stats
+ dedup_stats2.skip_shared_manifest=dedup_stats.deduped_obj
+ dedup_stats2.skip_src_record=dedup_stats.set_shared_manifest_src
+ dedup_stats2.set_shared_manifest_src=0
+ dedup_stats2.deduped_obj=0
+ dedup_stats2.deduped_obj_bytes=0
+ dedup_stats2.valid_sha256=dedup_stats.invalid_sha256
+ dedup_stats2.invalid_sha256=0
+ dedup_stats2.set_sha256=0
+
+ log.info("test_dedup_inc_0_with_tenants: incremental dedup:")
+ # run dedup again and make sure nothing has changed
+ dry_run=False
+ exec_dedup(dedup_stats2, dry_run)
+ verify_objects_multi(files, conns, bucket_names, expected_results, config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#------------------------------------------------------------------------------
+# Trivial incremental dedup:
+# 1) Run the @simple_dedup test above without cleanup post dedup
+# 2) Run a second dedup *without making any change*
+# 3) The stats-counters should show the same dedup ratio, but no change
+# should be made to the system
+@pytest.mark.basic_test
+def test_dedup_inc_0():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ config=default_config
+ prepare_test()
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_inc_0: connect to AWS ...")
+ conn=get_single_connection()
+ try:
+ files=[]
+ num_files = 11
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret = simple_dedup(conn, files, bucket_name, False, config, False)
+ expected_results = ret[0]
+ dedup_stats = ret[1]
+ s3_objects_total = ret[2]
+
+ dedup_stats2 = dedup_stats
+ dedup_stats2.skip_shared_manifest=dedup_stats.deduped_obj
+ dedup_stats2.skip_src_record=dedup_stats.set_shared_manifest_src
+ dedup_stats2.set_shared_manifest_src=0
+ dedup_stats2.deduped_obj=0
+ dedup_stats2.deduped_obj_bytes=0
+ dedup_stats2.valid_sha256=dedup_stats.invalid_sha256
+ dedup_stats2.invalid_sha256=0
+ dedup_stats2.set_sha256=0
+
+ log.info("test_dedup_inc_0: incremental dedup:")
+ # run dedup again and make sure nothing has changed
+ dry_run=False
+ exec_dedup(dedup_stats2, dry_run)
+ verify_objects(bucket_name, files, conn, expected_results, config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conn)
+
+
+#-------------------------------------------------------------------------------
+# Basic incremental dedup:
+# 1) Run the @simple_dedup test above without cleanup post dedup
+# 2) Add more copies of the *original objects*
+# 3) Run another dedup
+@pytest.mark.basic_test
+def test_dedup_inc_1_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ log.info("test_dedup_inc_1_with_tenants: connect to AWS ...")
+ max_copies_count=6
+ config=default_config
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+ try:
+ files=[]
+ num_files=17
+ # gen_files_in_range creates 2-3 copies
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret=simple_dedup_with_tenants(files, conns, bucket_names, config)
+ expected_results_base=ret[0]
+ stats_base=ret[1]
+
+ # upload more copies of the same objects
+ indices=[]
+ files_combined=[]
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies_base=f[2]
+ # indices holds the start index of the new copies
+ indices.append(num_copies_base)
+ num_copies_to_add=random.randint(0, 2)
+ num_copies_combined=num_copies_to_add+num_copies_base
+ files_combined.append((filename, obj_size, num_copies_combined))
+
+ ret=upload_objects_multi(files_combined, conns, bucket_names, indices, config, False)
+ expected_results=ret[0]
+ stats_combined=ret[1]
+ stats_combined.skip_shared_manifest = stats_base.deduped_obj
+ stats_combined.skip_src_record -= stats_base.skip_src_record
+ stats_combined.skip_src_record += stats_base.set_shared_manifest_src
+
+ stats_combined.set_shared_manifest_src -= stats_base.set_shared_manifest_src
+ stats_combined.deduped_obj -= stats_base.deduped_obj
+ stats_combined.deduped_obj_bytes -= stats_base.deduped_obj_bytes
+
+ stats_combined.valid_sha256 = stats_base.set_sha256
+ stats_combined.invalid_sha256 -= stats_base.set_sha256
+ stats_combined.set_sha256 -= stats_base.set_sha256
+
+ log.info("test_dedup_inc_1_with_tenants: incremental dedup:")
+ # run dedup again
+ dry_run=False
+ exec_dedup(stats_combined, dry_run)
+ verify_objects_multi(files_combined, conns, bucket_names, expected_results, config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+# Basic incremental dedup:
+# 1) Run the @simple_dedup test above without cleanup post dedup
+# 2) Add more copies of the *original objects*
+# 3) Run another dedup
+@pytest.mark.basic_test
+def test_dedup_inc_1():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ config=default_config
+ prepare_test()
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_inc_1: connect to AWS ...")
+ conn=get_single_connection()
+ try:
+ files=[]
+ num_files = 4
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret = simple_dedup(conn, files, bucket_name, False, config, False)
+ expected_results_base = ret[0]
+ stats_base = ret[1]
+
+ # upload more copies of the same objects
+ indices=[]
+ files_combined=[]
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies_base=f[2]
+ # indices holds the start index of the new copies
+ indices.append(num_copies_base)
+ num_copies_to_add=random.randint(0, 2)
+ num_copies_combined=num_copies_to_add+num_copies_base
+ files_combined.append((filename, obj_size, num_copies_combined))
+
+ ret=upload_objects(bucket_name, files_combined, indices, conn, config, False)
+ expected_results = ret[0]
+ stats_combined = ret[1]
+ stats_combined.skip_shared_manifest = stats_base.deduped_obj
+ stats_combined.skip_src_record -= stats_base.skip_src_record
+ stats_combined.skip_src_record += stats_base.set_shared_manifest_src
+
+ stats_combined.set_shared_manifest_src -= stats_base.set_shared_manifest_src
+ stats_combined.deduped_obj -= stats_base.deduped_obj
+ stats_combined.deduped_obj_bytes -= stats_base.deduped_obj_bytes
+
+ stats_combined.valid_sha256 = stats_base.set_sha256
+ stats_combined.invalid_sha256 -= stats_base.set_sha256
+ stats_combined.set_sha256 -= stats_base.set_sha256
+
+ log.info("test_dedup_inc_1: incremental dedup:")
+ # run dedup again
+ dry_run=False
+ exec_dedup(stats_combined, dry_run)
+ verify_objects(bucket_name, files_combined, conn, expected_results, config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conn)
+
+
+#-------------------------------------------------------------------------------
+# Simple incremental dedup:
+# 1) Run the @simple_dedup test above without cleanup post dedup
+# 2) Add more copies of the original objects
+# 3) Add new objects to buckets
+# 4) Run another dedup
+@pytest.mark.basic_test
+def test_dedup_inc_2_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ log.info("test_dedup_inc_2_with_tenants: connect to AWS ...")
+ max_copies_count=6
+ config=default_config
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+ try:
+ files=[]
+ num_files = 17
+ # gen_files_in_range creates 2-3 copies
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret=simple_dedup_with_tenants(files, conns, bucket_names, config)
+ expected_results_base=ret[0]
+ stats_base=ret[1]
+
+ # upload more copies of the same files
+ indices=[]
+ files_combined=[]
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies_base=f[2]
+ # indices holds the start index of the new copies
+ indices.append(num_copies_base)
+ num_copies_inc=random.randint(0, 2)
+ num_copies_combined=num_copies_inc+num_copies_base
+ files_combined.append((filename, obj_size, num_copies_combined))
+
+ # add new files
+ num_files_new = 13
+ gen_files_in_range(files_combined, num_files_new, 2*MB, 32*MB)
+ pad_count = len(files_combined) - len(files)
+ for i in range(0, pad_count):
+ indices.append(0)
+
+ assert(len(indices) == len(files_combined))
+ ret=upload_objects_multi(files_combined, conns, bucket_names, indices, config, False)
+ expected_results = ret[0]
+ stats_combined = ret[1]
+ stats_combined.skip_shared_manifest = stats_base.deduped_obj
+ stats_combined.skip_src_record -= stats_base.skip_src_record
+ stats_combined.skip_src_record += stats_base.set_shared_manifest_src
+
+ stats_combined.set_shared_manifest_src -= stats_base.set_shared_manifest_src
+ stats_combined.deduped_obj -= stats_base.deduped_obj
+ stats_combined.deduped_obj_bytes -= stats_base.deduped_obj_bytes
+
+ stats_combined.valid_sha256 = stats_base.set_sha256
+ stats_combined.invalid_sha256 -= stats_base.set_sha256
+ stats_combined.set_sha256 -= stats_base.set_sha256
+
+ log.info("test_dedup_inc_2_with_tenants: incremental dedup:")
+ # run dedup again
+ dry_run=False
+ exec_dedup(stats_combined, dry_run)
+ verify_objects_multi(files_combined, conns, bucket_names, expected_results, config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+# Simple incremental dedup:
+# 1) Run the @simple_dedup test above without cleanup post dedup
+# 2) Add more copies of the original objects
+# 3) Add new objects to buckets
+# 4) Run another dedup
+@pytest.mark.basic_test
+def test_dedup_inc_2():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ config=default_config
+ prepare_test()
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_inc_2: connect to AWS ...")
+ conn=get_single_connection()
+ try:
+ files=[]
+ num_files = 17
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret = simple_dedup(conn, files, bucket_name, False, config, False)
+ expected_results_base = ret[0]
+ stats_base = ret[1]
+
+ # upload more copies of the same files
+ indices=[]
+ files_combined=[]
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies_base=f[2]
+ indices.append(num_copies_base)
+ num_copies_inc=random.randint(0, 2)
+ num_copies_combined=num_copies_inc+num_copies_base
+ files_combined.append((filename, obj_size, num_copies_combined))
+
+ # add new files
+ num_files_new = 13
+ gen_files_in_range(files_combined, num_files_new, 2*MB, 32*MB)
+ pad_count = len(files_combined) - len(files)
+ for i in range(0, pad_count):
+ indices.append(0)
+
+ assert(len(indices) == len(files_combined))
+ ret=upload_objects(bucket_name, files_combined, indices, conn, config, False)
+ expected_results = ret[0]
+ stats_combined = ret[1]
+ stats_combined.skip_shared_manifest = stats_base.deduped_obj
+ stats_combined.skip_src_record -= stats_base.skip_src_record
+ stats_combined.skip_src_record += stats_base.set_shared_manifest_src
+
+ stats_combined.set_shared_manifest_src -= stats_base.set_shared_manifest_src
+ stats_combined.deduped_obj -= stats_base.deduped_obj
+ stats_combined.deduped_obj_bytes -= stats_base.deduped_obj_bytes
+
+ stats_combined.valid_sha256 = stats_base.set_sha256
+ stats_combined.invalid_sha256 -= stats_base.set_sha256
+ stats_combined.set_sha256 -= stats_base.set_sha256
+
+ log.info("test_dedup_inc_2: incremental dedup:")
+ # run dedup again
+ dry_run=False
+ exec_dedup(stats_combined, dry_run)
+ verify_objects(bucket_name, files_combined, conn, expected_results,
+ config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conn)
+
+
+#-------------------------------------------------------------------------------
+# Incremental dedup with object removal:
+# 1) Run the @simple_dedup test above without cleanup post dedup
+# 2) Remove copies of some objects
+# 3) Run another dedup
+@pytest.mark.basic_test
+def test_dedup_inc_with_remove_multi_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ log.info("test_dedup_inc_with_remove_multi_tenants: connect to AWS ...")
+ max_copies_count=6
+ config=default_config
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+ try:
+ files=[]
+ num_files = 17
+ # gen_files_in_range creates 2-3 copies
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret=simple_dedup_with_tenants(files, conns, bucket_names, config)
+ expected_results_base = ret[0]
+ stats_base = ret[1]
+
+ # REMOVE some objects and update stats/expected
+ src_record=0
+ shared_manifest=0
+ valid_sha=0
+ object_keys=[]
+ files_sub=[]
+ dedup_stats = Dedup_Stats()
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ num_remove=random.randint(0, num_copies)
+ num_copies_2=num_copies-num_remove
+ log.debug("objects::%s::size=%d, num_copies=%d", filename, obj_size, num_copies_2);
+ if num_copies_2:
+ if num_copies_2 > 1 and obj_size > RADOS_OBJ_SIZE:
+ valid_sha += num_copies_2
+ src_record += 1
+ shared_manifest += (num_copies_2 - 1)
+
+ files_sub.append((filename, obj_size, num_copies_2))
+ calc_expected_stats(dedup_stats, obj_size, num_copies_2, config)
+
+ start_idx=num_copies_2
+ for i in range(start_idx, num_copies):
+ key = gen_object_name(filename, i)
+ log.debug("delete object Bucket=%s, Key=%s", bucket_names[i], key);
+ conns[i].delete_object(Bucket=bucket_names[i], Key=key)
+
+ # must call garbage collection for a predictable count
+ result = admin(['gc', 'process', '--include-all'])
+ assert result[1] == 0
+
+ # run dedup again
+ dedup_stats.set_shared_manifest_src=0
+ dedup_stats.deduped_obj=0
+ dedup_stats.deduped_obj_bytes=0
+ dedup_stats.skip_src_record=src_record
+ dedup_stats.skip_shared_manifest=shared_manifest
+ dedup_stats.valid_sha256=valid_sha
+ dedup_stats.invalid_sha256=0
+ dedup_stats.set_sha256=0
+
+ log.info("test_dedup_inc_with_remove: incremental dedup:")
+ dry_run=False
+ exec_dedup(dedup_stats, dry_run)
+ expected_results=calc_expected_results(files_sub, config)
+ verify_objects_multi(files_sub, conns, bucket_names, expected_results, config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+# Incremental dedup with object removal:
+# 1) Run the @simple_dedup test above without cleanup post dedup
+# 2) Remove copies of some objects
+# 3) Run another dedup
+@pytest.mark.basic_test
+def test_dedup_inc_with_remove():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ config=default_config
+ prepare_test()
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_inc_with_remove: connect to AWS ...")
+ conn=get_single_connection()
+ try:
+ files=[]
+ num_files = 17
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret = simple_dedup(conn, files, bucket_name, False, config, False)
+ expected_results_base = ret[0]
+ stats_base = ret[1]
+
+ # REMOVE some objects and update stats/expected
+ src_record=0
+ shared_manifest=0
+ valid_sha=0
+ object_keys=[]
+ files_sub=[]
+ dedup_stats = Dedup_Stats()
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies=f[2]
+ num_remove=random.randint(0, num_copies)
+ num_copies_2=num_copies-num_remove
+ log.debug("objects::%s::size=%d, num_copies=%d", filename, obj_size, num_copies_2);
+ if num_copies_2:
+ if num_copies_2 > 1 and obj_size > RADOS_OBJ_SIZE:
+ valid_sha += num_copies_2
+ src_record += 1
+ shared_manifest += (num_copies_2 - 1)
+
+ files_sub.append((filename, obj_size, num_copies_2))
+ calc_expected_stats(dedup_stats, obj_size, num_copies_2, config)
+
+ start_idx=num_copies_2
+ for i in range(start_idx, num_copies):
+ key = gen_object_name(filename, i)
+ log.debug("delete key::%s::", key);
+ object_keys.append(key)
+
+ if len(object_keys) == 0:
+ log.info("Skiping file=%s, num_remove=%d", filename, num_remove)
+ continue
+
+ response=conn.delete_objects(Bucket=bucket_name,
+ Delete={"Objects": [{"Key": key} for key in object_keys]})
+
+ # must call garbage collection for predictable count
+ result = admin(['gc', 'process', '--include-all'])
+ assert result[1] == 0
+
+ # run dedup again
+ dedup_stats.set_shared_manifest_src=0
+ dedup_stats.deduped_obj=0
+ dedup_stats.deduped_obj_bytes=0
+ dedup_stats.skip_src_record=src_record
+ dedup_stats.skip_shared_manifest=shared_manifest
+ dedup_stats.valid_sha256=valid_sha
+ dedup_stats.invalid_sha256=0
+ dedup_stats.set_sha256=0
+
+ log.info("test_dedup_inc_with_remove: incremental dedup:")
+ log.info("stats_base.size_before_dedup=%d", stats_base.size_before_dedup)
+ log.info("dedup_stats.size_before_dedup=%d", dedup_stats.size_before_dedup)
+ dry_run=False
+ exec_dedup(dedup_stats, dry_run)
+ expected_results=calc_expected_results(files_sub, config)
+ verify_objects(bucket_name, files_sub, conn, expected_results, config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conn)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_multipart_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ log.info("test_dedup_multipart_with_tenants: connect to AWS ...")
+ max_copies_count=3
+ num_files=8
+ files=[]
+ min_size=MULTIPART_SIZE
+ # create files in range [MULTIPART_SIZE, 4*MULTIPART_SIZE] aligned on RADOS_OBJ_SIZE
+ gen_files_in_range(files, num_files, min_size, min_size*8)
+
+ # add files in range [MULTIPART_SIZE, 4*MULTIPART_SIZE] aligned on MULTIPART_SIZE
+ gen_files_in_range(files, num_files, min_size, min_size*8, MULTIPART_SIZE)
+
+ # add file with excatly MULTIPART_SIZE
+ write_random(files, MULTIPART_SIZE, 2, 2)
+
+ dedup_basic_with_tenants_common(files, max_copies_count, default_config, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_multipart():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_multipart: connect to AWS ...")
+ conn=get_single_connection()
+ files=[]
+
+ num_files=8
+ min_size=MULTIPART_SIZE
+ # create files in range [MULTIPART_SIZE, 4*MULTIPART_SIZE] aligned on RADOS_OBJ_SIZE
+ gen_files_in_range(files, num_files, min_size, min_size*8)
+
+ # add files in range [MULTIPART_SIZE, 4*MULTIPART_SIZE] aligned on MULTIPART_SIZE
+ gen_files_in_range(files, num_files, min_size, min_size*8, MULTIPART_SIZE)
+
+ # add file with excatly MULTIPART_SIZE
+ write_random(files, MULTIPART_SIZE, 2, 2)
+
+ simple_dedup(conn, files, bucket_name, True, default_config, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_basic_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ max_copies_count=3
+ num_files=23
+ file_size=33*MB
+ files=[]
+ log.info("test_dedup_basic_with_tenants: connect to AWS ...")
+ gen_files_fixed_size(files, num_files, file_size, max_copies_count)
+ dedup_basic_with_tenants_common(files, max_copies_count, default_config, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_basic():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_basic: connect to AWS ...")
+ conn=get_single_connection()
+ files=[]
+ num_files=5
+ base_size = MULTIPART_SIZE
+ log.info("generate files: base size=%d MiB, max_size=%d MiB",
+ base_size/MB, (pow(2, num_files) * base_size)/MB)
+ gen_files(files, base_size, num_files)
+ log.info("call simple_dedup()")
+ simple_dedup(conn, files, bucket_name, True, default_config, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_small_multipart_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ max_copies_count=4
+ num_files=10
+ min_size=4*KB
+ max_size=512*KB
+ files=[]
+ config=TransferConfig(multipart_threshold=min_size, multipart_chunksize=1*MB)
+ log.info("test_dedup_small_multipart_with_tenants: connect to AWS ...")
+
+ # create files in range [4KB-512KB] aligned on 4KB
+ gen_files_in_range(files, num_files, min_size, max_size, min_size)
+ dedup_basic_with_tenants_common(files, max_copies_count, config, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_small_multipart():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ log.info("test_dedup_small_multipart: connect to AWS ...")
+ config2=TransferConfig(multipart_threshold=4*KB, multipart_chunksize=1*MB)
+ conn=get_single_connection()
+ files=[]
+ bucket_name=gen_bucket_name()
+ bucket = conn.create_bucket(Bucket=bucket_name)
+ num_files = 10
+ min_size = 4*KB
+ max_size = 512*KB
+
+ # create files in range [4KB-512KB] aligned on 4KB
+ gen_files_in_range(files, num_files, min_size, max_size, min_size)
+ simple_dedup(conn, files, bucket_name, True, config2, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_large_scale_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ max_copies_count=3
+ num_threads=16
+ num_files=8*1024
+ size=1*KB
+ files=[]
+ config=TransferConfig(multipart_threshold=size, multipart_chunksize=1*MB)
+ log.info("test_dedup_large_scale_with_tenants: connect to AWS ...")
+ gen_files_fixed_size(files, num_files, size, max_copies_count)
+ threads_dedup_basic_with_tenants_common(files, num_threads, config, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_large_scale():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ max_copies_count=3
+ num_threads=16
+ num_files=8*1024
+ size=1*KB
+ files=[]
+ config=TransferConfig(multipart_threshold=size, multipart_chunksize=1*MB)
+ log.info("test_dedup_dry_large_scale_with_tenants: connect to AWS ...")
+ gen_files_fixed_size(files, num_files, size, max_copies_count)
+ threads_dedup_basic_with_tenants_common(files, num_threads, config, False)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_empty_bucket():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ log.info("test_empty_bucket: connect to AWS ...")
+
+ max_copies_count=2
+ config = default_config
+
+ files=[]
+ try:
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+def inc_step_with_tenants(stats_base, files, conns, bucket_names, config):
+ max_copies_count=len(conns)
+ # upload more copies of the same files
+ indices=[]
+ files_combined=[]
+ for f in files:
+ filename=f[0]
+ obj_size=f[1]
+ num_copies_base=f[2]
+ # indices holds the start index of the new copies
+ indices.append(num_copies_base)
+ num_copies_inc=random.randint(0, 2)
+ num_copies_combined=num_copies_inc+num_copies_base
+ files_combined.append((filename, obj_size, num_copies_combined))
+
+ # add new files
+ num_files_new = 11
+ gen_files_in_range(files_combined, num_files_new, 2*MB, 32*MB)
+ pad_count = len(files_combined) - len(files)
+ for i in range(0, pad_count):
+ indices.append(0)
+
+ assert(len(indices) == len(files_combined))
+ ret=upload_objects_multi(files_combined, conns, bucket_names, indices, config, False)
+ expected_results = ret[0]
+ stats_combined = ret[1]
+
+ src_record=0
+ for f in files_combined:
+ obj_size=f[1]
+ num_copies=f[2]
+ if num_copies > 1 and obj_size > RADOS_OBJ_SIZE:
+ src_record += 1
+
+ stats_combined.skip_shared_manifest = stats_base.deduped_obj
+ stats_combined.skip_src_record = src_record
+ stats_combined.set_shared_manifest_src -= stats_base.set_shared_manifest_src
+ stats_combined.deduped_obj -= stats_base.deduped_obj
+ stats_combined.deduped_obj_bytes -= stats_base.deduped_obj_bytes
+
+ stats_combined.valid_sha256 = stats_base.set_sha256
+ stats_combined.invalid_sha256 -= stats_base.set_sha256
+ stats_combined.set_sha256 -= stats_base.set_sha256
+
+ log.info("test_dedup_inc_2_with_tenants: incremental dedup:")
+ # run dedup again
+ dry_run=False
+ exec_dedup(stats_combined, dry_run)
+ verify_objects_multi(files_combined, conns, bucket_names, expected_results, config)
+
+ return (files_combined, stats_combined)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+#@pytest.mark.inc_test
+def test_dedup_inc_loop_with_tenants():
+ #return
+
+ if full_dedup_is_disabled():
+ return
+
+ prepare_test()
+ log.info("test_dedup_inc_loop_with_tenants: connect to AWS ...")
+ max_copies_count=3
+ config=default_config
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+ try:
+ files=[]
+ num_files = 13
+ # gen_files_in_range creates 2-3 copies
+ gen_files_in_range(files, num_files, 1*MB, 64*MB)
+ # upload objects, dedup, verify, but don't cleanup
+ ret=simple_dedup_with_tenants(files, conns, bucket_names, config)
+ stats_base=ret[1]
+
+ for idx in range(0, 9):
+ ret = inc_step_with_tenants(stats_base, files, conns, bucket_names, config)
+ files=ret[0]
+ stats_last=ret[1]
+ stats_base.set_shared_manifest_src += stats_last.set_shared_manifest_src
+ stats_base.deduped_obj += stats_last.deduped_obj
+ stats_base.deduped_obj_bytes += stats_last.deduped_obj_bytes
+ stats_base.set_sha256 += stats_last.set_sha256
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+# DRY RUN TESTS
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_small_with_tenants():
+ #return
+
+ log.info("test_dedup_dry_small_with_tenants: connect to AWS ...")
+ prepare_test()
+ max_copies_count=3
+ files=[]
+ num_files=10 # [4KB-4MB]
+ base_size = 4*KB
+ log.info("generate files: base size=%d KiB, max_size=%d KiB",
+ base_size/KB, (pow(2, num_files) * base_size)/KB)
+ try:
+ gen_files(files, base_size, num_files, max_copies_count)
+ indices=[0] * len(files)
+ ret=gen_connections_multi2(max_copies_count)
+ tenants=ret[0]
+ bucket_names=ret[1]
+ conns=ret[2]
+
+ ret=upload_objects_multi(files, conns, bucket_names, indices, default_config)
+ expected_results = ret[0]
+ dedup_stats = ret[1]
+ s3_objects_total = ret[2]
+
+ # expected stats for small objects - all zeros except for skip_too_small
+ small_objs_dedup_stats = Dedup_Stats()
+ small_objs_dedup_stats.size_before_dedup=dedup_stats.size_before_dedup
+ small_objs_dedup_stats.skip_too_small_bytes=dedup_stats.size_before_dedup
+ small_objs_dedup_stats.skip_too_small=s3_objects_total
+ assert small_objs_dedup_stats == dedup_stats
+ dry_run=True
+ exec_dedup(dedup_stats, dry_run)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_multipart():
+ #return
+
+ prepare_test()
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_dry_multipart: connect to AWS ...")
+ conn=get_single_connection()
+ files=[]
+
+ num_files=8
+ min_size=MULTIPART_SIZE
+ # create files in range [MULTIPART_SIZE, 4*MULTIPART_SIZE] aligned on RADOS_OBJ_SIZE
+ # create files in range [MULTIPART_SIZE, 1GB] aligned on RADOS_OBJ_SIZE
+ gen_files_in_range(files, num_files, min_size, 1024*MB)
+
+ # add files in range [MULTIPART_SIZE, 4*MULTIPART_SIZE] aligned on MULTIPART_SIZE
+ gen_files_in_range(files, num_files, min_size, min_size*8, MULTIPART_SIZE)
+
+ # add file with excatly MULTIPART_SIZE
+ write_random(files, MULTIPART_SIZE, 2, 2)
+
+ simple_dedup(conn, files, bucket_name, True, default_config, True)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_basic():
+ #return
+
+ prepare_test()
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_dry_basic: connect to AWS ...")
+ conn=get_single_connection()
+ files=[]
+ num_files=5
+ base_size = MULTIPART_SIZE
+ log.info("generate files: base size=%d MiB, max_size=%d MiB",
+ base_size/MB, (pow(2, num_files) * base_size)/MB)
+ gen_files(files, base_size, num_files)
+ log.info("call simple_dedup()")
+ simple_dedup(conn, files, bucket_name, True, default_config, True)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_small_multipart():
+ #return
+
+ prepare_test()
+ log.info("test_dedup_dry_small_multipart: connect to AWS ...")
+ config2 = TransferConfig(multipart_threshold=4*KB, multipart_chunksize=1*MB)
+ conn=get_single_connection()
+ files=[]
+ bucket_name = gen_bucket_name()
+ bucket = conn.create_bucket(Bucket=bucket_name)
+ num_files = 10
+ min_size = 4*KB
+ max_size = 512*KB
+
+ # create files in range [4KB-512KB] aligned on 4KB
+ gen_files_in_range(files, num_files, min_size, max_size, min_size)
+ simple_dedup(conn, files, bucket_name, True, config2, True)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_small():
+ #return
+
+ bucket_name = gen_bucket_name()
+ log.info("test_dedup_dry_small: connect to AWS ...")
+ conn=get_single_connection()
+ small_single_part_objs_dedup(conn, bucket_name, True)
+
+
+#-------------------------------------------------------------------------------
+# 1) generate a mix of small and large random files and store them on disk
+# 2) upload a random number of copies from each file to bucket
+# 3) execute DEDUP!!
+# 4) Read dedup stat-counters:
+# 5) verify that objects smaller than RADOS_OBJ_SIZE were skipped
+# 6) verify that dedup ratio is reported correctly
+@pytest.mark.basic_test
+def test_dedup_dry_small_large_mix():
+ #return
+
+ dry_run=True
+ log.info("test_dedup_dry_small_large_mix: connect to AWS ...")
+ prepare_test()
+
+ num_threads=4
+ max_copies_count=3
+ small_file_size=1*MB
+ large_file_size=16*MB
+ num_small_files=128
+ num_large_files=16
+ files=[]
+ conns=[]
+ bucket_names=get_buckets(num_threads)
+ try:
+ gen_files_fixed_size(files, num_small_files, small_file_size, max_copies_count)
+ gen_files_fixed_size(files, num_large_files, large_file_size, max_copies_count)
+
+ start = time.time_ns()
+ conns=get_connections(num_threads)
+ for i in range(num_threads):
+ conns[i].create_bucket(Bucket=bucket_names[i])
+
+ indices = [0] * len(files)
+ ret=procs_upload_objects(files, conns, bucket_names, indices, default_config)
+ upload_time_sec = (time.time_ns() - start) / (1000*1000*1000)
+ expected_results = ret[0]
+ dedup_stats = ret[1]
+ s3_objects_total = ret[2]
+ log.info("new[%d] obj_count=%d, upload_time=%d(sec)",
+ len(conns), s3_objects_total, upload_time_sec)
+
+ exec_dedup(dedup_stats, dry_run)
+ if dry_run == False:
+ verify_objects(bucket_name, files, conn, expected_results, default_config)
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_basic_with_tenants():
+ #return
+
+ prepare_test()
+ max_copies_count=3
+ num_files=23
+ file_size=33*MB
+ files=[]
+ log.info("test_dedup_basic_with_tenants: connect to AWS ...")
+ gen_files_fixed_size(files, num_files, file_size, max_copies_count)
+ dedup_basic_with_tenants_common(files, max_copies_count, default_config, True)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_multipart_with_tenants():
+ #return
+
+ prepare_test()
+ log.info("test_dedup_dry_multipart_with_tenants: connect to AWS ...")
+ max_copies_count=3
+ num_files=8
+ files=[]
+ min_size=MULTIPART_SIZE
+ # create files in range [MULTIPART_SIZE, 4*MULTIPART_SIZE] aligned on RADOS_OBJ_SIZE
+ gen_files_in_range(files, num_files, min_size, min_size*32)
+
+ # add files in range [MULTIPART_SIZE, 4*MULTIPART_SIZE] aligned on MULTIPART_SIZE
+ gen_files_in_range(files, num_files, min_size, min_size*8, MULTIPART_SIZE)
+
+ # add file with excatly MULTIPART_SIZE
+ write_random(files, MULTIPART_SIZE, 2, 2)
+
+ dedup_basic_with_tenants_common(files, max_copies_count, default_config, True)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_small_multipart_with_tenants():
+ #return
+
+ prepare_test()
+ max_copies_count=4
+ num_files=10
+ min_size=4*KB
+ max_size=512*KB
+ files=[]
+ config=TransferConfig(multipart_threshold=min_size, multipart_chunksize=1*MB)
+ log.info("test_dedup_small_multipart_with_tenants: connect to AWS ...")
+
+ # create files in range [4KB-512KB] aligned on 4KB
+ gen_files_in_range(files, num_files, min_size, max_size, min_size)
+ dedup_basic_with_tenants_common(files, max_copies_count, config, True)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_large_scale_with_tenants():
+ #return
+
+ prepare_test()
+ max_copies_count=3
+ num_threads=64
+ num_files=32*1024
+ size=1*KB
+ files=[]
+ config=TransferConfig(multipart_threshold=size, multipart_chunksize=1*MB)
+ log.info("test_dedup_dry_large_scale_with_tenants: connect to AWS ...")
+ gen_files_fixed_size(files, num_files, size, max_copies_count)
+ threads_dedup_basic_with_tenants_common(files, num_threads, config, True)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_large_scale():
+ #return
+
+ prepare_test()
+ max_copies_count=3
+ num_threads=64
+ num_files=32*1024
+ size=1*KB
+ files=[]
+ config=TransferConfig(multipart_threshold=size, multipart_chunksize=1*MB)
+ log.info("test_dedup_dry_large_scale_new: connect to AWS ...")
+ gen_files_fixed_size(files, num_files, size, max_copies_count)
+ conns=get_connections(num_threads)
+ bucket_names=get_buckets(num_threads)
+ for i in range(num_threads):
+ conns[i].create_bucket(Bucket=bucket_names[i])
+ try:
+ threads_simple_dedup_with_tenants(files, conns, bucket_names, config, True)
+ except:
+ log.warning("test_dedup_dry_large_scale: failed!!")
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup_all_buckets(bucket_names, conns)
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_dedup_dry_large_scale_single_bucket():
+ return
+
+ prepare_test()
+ max_copies_count=3
+ num_threads=16
+ num_files=32*1024
+ size=1*KB
+ files=[]
+ config=TransferConfig(multipart_threshold=size, multipart_chunksize=1*MB)
+ log.info("test_dedup_dry_large_scale_new: connect to AWS ...")
+ gen_files_fixed_size(files, num_files, size, max_copies_count)
+ conns=get_connections(num_threads)
+
+ bucket_name=gen_bucket_name()
+ conns[0].create_bucket(Bucket=bucket_name)
+
+ bucket_names=[bucket_name] * num_threads
+
+ try:
+ threads_simple_dedup_with_tenants(files, conns, bucket_names, config, True)
+ except:
+ log.warning("test_dedup_dry_large_scale: failed!!")
+ finally:
+ # cleanup must be executed even after a failure
+ cleanup(bucket_name, conns[0])
+
+
+#-------------------------------------------------------------------------------
+@pytest.mark.basic_test
+def test_cleanup():
+ close_all_connections()
+
--- /dev/null
+[tox]
+envlist = py
+skipsdist = True
+
+[testenv]
+deps = -rrequirements.txt
+passenv =
+ DEDUPTESTS_CONF
+commands = pytest {posargs}
echo "$mrun $1 radosgw-admin"
}
+function rgw_rados {
+ [ $# -lt 1 ] && echo "rgw_rados() needs 1 param" && exit 1
+
+ echo "$mrun $1 rados"
+}
+
function rgw {
[ $# -lt 2 ] && echo "rgw() needs at least 2 params" && exit 1
x $(rgw_admin $cid) "$@"
}
+function call_rgw_rados {
+ cid=$1
+ shift 1
+ x $(rgw_rados $cid) "$@"
+}
+
function get_mstart_parameters {
[ $# -ne 1 ] && echo "get_mstart_parameters() needs 1 param" && exit 1
# bash arrays start from zero