]> git.proxmox.com Git - ceph.git/blobdiff - ceph/src/test/osd/RadosModel.h
import quincy beta 17.1.0
[ceph.git] / ceph / src / test / osd / RadosModel.h
index d5689f6067af7b687040645032189bb12f4e2761..a429476749db4d55bd764005d996356163a7caae 100644 (file)
 #include "common/sharedptr_registry.hpp"
 #include "common/errno.h"
 #include "osd/HitSet.h"
+#include "common/ceph_crypto.h"
+
+#include "cls/cas/cls_cas_client.h"
+#include "cls/cas/cls_cas_internal.h"
 
 #ifndef RADOSMODEL_H
 #define RADOSMODEL_H
@@ -64,7 +68,9 @@ enum TestOpType {
   TEST_OP_UNSET_REDIRECT,
   TEST_OP_CHUNK_READ,
   TEST_OP_TIER_PROMOTE,
-  TEST_OP_TIER_FLUSH
+  TEST_OP_TIER_FLUSH,
+  TEST_OP_SET_CHUNK,
+  TEST_OP_TIER_EVICT
 };
 
 class TestWatchContext : public librados::WatchCtx2 {
@@ -166,6 +172,7 @@ public:
   set<string> oid_not_flushing;
   set<string> oid_redirect_not_in_use;
   set<string> oid_redirect_in_use;
+  set<string> oid_set_chunk_tgt_pool;
   SharedPtrRegistry<int, int> snaps_in_use;
   int current_snap;
   string pool_name;
@@ -194,6 +201,8 @@ public:
   int snapname_num;
   map<string,string > redirect_objs;
   bool enable_dedup;
+  string chunk_algo;
+  string chunk_size;
 
   RadosTestContext(const string &pool_name, 
                   int max_in_flight,
@@ -206,6 +215,8 @@ public:
                   bool write_fadvise_dontneed,
                   const string &low_tier_pool_name,
                   bool enable_dedup,
+                  string chunk_algo,
+                  string chunk_size,
                   const char *id = 0) :
     pool_obj_cont(),
     current_snap(0),
@@ -224,7 +235,9 @@ public:
     write_fadvise_dontneed(write_fadvise_dontneed),
     low_tier_pool_name(low_tier_pool_name),
     snapname_num(0),
-    enable_dedup(enable_dedup)
+    enable_dedup(enable_dedup),
+    chunk_algo(chunk_algo),
+    chunk_size(chunk_size)
   {
   }
 
@@ -272,6 +285,30 @@ public:
        rados.shutdown();
        return r;
       }
+      r = rados.mon_command(
+       "{\"prefix\": \"osd pool set\", \"pool\": \"" + pool_name +
+       "\", \"var\": \"dedup_tier\", \"val\": \"" + low_tier_pool_name + "\"}",
+       inbl, NULL, NULL);
+      if (r < 0) {
+       rados.shutdown();
+       return r;
+      }
+      r = rados.mon_command(
+       "{\"prefix\": \"osd pool set\", \"pool\": \"" + pool_name +
+       "\", \"var\": \"dedup_chunk_algorithm\", \"val\": \"" + chunk_algo  + "\"}",
+       inbl, NULL, NULL);
+      if (r < 0) {
+       rados.shutdown();
+       return r;
+      }
+      r = rados.mon_command(
+       "{\"prefix\": \"osd pool set\", \"pool\": \"" + pool_name +
+       "\", \"var\": \"dedup_cdc_chunk_size\", \"val\": \"" + chunk_size + "\"}",
+       inbl, NULL, NULL);
+      if (r < 0) {
+       rados.shutdown();
+       return r;
+      }
     }
 
     char hostname_cstr[100];
@@ -385,6 +422,7 @@ public:
       new_obj.attrs.erase(*i);
     }
     new_obj.dirty = true;
+    new_obj.flushed = false;
     pool_obj_cont[current_snap].insert_or_assign(oid, new_obj);
   }
 
@@ -393,6 +431,7 @@ public:
     ObjectDesc new_obj = get_most_recent(oid);
     new_obj.header = bufferlist();
     new_obj.dirty = true;
