policy, obj);
}
-static int get_obj_attrs(RGWRados *store, struct req_state *s, rgw_obj& obj, map<string, bufferlist>& attrs,
+static int get_obj_attrs(RGWRados *store, struct req_state *s, rgw_obj& obj, map<string, bufferlist>& attrs)
+{
+ RGWRados::Object op_target(store, *(RGWRados::ObjectCtx *)s->obj_ctx, obj);
+ RGWRados::Object::Read read_op(&op_target);
+
+ read_op.params.attrs = &attrs;
+ read_op.params.perr = &s->err;
+
+ return read_op.prepare(NULL, NULL);
+}
+
+static int get_system_obj_attrs(RGWRados *store, struct req_state *s, rgw_obj& obj, map<string, bufferlist>& attrs,
uint64_t *obj_size, RGWObjVersionTracker *objv_tracker)
{
void *handle;
{
ldout(s->cct, 20) << "user manifest obj=" << ent.key.name << "[" << ent.key.instance << "]" << dendl;
- void *handle = NULL;
- off_t cur_ofs = start_ofs;
- off_t cur_end = end_ofs;
+ uint64_t cur_ofs = start_ofs;
+ uint64_t cur_end = end_ofs;
utime_t start_time = s->time;
rgw_obj part(bucket, ent.key);
map<string, bufferlist> attrs;
uint64_t obj_size;
- void *obj_ctx = store->create_context(s);
+ RGWRados::ObjectCtx obj_ctx(store);
RGWAccessControlPolicy obj_policy(s->cct);
ldout(s->cct, 20) << "reading obj=" << part << " ofs=" << cur_ofs << " end=" << cur_end << dendl;
- store->set_atomic(obj_ctx, part);
- store->set_prefetch_data(obj_ctx, part);
- ret = store->prepare_get_obj(obj_ctx, part, &cur_ofs, &cur_end, &attrs, NULL,
- NULL, NULL, NULL, NULL, NULL, &obj_size, NULL, &handle, &s->err);
+ obj_ctx.set_atomic(part);
+ store->set_prefetch_data(&obj_ctx, part);
+
+ RGWRados::Object op_target(store, obj_ctx, part);
+ RGWRados::Object::Read read_op(&op_target);
+
+ read_op.params.attrs = &attrs;
+ read_op.params.obj_size = &obj_size;
+ read_op.params.perr = &s->err;
+
+ ret = read_op.prepare(&cur_ofs, &cur_end);
if (ret < 0)
- goto done_err;
+ return ret;
if (obj_size != ent.size) {
// hmm.. something wrong, object not as expected, abort!
ldout(s->cct, 0) << "ERROR: expected obj_size=" << obj_size << ", actual read size=" << ent.size << dendl;
- ret = -EIO;
- goto done_err;
+ return -EIO;
}
ret = rgw_policy_from_attrset(s->cct, attrs, &obj_policy);
if (ret < 0)
- goto done_err;
+ return ret;
if (!verify_object_permission(s, bucket_policy, &obj_policy, RGW_PERM_READ)) {
- ret = -EPERM;
- goto done_err;
+ return -EPERM;
}
perfcounter->inc(l_rgw_get_b, cur_end - cur_ofs);
while (cur_ofs <= cur_end) {
bufferlist bl;
- ret = store->get_obj(obj_ctx, NULL, &handle, part, bl, cur_ofs, cur_end, NULL);
+ ret = read_op.read(cur_ofs, cur_end, bl);
if (ret < 0)
- goto done_err;
+ return ret;
off_t len = bl.length();
cur_ofs += len;
start_time = ceph_clock_now(s->cct);
}
- store->destroy_context(obj_ctx);
- obj_ctx = NULL;
-
- store->finish_get_obj(&handle);
-
return 0;
-
-done_err:
- if (obj_ctx)
- store->destroy_context(obj_ctx);
- return ret;
}
static int iterate_user_manifest_parts(CephContext *cct, RGWRados *store, off_t ofs, off_t end,
map<string, bufferlist>::iterator attr_iter;
perfcounter->inc(l_rgw_get);
- off_t new_ofs, new_end;
+ uint64_t new_ofs, new_end;
+
+ RGWRados::Object op_target(store, *(RGWRados::ObjectCtx *)s->obj_ctx, obj);
+ RGWRados::Object::Read read_op(&op_target);
ret = get_params();
if (ret < 0)
new_ofs = ofs;
new_end = end;
- ret = store->prepare_get_obj(s->obj_ctx, obj, &new_ofs, &new_end, &attrs, mod_ptr,
- unmod_ptr, &lastmod, if_match, if_nomatch, &total_len, &s->obj_size, NULL, &handle, &s->err);
+ read_op.conds.mod_ptr = mod_ptr;
+ read_op.conds.unmod_ptr = unmod_ptr;
+ read_op.conds.if_match = if_match;
+ read_op.conds.if_nomatch = if_nomatch;
+ read_op.params.attrs = &attrs;
+ read_op.params.lastmod = &lastmod;
+ read_op.params.read_size = &total_len;
+ read_op.params.obj_size = &s->obj_size;
+ read_op.params.perr = &s->err;
+
+ ret = read_op.prepare(&new_ofs, &new_end);
if (ret < 0)
goto done_err;
rgw_get_request_metadata(s->cct, s->info, attrs);
- /* no need to track object versioning, need it for bucket's data only */
- RGWObjVersionTracker *ptracker = (!s->object.empty() ? NULL : &s->bucket_info.objv_tracker);
+ RGWObjVersionTracker *ptracker = NULL;
- if (!s->object.empty()) {
+ bool is_object_op = (!s->object.empty());
+
+ if (is_object_op) {
/* check if obj exists, read orig attrs */
- ret = get_obj_attrs(store, s, obj, orig_attrs, NULL, ptracker);
+ ret = get_obj_attrs(store, s, obj, orig_attrs);
if (ret < 0)
return;
} else {
+ ptracker = &s->bucket_info.objv_tracker;
orig_attrs = s->bucket_attrs;
- }
- if (s->object.empty() && !placement_rule.empty()) {
- if (placement_rule != s->bucket_info.placement_rule) {
+ if (!placement_rule.empty() &&
+ placement_rule != s->bucket_info.placement_rule) {
ret = -EEXIST;
return;
}
cors_config.encode(cors_bl);
attrs[RGW_ATTR_CORS] = cors_bl;
}
- if (!s->object.empty()) {
+ if (is_object_op) {
ret = store->set_attrs(s->obj_ctx, obj, attrs, &rmattrs, ptracker);
} else {
ret = rgw_bucket_set_attrs(store, s->bucket_info, attrs, &rmattrs, ptracker);
map<string, bufferlist> orig_attrs, attrs, rmattrs;
map<string, bufferlist>::iterator iter;
- RGWObjVersionTracker *ptracker = (!s->object.empty() ? NULL : &s->bucket_info.objv_tracker);
+ bool is_object_op = (!s->object.empty());
+ RGWObjVersionTracker *ptracker = NULL;
- /* check if obj exists, read orig attrs */
- ret = get_obj_attrs(store, s, obj, orig_attrs, NULL, ptracker);
- if (ret < 0)
- return;
+
+ if (is_object_op) {
+ /* check if obj exists, read orig attrs */
+ ret = get_obj_attrs(store, s, obj, orig_attrs);
+ if (ret < 0)
+ return;
+ } else {
+ ptracker = (!s->object.empty() ? NULL : &s->bucket_info.objv_tracker);
+ ret = get_system_obj_attrs(store, s, obj, orig_attrs, NULL, ptracker);
+ if (ret < 0)
+ return;
+ }
/* only remove meta attrs */
for (iter = orig_attrs.begin(); iter != orig_attrs.end(); ++iter) {
attrs[name] = iter->second;
}
}
- ret = store->set_attrs(s->obj_ctx, obj, attrs, &rmattrs, ptracker);
+ if (is_object_op) {
+ ret = store->set_attrs(s->obj_ctx, obj, attrs, &rmattrs, ptracker);
+ } else {
+ ret = rgw_bucket_set_attrs(store, s->bucket_info, attrs, &rmattrs, ptracker);
+ }
}
void RGWOptionsCORS::get_response_params(string& hdrs, string& exp_hdrs, unsigned *max_age) {
obj.init_ns(s->bucket, meta_oid, mp_ns);
obj.set_in_extra_data(true);
- int ret = get_obj_attrs(store, s, obj, attrs, NULL, NULL);
+ int ret = get_obj_attrs(store, s, obj, attrs);
if (ret < 0)
return ret;
meta_obj.init_ns(s->bucket, meta_oid, mp_ns);
meta_obj.set_in_extra_data(true);
- ret = get_obj_attrs(store, s, meta_obj, attrs, NULL, NULL);
+ ret = get_obj_attrs(store, s, meta_obj, attrs);
if (ret < 0) {
ldout(s->cct, 0) << "ERROR: failed to get obj attrs, obj=" << meta_obj << " ret=" << ret << dendl;
return;
}
}
-class GetObjHandleDestructor {
- RGWRados *store;
- void **handle;
-
-public:
- GetObjHandleDestructor(RGWRados *_store) : store(_store), handle(NULL) {}
- ~GetObjHandleDestructor() {
- if (handle) {
- store->finish_get_obj(handle);
- }
- }
- void set_handle(void **_h) {
- handle = _h;
- }
-};
-
int RGWRados::rewrite_obj(RGWBucketInfo& dest_bucket_info, rgw_obj& obj)
{
map<string, bufferlist> attrset;
- off_t ofs = 0;
- off_t end = -1;
- void *handle = NULL;
+ int64_t ofs = 0;
+ int64_t end = -1;
time_t mtime;
uint64_t total_len;
uint64_t obj_size;
ObjectCtx rctx(this);
- int ret = prepare_get_obj((void *)&rctx, obj, &ofs, &end, &attrset,
- NULL, NULL, &mtime, NULL, NULL, &total_len,
- &obj_size, NULL, &handle, NULL);
+
+ RGWRados::Object op_target(this, rctx, obj);
+ RGWRados::Object::Read read_op(&op_target);
+
+ read_op.params.attrs = &attrset;
+ read_op.params.lastmod = &mtime;
+ read_op.params.read_size = &total_len;
+ read_op.params.obj_size = &obj_size;
+
+ int ret = read_op.prepare(&ofs, &end);
if (ret < 0)
return ret;
return ret;
}
- return copy_obj_data(rctx, dest_bucket_info, &handle, end, obj, obj, max_chunk_size, NULL, mtime, attrset, RGW_OBJ_CATEGORY_MAIN, NULL, NULL, NULL);
+ return copy_obj_data(rctx, dest_bucket_info, read_op, end, obj, obj, max_chunk_size, NULL, mtime, attrset, RGW_OBJ_CATEGORY_MAIN, NULL, NULL, NULL);
}
/**
ldout(cct, 5) << "Copy object " << src_obj.bucket << ":" << src_obj.get_object() << " => " << dest_obj.bucket << ":" << dest_obj.get_object() << dendl;
- void *handle = NULL;
- GetObjHandleDestructor handle_destructor(this);
-
map<string, bufferlist> src_attrs;
- off_t ofs = 0;
- off_t end = -1;
+ int64_t ofs = 0;
+ int64_t end = -1;
+ RGWRados::Object src_op_target(this, obj_ctx, src_obj);
+ RGWRados::Object::Read read_op(&src_op_target);
+
if (!remote_src && source_zone.empty()) {
- ret = prepare_get_obj(&obj_ctx, src_obj, &ofs, &end, &src_attrs,
- mod_ptr, unmod_ptr, &lastmod, if_match, if_nomatch, &total_len, &obj_size, NULL, &handle, err);
+ read_op.conds.mod_ptr = mod_ptr;
+ read_op.conds.unmod_ptr = unmod_ptr;
+ read_op.conds.if_match = if_match;
+ read_op.conds.if_nomatch = if_nomatch;
+ read_op.params.attrs = &src_attrs;
+ read_op.params.lastmod = &lastmod;
+ read_op.params.read_size = &total_len;
+ read_op.params.obj_size = &obj_size;
+ read_op.params.perr = err;
+
+ ret = read_op.prepare(&ofs, &end);
if (ret < 0)
return ret;
-
- handle_destructor.set_handle(&handle);
} else {
/* source is in a different region, copy it there */
if (ret < 0)
return ret;
- ret = get_obj_iterate(&obj_ctx, &handle, src_obj, 0, astate->size - 1, out_stream_req->get_out_cb());
+ ret = get_obj_iterate(&obj_ctx, read_op, src_obj, 0, astate->size - 1, out_stream_req->get_out_cb());
if (ret < 0)
return ret;
}
if (copy_data) { /* refcounting tail wouldn't work here, just copy the data */
- return copy_obj_data(obj_ctx, dest_bucket_info, &handle, end, dest_obj, src_obj,
+ return copy_obj_data(obj_ctx, dest_bucket_info, read_op, end, dest_obj, src_obj,
max_chunk_size, mtime, 0, src_attrs, category, ptag, petag, err);
}
RGWObjManifest *pmanifest;
ldout(cct, 0) << "dest_obj=" << dest_obj << " src_obj=" << src_obj << " copy_itself=" << (int)copy_itself << dendl;
- RGWRados::Object op_target(this, obj_ctx, dest_obj);
- RGWRados::Object::Write obj_op(&op_target);
+ RGWRados::Object dest_op_target(this, obj_ctx, dest_obj);
+ RGWRados::Object::Write write_op(&dest_op_target);
string tag;
}
if (copy_first) {
- ret = get_obj(&obj_ctx, NULL, &handle, src_obj, first_chunk, 0, max_chunk_size, NULL);
+ ret = read_op.read(0, max_chunk_size, first_chunk);
if (ret < 0)
goto done_ret;
pmanifest->set_head_size(first_chunk.length());
}
- obj_op.meta.data = &first_chunk;
- obj_op.meta.manifest = pmanifest;
- obj_op.meta.ptag = &tag;
- obj_op.meta.owner = dest_bucket_info.owner;
- obj_op.meta.mtime = mtime;
- obj_op.meta.flags = PUT_OBJ_CREATE;
- obj_op.meta.category = category;
+ write_op.meta.data = &first_chunk;
+ write_op.meta.manifest = pmanifest;
+ write_op.meta.ptag = &tag;
+ write_op.meta.owner = dest_bucket_info.owner;
+ write_op.meta.mtime = mtime;
+ write_op.meta.flags = PUT_OBJ_CREATE;
+ write_op.meta.category = category;
- ret = obj_op.write_meta(end + 1, attrs);
+ ret = write_op.write_meta(end + 1, attrs);
if (ret < 0)
goto done_ret;
int RGWRados::copy_obj_data(ObjectCtx& obj_ctx,
RGWBucketInfo& dest_bucket_info,
- void **handle, off_t end,
+ RGWRados::Object::Read& read_op, off_t end,
rgw_obj& dest_obj,
rgw_obj& src_obj,
uint64_t max_chunk_size,
do {
bufferlist bl;
- ret = get_obj(&obj_ctx, NULL, handle, src_obj, bl, ofs, end, NULL);
- if (ret < 0)
- return ret;
+ ret = read_op.read(ofs, end, bl);
uint64_t read_len = ret;
bool again;
return 0;
}
+/**
+ * Get data about an object out of RADOS and into memory.
+ * bucket: name of the bucket the object is in.
+ * obj: name/key of the object to read
+ * data: if get_data==true, this pointer will be set
+ * to an address containing the object's data/value
+ * ofs: the offset of the object to read from
+ * end: the point in the object to stop reading
+ * attrs: if non-NULL, the pointed-to map will contain
+ * all the attrs of the object when this function returns
+ * mod_ptr: if non-NULL, compares the object's mtime to *mod_ptr,
+ * and if mtime is smaller it fails.
+ * unmod_ptr: if non-NULL, compares the object's mtime to *unmod_ptr,
+ * and if mtime is >= it fails.
+ * if_match/nomatch: if non-NULL, compares the object's etag attr
+ * to the string and, if it doesn't/does match, fails out.
+ * get_data: if true, the object's data/value will be read out, otherwise not
+ * err: Many errors will result in this structure being filled
+ * with extra informatin on the error.
+ * Returns: -ERR# on failure, otherwise
+ * (if get_data==true) length of read data,
+ * (if get_data==false) length of the object
+ */
+int RGWRados::Object::Read::prepare(int64_t *pofs, int64_t *pend)
+{
+ RGWRados *store = source->get_store();
+ rgw_obj& obj = source->get_obj();
+ CephContext *cct = store->ctx();
+
+ bufferlist etag;
+ time_t ctime;
+
+ off_t ofs = 0;
+ off_t end = -1;
+
+ map<string, bufferlist>::iterator iter;
+
+ int r = store->get_obj_ioctx(obj, &state.io_ctx);
+ if (r < 0) {
+ return r;
+ }
+
+ RGWObjState *astate;
+ r = source->get_state(&astate);
+ if (r < 0)
+ return r;
+
+ if (!astate->exists) {
+ return -ENOENT;
+ }
+
+ if (params.attrs) {
+ *params.attrs = astate->attrset;
+ if (cct->_conf->subsys.should_gather(ceph_subsys_rgw, 20)) {
+ for (iter = params.attrs->begin(); iter != params.attrs->end(); ++iter) {
+ ldout(cct, 20) << "Read xattr: " << iter->first << dendl;
+ }
+ }
+ if (r < 0)
+ return r;
+ }
+
+ /* Convert all times go GMT to make them compatible */
+ if (conds.mod_ptr || conds.unmod_ptr) {
+ ctime = astate->mtime;
+
+ if (conds.mod_ptr) {
+ ldout(cct, 10) << "If-Modified-Since: " << *conds.mod_ptr << " Last-Modified: " << ctime << dendl;
+ if (ctime < *conds.mod_ptr) {
+ return -ERR_NOT_MODIFIED;
+ }
+ }
+
+ if (conds.unmod_ptr) {
+ ldout(cct, 10) << "If-UnModified-Since: " << *conds.unmod_ptr << " Last-Modified: " << ctime << dendl;
+ if (ctime > *conds.unmod_ptr) {
+ return -ERR_PRECONDITION_FAILED;
+ }
+ }
+ }
+ if (conds.if_match || conds.if_nomatch) {
+ r = get_attr(source->get_ctx(), obj, RGW_ATTR_ETAG, etag);
+ if (r < 0)
+ return r;
+
+ if (conds.if_match) {
+ string if_match_str = rgw_string_unquote(conds.if_match);
+ ldout(cct, 10) << "ETag: " << etag.c_str() << " " << " If-Match: " << if_match_str << dendl;
+ if (if_match_str.compare(etag.c_str()) != 0) {
+ return -ERR_PRECONDITION_FAILED;
+ }
+ }
+
+ if (conds.if_nomatch) {
+ string if_nomatch_str = rgw_string_unquote(conds.if_nomatch);
+ ldout(cct, 10) << "ETag: " << etag.c_str() << " " << " If-NoMatch: " << if_nomatch_str << dendl;
+ if (if_nomatch_str.compare(etag.c_str()) == 0) {
+ return -ERR_NOT_MODIFIED;
+ }
+ }
+ }
+
+ if (pofs)
+ ofs = *pofs;
+ if (pend)
+ end = *pend;
+
+ if (ofs < 0) {
+ ofs += astate->size;
+ if (ofs < 0)
+ ofs = 0;
+ end = astate->size - 1;
+ } else if (end < 0) {
+ end = astate->size - 1;
+ }
+
+ if (astate->size > 0) {
+ if (ofs >= (off_t)astate->size) {
+ return -ERANGE;
+ }
+ if (end >= (off_t)astate->size) {
+ end = astate->size - 1;
+ }
+ }
+
+ if (pofs)
+ *pofs = ofs;
+ if (pend)
+ *pend = end;
+ if (params.read_size)
+ *params.read_size = (ofs <= end ? end + 1 - ofs : 0);
+ if (params.obj_size)
+ *params.obj_size = astate->size;
+ if (params.lastmod)
+ *params.lastmod = astate->mtime;
+
+ return 0;
+}
+
/**
* Get data about an object out of RADOS and into memory.
* bucket: name of the bucket the object is in.
return store->cls_obj_complete_cancel(target->get_bucket(), optag, obj);
}
+int RGWRados::Object::Read::read(int64_t ofs, int64_t end, bufferlist& bl)
+{
+ RGWRados *store = source->get_store();
+ rgw_obj& obj = source->get_obj();
+ CephContext *cct = store->ctx();
+
+ rgw_bucket bucket;
+ std::string oid, key;
+ rgw_obj read_obj = obj;
+ uint64_t read_ofs = ofs;
+ uint64_t len, read_len;
+ bool reading_from_head = true;
+ ObjectReadOperation op;
+
+ bool merge_bl = false;
+ bufferlist *pbl = &bl;
+ bufferlist read_bl;
+ uint64_t max_chunk_size;
+
+
+ get_obj_bucket_and_oid_loc(obj, bucket, oid, key);
+
+ RGWObjState *astate;
+ int r = source->get_state(&astate);
+ if (r < 0)
+ return r;
+
+ if (end < 0)
+ len = 0;
+ else
+ len = end - ofs + 1;
+
+ if (astate->has_manifest && astate->manifest.has_tail()) {
+ /* now get the relevant object part */
+ RGWObjManifest::obj_iterator iter = astate->manifest.obj_find(ofs);
+
+ uint64_t stripe_ofs = iter.get_stripe_ofs();
+ read_obj = iter.get_location();
+ len = min(len, iter.get_stripe_size() - (ofs - stripe_ofs));
+ read_ofs = iter.location_ofs() + (ofs - stripe_ofs);
+ reading_from_head = (read_obj == obj);
+
+ if (!reading_from_head) {
+ get_obj_bucket_and_oid_loc(read_obj, bucket, oid, key);
+ }
+ }
+
+ r = store->get_max_chunk_size(bucket, &max_chunk_size);
+ if (r < 0) {
+ ldout(cct, 0) << "ERROR: failed to get max_chunk_size() for bucket " << bucket << dendl;
+ return r;
+ }
+
+ if (len > max_chunk_size)
+ len = max_chunk_size;
+
+
+ state.io_ctx.locator_set_key(key);
+
+ read_len = len;
+
+ if (reading_from_head) {
+ /* only when reading from the head object do we need to do the atomic test */
+ r = store->append_atomic_test(&source->get_ctx(), read_obj, op, &astate);
+ if (r < 0)
+ return r;
+
+ if (astate && astate->prefetch_data) {
+ if (!ofs && astate->data.length() >= len) {
+ bl = astate->data;
+ return bl.length();
+ }
+
+ if (ofs < astate->data.length()) {
+ unsigned copy_len = min((uint64_t)astate->data.length() - ofs, len);
+ astate->data.copy(ofs, copy_len, bl);
+ read_len -= copy_len;
+ read_ofs += copy_len;
+ if (!read_len)
+ return bl.length();
+
+ merge_bl = true;
+ pbl = &read_bl;
+ }
+ }
+ }
+
+ ldout(cct, 20) << "rados->read obj-ofs=" << ofs << " read_ofs=" << read_ofs << " read_len=" << read_len << dendl;
+ op.read(read_ofs, read_len, pbl, NULL);
+
+ r = state.io_ctx.operate(oid, &op, NULL);
+ ldout(cct, 20) << "rados->read r=" << r << " bl.length=" << bl.length() << dendl;
+
+ if (merge_bl)
+ bl.append(read_bl);
+
+ return 0;
+}
+
int RGWRados::get_obj(void *ctx, RGWObjVersionTracker *objv_tracker, void **handle, rgw_obj& obj,
bufferlist& bl, off_t ofs, off_t end, rgw_cache_entry_info *cache_info)
{
return r;
}
-int RGWRados::get_obj_iterate(void *ctx, void **handle, rgw_obj& obj,
+int RGWRados::get_obj_iterate(void *ctx, RGWRados::Object::Read& read_op, rgw_obj& obj,
off_t ofs, off_t end,
RGWGetDataCB *cb)
{
struct get_obj_data *data = new get_obj_data(cct);
bool done = false;
- GetObjState *state = *(GetObjState **)handle;
-
data->rados = this;
data->ctx = ctx;
- data->io_ctx.dup(state->io_ctx);
+ data->io_ctx.dup(read_op.state.io_ctx);
data->client_cb = cb;
int r = iterate_obj(ctx, obj, ofs, end, cct->_conf->rgw_get_obj_max_req_size, _get_obj_iterate_cb, (void *)data);
void RGWRados::finish_get_obj(void **handle)
{
+#warning remove me
if (*handle) {
GetObjState *state = *(GetObjState **)handle;
delete state;