+    new_obj.flushed = false;
     pool_obj_cont[current_snap].insert_or_assign(oid, new_obj);
   }
 
@@ -403,6 +442,7 @@ public:
     new_obj.header = bl;
     new_obj.exists = true;
     new_obj.dirty = true;
+    new_obj.flushed = false;
     pool_obj_cont[current_snap].insert_or_assign(oid, new_obj);
   }
 
@@ -416,6 +456,7 @@ public:
     }
     new_obj.exists = true;
     new_obj.dirty = true;
+    new_obj.flushed = false;
     pool_obj_cont[current_snap].insert_or_assign(oid, new_obj);
   }
 
@@ -425,6 +466,7 @@ public:
     ObjectDesc new_obj = get_most_recent(oid);
     new_obj.exists = true;
     new_obj.dirty = true;
+    new_obj.flushed = false;
     new_obj.update(cont_gen,
                   contents);
     pool_obj_cont[current_snap].insert_or_assign(oid, new_obj);
@@ -549,8 +591,118 @@ public:
     ObjectDesc contents;
     find_object(oid, &contents, snap);
     contents.dirty = true;
+    contents.flushed = false;
     pool_obj_cont.rbegin()->second.insert_or_assign(oid, contents);
   }
+
+  void update_object_tier_flushed(const string &oid, int snap)
+  {
+    for (map<int, map<string,ObjectDesc> >::reverse_iterator i = 
+          pool_obj_cont.rbegin();
+        i != pool_obj_cont.rend();
+        ++i) {
+      if (snap != -1 && snap < i->first)
+       continue;
+      map<string,ObjectDesc>::iterator j = i->second.find(oid);
+      if (j != i->second.end()) {
+       j->second.flushed = true;
+       break;
+      }
+    }
+  }
+
+  bool check_oldest_snap_flushed(const string &oid, int snap) 
+  {
+    for (map<int, map<string,ObjectDesc> >::reverse_iterator i = 
+          pool_obj_cont.rbegin();
+        i != pool_obj_cont.rend();
+        ++i) {
+      if (snap != -1 && snap < i->first)
+       continue;
+      map<string,ObjectDesc>::iterator j = i->second.find(oid);
+      if (j != i->second.end() && !j->second.flushed) {
+       cout << __func__ << " oid " << oid
+            << " v " << j->second.version << " " << j->second.most_recent()
+            << " " << (j->second.flushed ? "flushed" : "unflushed")
+            << " " << i->first << std::endl;
+       return false;
+      }
+    }
+    return true;
+  }
+
+  bool check_chunks_refcount(librados::IoCtx &chunk_pool_ctx, librados::IoCtx &manifest_pool_ctx)
+  {
+    librados::ObjectCursor shard_start;
+    librados::ObjectCursor shard_end;
+    librados::ObjectCursor begin;
+    librados::ObjectCursor end;
+    begin = chunk_pool_ctx.object_list_begin();
+    end = chunk_pool_ctx.object_list_end();
+
+    chunk_pool_ctx.object_list_slice(
+      begin,
+      end,
+      1,
+      1,
+      &shard_start,
+      &shard_end);
+
+    librados::ObjectCursor c(shard_start);
+    while(c < shard_end)
+    {
+      std::vector<librados::ObjectItem> result;
+      int r = chunk_pool_ctx.object_list(c, shard_end, 12, {}, &result, &c);
+      if (r < 0) {
+       cerr << "error object_list : " << cpp_strerror(r) << std::endl;
+       return false;
+      }
+
+      for (const auto & i : result) {
+       auto oid = i.oid;
+       chunk_refs_t refs;
+       {
+         bufferlist t;
+         r = chunk_pool_ctx.getxattr(oid, CHUNK_REFCOUNT_ATTR, t);
+         if (r < 0) {
+           continue;
+         }
+         auto p = t.cbegin();
+         decode(refs, p);
+       }
+       ceph_assert(refs.get_type() == chunk_refs_t::TYPE_BY_OBJECT);
+
+       chunk_refs_by_object_t *byo =
+         static_cast<chunk_refs_by_object_t*>(refs.r.get());
+
+       for (auto& pp : byo->by_object) {
+         int src_refcount = 0;
+         int dst_refcount = byo->by_object.count(pp);
+         for (int tries = 0; tries < 10; tries++) {
+           r = cls_cas_references_chunk(manifest_pool_ctx, pp.oid.name, oid);
+           if (r == -ENOENT || r == -ENOLINK) {
+             src_refcount = 0;
+           } else if (r == -EBUSY) {
+             sleep(10);
+             continue;
+           } else {
+             src_refcount = r;
+           }
+           break;
+         }
+         if (src_refcount > dst_refcount) {
+           cerr << " src_object " << pp
+                << ": src_refcount " << src_refcount 
+                << ", dst_object " << oid 
+                << ": dst_refcount " << dst_refcount 
+                << std::endl;
+           return false;
+         }
+       }
+      }
+    }
+    return true;
+  }
 };
 
 void read_callback(librados::completion_t comp, void *arg);
@@ -2287,48 +2439,103 @@ public:
 
 class SetChunkOp : public TestOp {
 public:
-  string oid, oid_tgt, tgt_pool_name;
+  string oid, oid_tgt;
   ObjectDesc src_value, tgt_value;
   librados::ObjectReadOperation op;
-  librados::ObjectReadOperation rd_op;
   librados::AioCompletion *comp;
-  std::shared_ptr<int> in_use;
   int done;
   int r;
   uint64_t offset;
   uint32_t length;
-  uint64_t tgt_offset;
+  uint32_t tgt_offset;
+  int snap;
+  std::shared_ptr<int> in_use;
   SetChunkOp(int n,
             RadosTestContext *context,
             const string &oid,
-            uint64_t offset,
-            uint32_t length,
             const string &oid_tgt,
-            const string &tgt_pool_name,
-            uint64_t tgt_offset,
             TestOpStat *stat = 0)
     : TestOp(n, context, stat),
-      oid(oid), oid_tgt(oid_tgt), tgt_pool_name(tgt_pool_name),
+      oid(oid), oid_tgt(oid_tgt),
       comp(NULL), done(0), 
-      r(0), offset(offset), length(length), 
-      tgt_offset(tgt_offset)
+      r(0), offset(0), length(0), 
+      tgt_offset(0),
+      snap(0)
   {}
 
+  pair<uint64_t, uint64_t> get_rand_off_len(uint32_t max_len) {
+    pair<uint64_t, uint64_t> r (0, 0);
+    r.first = rand() % max_len;
+    r.second = rand() % max_len;
+    r.first = r.first - (r.first % 512);
+    r.second = r.second - (r.second % 512);
+
+    while (r.first + r.second > max_len || r.second == 0) {
+      r.first = rand() % max_len;
+      r.second = rand() % max_len;
+      r.first = r.first - (r.first % 512);
+      r.second = r.second - (r.second % 512);
+    }
+    return r;
+  }
+
   void _begin() override
   {
     std::lock_guard l{context->state_lock};
+    if (!(rand() % 4) && !context->snaps.empty()) {
+      snap = rand_choose(context->snaps)->first;
+      in_use = context->snaps_in_use.lookup_or_create(snap, snap);
+    } else {
+      snap = -1;
+    }
     context->oid_in_use.insert(oid);
     context->oid_not_in_use.erase(oid);
 
-    if (tgt_pool_name.empty()) ceph_abort();
-
-    context->find_object(oid, &src_value); 
+    context->find_object(oid, &src_value, snap); 
     context->find_object(oid_tgt, &tgt_value);
 
-    if (src_value.version != 0 && !src_value.deleted())
+    uint32_t max_len = 0;
+    if (src_value.deleted()) {
+      /* just random length to check ENOENT */
+      max_len = context->max_size;
+    } else {
+      max_len = src_value.most_recent_gen()->get_length(src_value.most_recent());
+    }
+    pair<uint64_t, uint64_t> off_len; // first: offset, second: length
+    if (snap >= 0) {
+      context->io_ctx.snap_set_read(context->snaps[snap]);
+      off_len = get_rand_off_len(max_len);
+    } else if (src_value.version != 0 && !src_value.deleted()) {
       op.assert_version(src_value.version);
+      off_len = get_rand_off_len(max_len);
+    } else if (src_value.deleted()) {
+      off_len.first = 0;
+      off_len.second = max_len;
+    }
+    offset = off_len.first;
+    length = off_len.second;
+    tgt_offset = offset;
+
+    string target_oid;
+    if (!src_value.deleted() && oid_tgt.empty()) {
+      bufferlist bl;
+      int r = context->io_ctx.read(context->prefix+oid, bl, length, offset);
+      ceph_assert(r > 0);
+      string fp_oid = ceph::crypto::digest<ceph::crypto::SHA256>(bl).to_str();
+      r = context->low_tier_io_ctx.write(fp_oid, bl, bl.length(), 0);
+      ceph_assert(r == 0);
+      target_oid = fp_oid;
+      tgt_offset = 0;
+    } else {
+      target_oid = context->prefix+oid_tgt;
+    }
+
+    cout << num << ": " << "set_chunk oid " << oid << " offset: " << offset
+         << " length: " << length <<  " target oid " << target_oid
+         << " offset: " << tgt_offset << " snap " << snap << std::endl;
+
     op.set_chunk(offset, length, context->low_tier_io_ctx, 
-                context->prefix+oid_tgt, tgt_offset, CEPH_OSD_OP_FLAG_WITH_REFERENCE);
+                target_oid, tgt_offset, CEPH_OSD_OP_FLAG_WITH_REFERENCE);
 
     pair<TestOp*, TestOp::CallbackInfo*> *cb_arg =
       new pair<TestOp*, TestOp::CallbackInfo*>(this,
@@ -2337,6 +2544,9 @@ public:
                                                &write_callback);
     context->io_ctx.aio_operate(context->prefix+oid, comp, &op,
                                librados::OPERATION_ORDER_READS_WRITES, NULL);
+    if (snap >= 0) {
+      context->io_ctx.snap_set_read(0);
+    }
   }
 
   void _finish(CallbackInfo *info) override
@@ -2349,39 +2559,31 @@ public:
       if ((r = comp->get_return_value())) {
        if (r == -ENOENT && src_value.deleted()) {
          cout << num << ":  got expected ENOENT (src dne)" << std::endl;
+       } else if (r == -ENOENT && context->oid_set_chunk_tgt_pool.find(oid_tgt) != 
+                 context->oid_set_chunk_tgt_pool.end()) {
+         cout << num << ": get expected ENOENT tgt oid " << oid_tgt << std::endl;
+       } else if (r == -ERANGE && src_value.deleted()) {
+         cout << num << ":  got expected ERANGE (src dne)" << std::endl;
        } else if (r == -EOPNOTSUPP) {
-         bool is_overlapped = false;
-         for (auto &p : src_value.chunk_info) {
-           if ((p.first <= offset && p.first + p.second.length > offset) ||
-                (p.first > offset && p.first <= offset + length)) {
-             cout << " range is overlapped  offset: " << offset << " length: " << length
-                   << " chunk_info offset: " << p.second.offset << " length " 
-                   << p.second.length << std::endl;
-             is_overlapped = true;
-             context->update_object_version(oid, comp->get_version64());
-           } 
-         }
-         if (!is_overlapped) {
-           cerr << "Error: oid " << oid << " set_chunk " << oid_tgt << " returned error code "
-                 << r << " offset: " << offset << " length: " << length <<  std::endl;
-           ceph_abort();
-         }
+         cout << "Range is overlapped: oid " << oid << " set_chunk " << oid_tgt << " returned error code "
+               << r << " offset: " << offset << " length: " << length <<  std::endl;
+         context->update_object_version(oid, comp->get_version64());
        } else {
          cerr << "Error: oid " << oid << " set_chunk " << oid_tgt << " returned error code "
               << r << std::endl;
          ceph_abort();
        }
       } else {
-       ChunkDesc info;
-       info.offset = tgt_offset;
-       info.length = length;
-       info.oid = oid_tgt;
-       context->update_object_chunk_target(oid, offset, info);
-       context->update_object_version(oid, comp->get_version64());
+       if (snap == -1) {
+         ChunkDesc info {tgt_offset, length, oid_tgt};
+         context->update_object_chunk_target(oid, offset, info);
+         context->update_object_version(oid, comp->get_version64());
+       }
       }
     }
 
     if (++done == 1) {
+      context->oid_set_chunk_tgt_pool.insert(oid_tgt);
       context->oid_in_use.erase(oid);
       context->oid_not_in_use.insert(oid);
       context->kick();
@@ -2618,6 +2820,7 @@ public:
   librados::ObjectWriteOperation op;
   string oid;
   std::shared_ptr<int> in_use;
+  ObjectDesc src_value;
 
   TierPromoteOp(int n,
               RadosTestContext *context,
@@ -2635,6 +2838,8 @@ public:
     context->oid_in_use.insert(oid);
     context->oid_not_in_use.erase(oid);
 
+    context->find_object(oid, &src_value); 
+
     pair<TestOp*, TestOp::CallbackInfo*> *cb_arg =
       new pair<TestOp*, TestOp::CallbackInfo*>(this,
                                               new TestOp::CallbackInfo(0));
@@ -2660,6 +2865,8 @@ public:
     cout << num << ":  got " << cpp_strerror(r) << std::endl;
     if (r == 0) {
       // sucess
+    } else if (r == -ENOENT && src_value.deleted()) {
+      cout << num << ":  got expected ENOENT (src dne)" << std::endl;
     } else {
       ceph_abort_msg("shouldn't happen");
     }
@@ -2688,6 +2895,9 @@ public:
   librados::ObjectReadOperation op;
   string oid;
   std::shared_ptr<int> in_use;
+  int snap;
+  ObjectDesc src_value;
+
 
   TierFlushOp(int n,
               RadosTestContext *context,
@@ -2695,7 +2905,8 @@ public:
               TestOpStat *stat)
     : TestOp(n, context, stat),
       completion(NULL),
-      oid(oid)
+      oid(oid),
+      snap(-1)
   {}
 
   void _begin() override
@@ -2705,6 +2916,21 @@ public:
     context->oid_in_use.insert(oid);
     context->oid_not_in_use.erase(oid);
 
+    if (0 && !(rand() % 4) && !context->snaps.empty()) {
+      snap = rand_choose(context->snaps)->first;
+      in_use = context->snaps_in_use.lookup_or_create(snap, snap);
+    } else {
+      snap = -1;
+    }
+
+    cout << num << ": tier_flush oid " << oid << " snap " << snap << std::endl;
+
+    if (snap >= 0) {
+      context->io_ctx.snap_set_read(context->snaps[snap]);
+    }
+
+    context->find_object(oid, &src_value, snap); 
+
     pair<TestOp*, TestOp::CallbackInfo*> *cb_arg =
       new pair<TestOp*, TestOp::CallbackInfo*>(this,
                                               new TestOp::CallbackInfo(0));
@@ -2717,6 +2943,10 @@ public:
     int r = context->io_ctx.aio_operate(context->prefix+oid, completion,
                                        &op, flags, NULL);
     ceph_assert(!r);
+
+    if (snap >= 0) {
+      context->io_ctx.snap_set_read(0);
+    }
   }
 
   void _finish(CallbackInfo *info) override
@@ -2729,10 +2959,25 @@ public:
     cout << num << ":  got " << cpp_strerror(r) << std::endl;
     if (r == 0) {
       // sucess
+      context->update_object_tier_flushed(oid, snap);
+      context->update_object_version(oid, completion->get_version64(), snap);
+    } else if (r == -EBUSY) {
+      // could fail if snap is not oldest
+      ceph_assert(!context->check_oldest_snap_flushed(oid, snap)); 
+    } else if (r == -ENOENT) {
+      // could fail if object is removed
+      if (src_value.deleted()) {
+       cout << num << ":  got expected ENOENT (src dne)" << std::endl;
+      } else {
+       cerr << num << ": got unexpected ENOENT" << std::endl;
+       ceph_abort();
+      }
     } else {
+      if (r != -ENOENT && src_value.deleted()) {
+       cerr << num << ": src dne, but r is not ENOENT" << std::endl;
+      }        
       ceph_abort_msg("shouldn't happen");
     }
-    context->update_object_version(oid, completion->get_version64());
     context->oid_in_use.erase(oid);
     context->oid_not_in_use.insert(oid);
     context->kick();
@@ -2751,6 +2996,110 @@ public:
   }
 };
 
+class TierEvictOp : public TestOp {
+public:
+  librados::AioCompletion *completion;
+  librados::ObjectReadOperation op;
+  string oid;
+  std::shared_ptr<int> in_use;
+  int snap;
+  ObjectDesc src_value;
+
+  TierEvictOp(int n,
+              RadosTestContext *context,
+              const string &oid,
+              TestOpStat *stat)
+    : TestOp(n, context, stat),
+      completion(NULL),
+      oid(oid),
+      snap(-1)
+  {}
+
+  void _begin() override
+  {
+    context->state_lock.lock();
+
+    context->oid_in_use.insert(oid);
+    context->oid_not_in_use.erase(oid);
+
+    if (0 && !(rand() % 4) && !context->snaps.empty()) {
+      snap = rand_choose(context->snaps)->first;
+      in_use = context->snaps_in_use.lookup_or_create(snap, snap);
+    } else {
+      snap = -1;
+    }
+
+    cout << num << ": tier_evict oid " << oid << " snap " << snap << std::endl;
+
+    if (snap >= 0) {
+      context->io_ctx.snap_set_read(context->snaps[snap]);
+    }
+
+    context->find_object(oid, &src_value, snap); 
+
+    pair<TestOp*, TestOp::CallbackInfo*> *cb_arg =
+      new pair<TestOp*, TestOp::CallbackInfo*>(this,
+                                              new TestOp::CallbackInfo(0));
+    completion = context->rados.aio_create_completion((void *) cb_arg,
+                                                     &write_callback);
+    context->state_lock.unlock();
+
+    op.cache_evict();
+    int r = context->io_ctx.aio_operate(context->prefix+oid, completion,
+                                       &op, librados::OPERATION_IGNORE_CACHE,
+                                       NULL);
+    ceph_assert(!r);
+
+    if (snap >= 0) {
+      context->io_ctx.snap_set_read(0);
+    }
+  }
+
+  void _finish(CallbackInfo *info) override
+  {
+    std::lock_guard state_locker{context->state_lock};
+    ceph_assert(!done);
+    ceph_assert(completion->is_complete());
+
+    int r = completion->get_return_value();
+    cout << num << ":  got " << cpp_strerror(r) << std::endl;
+    if (r == 0) {
+      // ok
+    } else if (r == -EINVAL) {
+      // modifying manifest object makes existing chunk_map clear
+      // as a result, the modified object is no longer manifest object 
+      // this casues to return -EINVAL
+    } else if (r == -ENOENT) {
+      // could fail if object is removed
+      if (src_value.deleted()) {
+       cout << num << ":  got expected ENOENT (src dne)" << std::endl;
+      } else {
+       cerr << num << ": got unexpected ENOENT" << std::endl;
+       ceph_abort();
+      }
+    } else {
+      if (r != -ENOENT && src_value.deleted()) {
+       cerr << num << ": src dne, but r is not ENOENT" << std::endl;
+      }        
+      ceph_abort_msg("shouldn't happen");
+    }
+    context->oid_in_use.erase(oid);
+    context->oid_not_in_use.insert(oid);
+    context->kick();
+    done = true;
+  }
+
+  bool finished() override
+  {
+    return done;
+  }
+
+  string getType() override
+  {
+    return "TierEvictOp";
+  }
+};
+
 class HitSetListOp : public TestOp {
   librados::AioCompletion *comp1, *comp2;
   uint32_t hash;