summaryrefslogtreecommitdiff
path: root/src/couchdb/couch_db.erl
diff options
context:
space:
mode:
Diffstat (limited to 'src/couchdb/couch_db.erl')
-rw-r--r--src/couchdb/couch_db.erl395
1 files changed, 277 insertions, 118 deletions
diff --git a/src/couchdb/couch_db.erl b/src/couchdb/couch_db.erl
index 92026ff1..46de8745 100644
--- a/src/couchdb/couch_db.erl
+++ b/src/couchdb/couch_db.erl
@@ -17,6 +17,7 @@
-export([open_ref_counted/2,is_idle/1,monitor/1,count_changes_since/2]).
-export([update_doc/3,update_docs/4,update_docs/2,update_docs/3,delete_doc/3]).
-export([get_doc_info/2,open_doc/2,open_doc/3,open_doc_revs/4]).
+-export([set_revs_limit/2,get_revs_limit/1]).
-export([get_missing_revs/2,name/1,doc_to_tree/1,get_update_seq/1,get_committed_update_seq/1]).
-export([enum_docs/4,enum_docs/5,enum_docs_since/4,enum_docs_since/5]).
-export([enum_docs_since_reduce_to_count/1,enum_docs_reduce_to_count/1]).
@@ -184,22 +185,42 @@ get_db_info(Db) ->
],
{ok, InfoList}.
+check_is_admin(#db{admins=Admins, user_ctx=#user_ctx{name=Name,roles=Roles}}) ->
+ DbAdmins = [<<"_admin">> | Admins],
+ case DbAdmins -- [Name | Roles] of
+ DbAdmins -> % same list, not an admin
+ throw({unauthorized, <<"You are not a db or server admin.">>});
+ _ ->
+ ok
+ end.
+
get_admins(#db{admins=Admins}) ->
Admins.
-set_admins(#db{update_pid=UpdatePid,user_ctx=Ctx},
- Admins) when is_list(Admins) ->
- case gen_server:call(UpdatePid, {set_admins, Admins, Ctx}, infinity) of
- ok -> ok;
- Error -> throw(Error)
- end.
+set_admins(#db{update_pid=Pid}=Db, Admins) when is_list(Admins) ->
+ check_is_admin(Db),
+ gen_server:call(Pid, {set_admins, Admins}, infinity).
+
+
+get_revs_limit(#db{revs_limit=Limit}) ->
+ Limit.
+
+set_revs_limit(#db{update_pid=Pid}=Db, Limit) when Limit > 0 ->
+ check_is_admin(Db),
+ gen_server:call(Pid, {set_revs_limit, Limit}, infinity);
+set_revs_limit(_Db, _Limit) ->
+ throw(invalid_revs_limit).
name(#db{name=Name}) ->
Name.
update_doc(Db, Doc, Options) ->
- {ok, [NewRev]} = update_docs(Db, [Doc], Options),
- {ok, NewRev}.
+ case update_docs(Db, [Doc], Options) of
+ {ok, [{ok, NewRev}]} ->
+ {ok, NewRev};
+ {ok, [Error]} ->
+ throw(Error)
+ end.
update_docs(Db, Docs) ->
update_docs(Db, Docs, []).
@@ -227,146 +248,283 @@ group_alike_docs([Doc|Rest], [Bucket|RestBuckets]) ->
validate_doc_update(#db{user_ctx=UserCtx, admins=Admins},
- #doc{id= <<"_design/",_/binary>>}=Doc, _GetDiskDocFun) ->
+ #doc{id= <<"_design/",_/binary>>}, _GetDiskDocFun) ->
UserNames = [UserCtx#user_ctx.name | UserCtx#user_ctx.roles],
% if the user is a server admin or db admin, allow the save
case length(UserNames -- [<<"_admin">> | Admins]) == length(UserNames) of
true ->
% not an admin
- throw({unauthorized, <<"You are not a server or database admin.">>});
+ {unauthorized, <<"You are not a server or database admin.">>};
false ->
- Doc
+ ok
end;
-validate_doc_update(#db{validate_doc_funs=[]}, Doc, _GetDiskDocFun) ->
- Doc;
-validate_doc_update(_Db, #doc{id= <<"_local/",_/binary>>}=Doc, _GetDiskDocFun) ->
- Doc;
+validate_doc_update(#db{validate_doc_funs=[]}, _Doc, _GetDiskDocFun) ->
+ ok;
+validate_doc_update(_Db, #doc{id= <<"_local/",_/binary>>}, _GetDiskDocFun) ->
+ ok;
validate_doc_update(#db{name=DbName,user_ctx=Ctx}=Db, Doc, GetDiskDocFun) ->
DiskDoc = GetDiskDocFun(),
JsonCtx = {[{<<"db">>, DbName},
{<<"name">>,Ctx#user_ctx.name},
{<<"roles">>,Ctx#user_ctx.roles}]},
- [case Fun(Doc, DiskDoc, JsonCtx) of
- ok -> ok;
- Error -> throw(Error)
- end || Fun <- Db#db.validate_doc_funs],
- Doc.
+ try [case Fun(Doc, DiskDoc, JsonCtx) of
+ ok -> ok;
+ Error -> throw(Error)
+ end || Fun <- Db#db.validate_doc_funs],
+ ok
+ catch
+ throw:Error ->
+ Error
+ end.
-prep_and_validate_new_edit(Db, #doc{id=Id,revs=[NewRev|PrevRevs]}=Doc,
+prep_and_validate_update(Db, #doc{id=Id,revs={RevStart, [_NewRev|PrevRevs]}}=Doc,
OldFullDocInfo, LeafRevsDict) ->
case PrevRevs of
[PrevRev|_] ->
- case dict:find(PrevRev, LeafRevsDict) of
- {ok, {Deleted, Sp, DiskRevs}} ->
- Doc2 = Doc#doc{revs=[NewRev|DiskRevs]},
- case couch_doc:has_stubs(Doc2) of
+ case dict:find({RevStart-1, PrevRev}, LeafRevsDict) of
+ {ok, {Deleted, DiskSp, DiskRevs}} ->
+ case couch_doc:has_stubs(Doc) of
true ->
- DiskDoc = make_doc(Db, Id, Deleted, Sp, DiskRevs),
- Doc3 = couch_doc:merge_stubs(Doc2, DiskDoc),
- validate_doc_update(Db, Doc3, fun() -> DiskDoc end);
+ DiskDoc = make_doc(Db, Id, Deleted, DiskSp, DiskRevs),
+ Doc2 = couch_doc:merge_stubs(Doc, DiskDoc),
+ {validate_doc_update(Db, Doc2, fun() -> DiskDoc end), Doc2};
false ->
- LoadDiskDoc = fun() -> make_doc(Db,Id,Deleted,Sp,DiskRevs) end,
- validate_doc_update(Db, Doc2, LoadDiskDoc)
+ LoadDiskDoc = fun() -> make_doc(Db,Id,Deleted,DiskSp,DiskRevs) end,
+ {validate_doc_update(Db, Doc, LoadDiskDoc), Doc}
end;
error ->
- throw(conflict)
+ {conflict, Doc}
end;
[] ->
% new doc, and we have existing revs.
if OldFullDocInfo#full_doc_info.deleted ->
% existing docs are deletions
- validate_doc_update(Db, Doc, nil);
+ {validate_doc_update(Db, Doc, fun() -> nil end), Doc};
true ->
- throw(conflict)
+ {conflict, Doc}
end
end.
+
+
+prep_and_validate_updates(_Db, [], [], AccPrepped, AccFatalErrors) ->
+ {AccPrepped, AccFatalErrors};
+prep_and_validate_updates(Db, [DocBucket|RestBuckets], [not_found|RestLookups], AccPrepped, AccErrors) ->
+ [#doc{id=Id}|_]=DocBucket,
+ % no existing revs are known,
+ {PreppedBucket, AccErrors3} = lists:foldl(
+ fun(#doc{revs=Revs}=Doc, {AccBucket, AccErrors2}) ->
+ case Revs of
+ {Pos, [NewRev,_OldRev|_]} ->
+ % old revs specified but none exist, a conflict
+ {AccBucket, [{{Id, {Pos, NewRev}}, conflict} | AccErrors2]};
+ {Pos, [NewRev]} ->
+ case validate_doc_update(Db, Doc, fun() -> nil end) of
+ ok ->
+ {[Doc | AccBucket], AccErrors2};
+ Error ->
+ {AccBucket, [{{Id, {Pos, NewRev}}, Error} | AccErrors2]}
+ end
+ end
+ end,
+ {[], AccErrors}, DocBucket),
+
+ prep_and_validate_updates(Db, RestBuckets, RestLookups,
+ [PreppedBucket | AccPrepped], AccErrors3);
+prep_and_validate_updates(Db, [DocBucket|RestBuckets],
+ [{ok, #full_doc_info{rev_tree=OldRevTree}=OldFullDocInfo}|RestLookups],
+ AccPrepped, AccErrors) ->
+ Leafs = couch_key_tree:get_all_leafs(OldRevTree),
+ LeafRevsDict = dict:from_list([{{Start, RevId}, {Deleted, Sp, Revs}} ||
+ {{Deleted, Sp}, {Start, [RevId|_]}=Revs} <- Leafs]),
+ {PreppedBucket, AccErrors3} = lists:foldl(
+ fun(Doc, {Docs2Acc, AccErrors2}) ->
+ case prep_and_validate_update(Db, Doc, OldFullDocInfo,
+ LeafRevsDict) of
+ {ok, Doc} ->
+ {[Doc | Docs2Acc], AccErrors2};
+ {Error, #doc{id=Id,revs={Pos, [NewRev|_]}}} ->
+ % Record the error
+ {Docs2Acc, [{{Id, {Pos, NewRev}}, Error} |AccErrors2]}
+ end
+ end,
+ {[], AccErrors}, DocBucket),
+ prep_and_validate_updates(Db, RestBuckets, RestLookups, [PreppedBucket | AccPrepped], AccErrors3).
+
+
update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options) ->
- update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options, true).
+ update_docs(#db{update_pid=UpdatePid}=Db, Docs, Options, interactive_edit).
-update_docs(Db, Docs, Options, false) ->
+
+prep_and_validate_replicated_updates(_Db, [], [], AccPrepped, AccErrors) ->
+ Errors2 = [{{Id, {Pos, Rev}}, Error} ||
+ {#doc{id=Id,revs={Pos,[Rev|_]}}, Error} <- AccErrors],
+ {lists:reverse(AccPrepped), lists:reverse(Errors2)};
+prep_and_validate_replicated_updates(Db, [Bucket|RestBuckets], [OldInfo|RestOldInfo], AccPrepped, AccErrors) ->
+ case OldInfo of
+ not_found ->
+ {ValidatedBucket, AccErrors3} = lists:foldl(
+ fun(Doc, {AccPrepped2, AccErrors2}) ->
+ case validate_doc_update(Db, Doc, fun() -> nil end) of
+ ok ->
+ {[Doc | AccPrepped2], AccErrors2};
+ Error ->
+ {AccPrepped2, [{Doc, Error} | AccErrors2]}
+ end
+ end,
+ {[], AccErrors}, Bucket),
+ prep_and_validate_replicated_updates(Db, RestBuckets, RestOldInfo, [ValidatedBucket | AccPrepped], AccErrors3);
+ {ok, #full_doc_info{rev_tree=OldTree}} ->
+ NewRevTree = lists:foldl(
+ fun(NewDoc, AccTree) ->
+ {NewTree, _} = couch_key_tree:merge(AccTree, [couch_db:doc_to_tree(NewDoc)]),
+ NewTree
+ end,
+ OldTree, Bucket),
+ Leafs = couch_key_tree:get_all_leafs_full(NewRevTree),
+ LeafRevsFullDict = dict:from_list( [{{Start, RevId}, FullPath} || {Start, [{RevId, _}|_]}=FullPath <- Leafs]),
+ {ValidatedBucket, AccErrors3} =
+ lists:foldl(
+ fun(#doc{id=Id,revs={Pos, [RevId|_]}}=Doc, {AccValidated, AccErrors2}) ->
+ case dict:find({Pos, RevId}, LeafRevsFullDict) of
+ {ok, {Start, Path}} ->
+ % our unflushed doc is a leaf node. Go back on the path
+ % to find the previous rev that's on disk.
+ PrevRevResult =
+ case couch_doc:has_stubs(Doc) of
+ true ->
+ [_PrevRevFull | [PrevRevFull | _]=PrevPath] = Path,
+ case PrevRevFull of
+ {_RevId, ?REV_MISSING} ->
+ conflict;
+ {RevId, {IsDel, DiskSp}} ->
+ DiskDoc = make_doc(Db, Id, IsDel, DiskSp, PrevPath),
+ Doc2 = couch_doc:merge_stubs(Doc, DiskDoc),
+ {ok, Doc2, fun() -> DiskDoc end}
+ end;
+ false ->
+ {ok, Doc,
+ fun() ->
+ make_first_doc_on_disk(Db,Id,Start-1, tl(Path))
+ end}
+ end,
+ case PrevRevResult of
+ {ok, NewDoc, LoadPrevRevFun} ->
+ case validate_doc_update(Db, NewDoc, LoadPrevRevFun) of
+ ok ->
+ {[NewDoc | AccValidated], AccErrors2};
+ Error ->
+ {AccValidated, [{NewDoc, Error} | AccErrors2]}
+ end;
+ Error ->
+ {AccValidated, [{Doc, Error} | AccErrors2]}
+ end;
+ _ ->
+ % this doc isn't a leaf or already exists in the tree.
+ % ignore but consider it a success.
+ {AccValidated, AccErrors2}
+ end
+ end,
+ {[], AccErrors}, Bucket),
+ prep_and_validate_replicated_updates(Db, RestBuckets, RestOldInfo, [ValidatedBucket | AccPrepped], AccErrors3)
+ end.
+
+update_docs(Db, Docs, Options, replicated_changes) ->
couch_stats_collector:increment({couchdb, database_writes}),
DocBuckets = group_alike_docs(Docs),
- Ids = [Id || [#doc{id=Id}|_] <- DocBuckets],
- ExistingDocs = get_full_doc_infos(Db, Ids),
+ case (Db#db.validate_doc_funs /= []) orelse
+ lists:any(
+ fun(#doc{id= <<?DESIGN_DOC_PREFIX, _/binary>>}) -> true;
+ (_) -> false
+ end, Docs) of
+ true ->
+ Ids = [Id || [#doc{id=Id}|_] <- DocBuckets],
+ ExistingDocs = get_full_doc_infos(Db, Ids),
- DocBuckets2 = lists:zipwith(
- fun(Bucket, not_found) ->
- [validate_doc_update(Db, Doc, fun()-> nil end) || Doc <- Bucket];
- (Bucket, {ok, #full_doc_info{rev_tree=OldRevTree}}) ->
- NewTree = lists:foldl(
- fun(Doc, RevTreeAcc) ->
- couch_key_tree:merge(RevTreeAcc, doc_to_tree(Doc))
- end,
- OldRevTree, Bucket),
- Leafs = couch_key_tree:get_all_leafs_full(NewTree),
- LeafRevsFullDict = dict:from_list( [{Rev, FullPath} || [{Rev, _}|_]=FullPath <- Leafs]),
- lists:flatmap(
- fun(#doc{revs=[Rev|_]}=Doc) ->
- case dict:find(Rev, LeafRevsFullDict) of
- {ok, [{Rev, #doc{id=Id}}|_]=Path} ->
- % our unflushed doc is a leaf node. Go back on the path
- % to find the previous rev that's on disk.
- LoadPrevRev = fun() ->
- make_first_doc_on_disk(Db, Id, Path)
- end,
- [validate_doc_update(Db, Doc, LoadPrevRev)];
- _ ->
- % this doc isn't a leaf or is already exists in the tree. ignore
- []
- end
- end, Bucket)
- end,
- DocBuckets, ExistingDocs),
- write_and_commit(Db, DocBuckets2, Options);
+ {DocBuckets2, DocErrors} =
+ prep_and_validate_replicated_updates(Db, DocBuckets, ExistingDocs, [], []),
+ DocBuckets3 = [Bucket || [_|_]=Bucket <- DocBuckets2]; % remove empty buckets
+ false ->
+ DocErrors = [],
+ DocBuckets3 = DocBuckets
+ end,
+ {ok, []} = write_and_commit(Db, DocBuckets3, [merge_conflicts | Options]),
+ {ok, DocErrors};
-update_docs(Db, Docs, Options, true) ->
+update_docs(Db, Docs, Options, interactive_edit) ->
couch_stats_collector:increment({couchdb, database_writes}),
-
+ AllOrNothing = lists:member(all_or_nothing, Options),
% go ahead and generate the new revision ids for the documents.
Docs2 = lists:map(
- fun(#doc{id=Id,revs=Revs}=Doc) ->
+ fun(#doc{id=Id,revs={Start, RevIds}}=Doc) ->
case Id of
<<?LOCAL_DOC_PREFIX, _/binary>> ->
- Rev = case Revs of [] -> 0; [Rev0|_] -> list_to_integer(binary_to_list(Rev0)) end,
- Doc#doc{revs=[list_to_binary(integer_to_list(Rev + 1))]};
+ Rev = case RevIds of [] -> 0; [Rev0|_] -> list_to_integer(?b2l(Rev0)) end,
+ Doc#doc{revs={Start, [?l2b(integer_to_list(Rev + 1))]}};
_ ->
- Doc#doc{revs=[list_to_binary(integer_to_list(couch_util:rand32())) | Revs]}
+ Doc#doc{revs={Start+1, [?l2b(integer_to_list(couch_util:rand32())) | RevIds]}}
end
end, Docs),
DocBuckets = group_alike_docs(Docs2),
- Ids = [Id || [#doc{id=Id}|_] <- DocBuckets],
-
- % lookup the doc by id and get the most recent
-
- ExistingDocs = get_full_doc_infos(Db, Ids),
- DocBuckets2 = lists:zipwith(
- fun(Bucket, not_found) ->
- % no existing revs on disk, make sure no old revs specified.
- [throw(conflict) || #doc{revs=[_NewRev, _OldRev | _]} <- Bucket],
- [validate_doc_update(Db, Doc, fun()-> nil end) || Doc <- Bucket];
- (Bucket, {ok, #full_doc_info{rev_tree=OldRevTree}=OldFullDocInfo}) ->
- Leafs = couch_key_tree:get_all_leafs(OldRevTree),
- LeafRevsDict = dict:from_list([{Rev, {Deleted, Sp, Revs}} || {Rev, {Deleted, Sp}, Revs} <- Leafs]),
- [prep_and_validate_new_edit(Db, Doc, OldFullDocInfo, LeafRevsDict) || Doc <- Bucket]
+ case (Db#db.validate_doc_funs /= []) orelse
+ lists:any(
+ fun(#doc{id= <<?DESIGN_DOC_PREFIX, _/binary>>}) ->
+ true;
+ (#doc{attachments=Atts}) ->
+ Atts /= []
+ end, Docs) of
+ true ->
+ % lookup the doc by id and get the most recent
+ Ids = [Id || [#doc{id=Id}|_] <- DocBuckets],
+ ExistingDocInfos = get_full_doc_infos(Db, Ids),
+
+ {DocBucketsPrepped, Failures} =
+ case AllOrNothing of
+ true ->
+ prep_and_validate_replicated_updates(Db, DocBuckets,
+ ExistingDocInfos, [], []);
+ false ->
+ prep_and_validate_updates(Db, DocBuckets, ExistingDocInfos, [], [])
end,
- DocBuckets, ExistingDocs),
- ok = write_and_commit(Db, DocBuckets2, [new_edits | Options]),
- {ok, [NewRev ||#doc{revs=[NewRev|_]} <- Docs2]}.
+
+ % strip out any empty buckets
+ DocBuckets2 = [Bucket || [_|_] = Bucket <- DocBucketsPrepped];
+ false ->
+ Failures = [],
+ DocBuckets2 = DocBuckets
+ end,
+ if (AllOrNothing) and (Failures /= []) ->
+ {aborted, Failures};
+ true ->
+ Options2 = if AllOrNothing -> [merge_conflicts];
+ true -> [] end ++ Options,
+ {ok, CommitFailures} = write_and_commit(Db, DocBuckets2, Options2),
+ FailDict = dict:from_list(CommitFailures ++ Failures),
+ % the output for each is either {ok, NewRev} or Error
+ {ok, lists:map(
+ fun(#doc{id=Id,revs={Pos, [NewRevId|_]}}) ->
+ case dict:find({Id, {Pos, NewRevId}}, FailDict) of
+ {ok, Error} ->
+ Error;
+ error ->
+ {ok, {Pos, NewRevId}}
+ end
+ end, Docs2)}
+ end.
% Returns the first available document on disk. Input list is a full rev path
% for the doc.
-make_first_doc_on_disk(_Db, _Id, []) ->
+make_first_doc_on_disk(_Db, _Id, _Pos, []) ->
nil;
-make_first_doc_on_disk(Db, Id, [{_Rev, ?REV_MISSING}|RestPath]) ->
- make_first_doc_on_disk(Db, Id, RestPath);
-make_first_doc_on_disk(Db, Id, [{_Rev, {IsDel, Sp}} |_]=DocPath) ->
+make_first_doc_on_disk(Db, Id, Pos, [{_Rev, ?REV_MISSING}|RestPath]) ->
+ make_first_doc_on_disk(Db, Id, Pos - 1, RestPath);
+make_first_doc_on_disk(Db, Id, Pos, [{_Rev, {IsDel, Sp}} |_]=DocPath) ->
Revs = [Rev || {Rev, _} <- DocPath],
- make_doc(Db, Id, IsDel, Sp, Revs).
+ make_doc(Db, Id, IsDel, Sp, {Pos, Revs}).
write_and_commit(#db{update_pid=UpdatePid, user_ctx=Ctx}=Db, DocBuckets,
@@ -374,20 +532,18 @@ write_and_commit(#db{update_pid=UpdatePid, user_ctx=Ctx}=Db, DocBuckets,
% flush unwritten binaries to disk.
DocBuckets2 = [[doc_flush_binaries(Doc, Db#db.fd) || Doc <- Bucket] || Bucket <- DocBuckets],
case gen_server:call(UpdatePid, {update_docs, DocBuckets2, Options}, infinity) of
- ok -> ok;
+ {ok, Conflicts} -> {ok, Conflicts};
retry ->
% This can happen if the db file we wrote to was swapped out by
- % compaction. Retry writing to the current file
+ % compaction. Retry by reopening the db and writing to the current file
{ok, Db2} = open_ref_counted(Db#db.main_pid, Ctx),
DocBuckets3 = [[doc_flush_binaries(Doc, Db2#db.fd) || Doc <- Bucket] || Bucket <- DocBuckets],
% We only retry once
close(Db2),
case gen_server:call(UpdatePid, {update_docs, DocBuckets3, Options}, infinity) of
- ok -> ok;
- Else -> throw(Else)
- end;
- Else->
- throw(Else)
+ {ok, Conflicts} -> {ok, Conflicts};
+ retry -> throw({update_error, compaction_retry})
+ end
end.
@@ -506,7 +662,7 @@ enum_docs_reduce_to_count(Reds) ->
{Count, _DelCount} = couch_btree:final_reduce(
fun couch_db_updater:btree_by_id_reduce/2, Reds),
Count.
-
+
count_changes_since(Db, SinceSeq) ->
{ok, Changes} =
couch_btree:fold_reduce(Db#db.docinfo_by_seq_btree,
@@ -518,7 +674,7 @@ count_changes_since(Db, SinceSeq) ->
end,
ok),
Changes.
-
+
enum_docs_since(Db, SinceSeq, Direction, InFun, Ctx) ->
couch_btree:fold(Db#db.docinfo_by_seq_btree, SinceSeq + 1, Direction, InFun, Ctx).
@@ -594,11 +750,11 @@ open_doc_revs_int(Db, IdRevs, Options) ->
end
end,
FoundResults =
- lists:map(fun({Rev, Value, FoundRevPath}) ->
+ lists:map(fun({Value, {Pos, [Rev|_]}=FoundRevPath}) ->
case Value of
?REV_MISSING ->
% we have the rev in our list but know nothing about it
- {{not_found, missing}, Rev};
+ {{not_found, missing}, {Pos, Rev}};
{IsDeleted, SummaryPtr} ->
{ok, make_doc(Db, Id, IsDeleted, SummaryPtr, FoundRevPath)}
end
@@ -616,18 +772,18 @@ open_doc_revs_int(Db, IdRevs, Options) ->
open_doc_int(Db, <<?LOCAL_DOC_PREFIX, _/binary>> = Id, _Options) ->
case couch_btree:lookup(Db#db.local_docs_btree, [Id]) of
[{ok, {_, {Rev, BodyData}}}] ->
- {ok, #doc{id=Id, revs=[list_to_binary(integer_to_list(Rev))], body=BodyData}};
+ {ok, #doc{id=Id, revs={0, [list_to_binary(integer_to_list(Rev))]}, body=BodyData}};
[not_found] ->
{not_found, missing}
end;
-open_doc_int(Db, #doc_info{id=Id,rev=Rev,deleted=IsDeleted,summary_pointer=Sp}=DocInfo, Options) ->
- Doc = make_doc(Db, Id, IsDeleted, Sp, [Rev]),
+open_doc_int(Db, #doc_info{id=Id,rev={Pos,RevId},deleted=IsDeleted,summary_pointer=Sp}=DocInfo, Options) ->
+ Doc = make_doc(Db, Id, IsDeleted, Sp, {Pos,[RevId]}),
{ok, Doc#doc{meta=doc_meta_info(DocInfo, [], Options)}};
open_doc_int(Db, #full_doc_info{id=Id,rev_tree=RevTree}=FullDocInfo, Options) ->
#doc_info{deleted=IsDeleted,rev=Rev,summary_pointer=Sp} = DocInfo =
couch_doc:to_doc_info(FullDocInfo),
- {[{_Rev,_Value, Revs}], []} = couch_key_tree:get(RevTree, [Rev]),
- Doc = make_doc(Db, Id, IsDeleted, Sp, Revs),
+ {[{_, RevPath}], []} = couch_key_tree:get(RevTree, [Rev]),
+ Doc = make_doc(Db, Id, IsDeleted, Sp, RevPath),
{ok, Doc#doc{meta=doc_meta_info(DocInfo, RevTree, Options)}};
open_doc_int(Db, Id, Options) ->
case get_full_doc_info(Db, Id) of
@@ -641,9 +797,10 @@ doc_meta_info(DocInfo, RevTree, Options) ->
case lists:member(revs_info, Options) of
false -> [];
true ->
- {[RevPath],[]} =
+ {[{Pos, RevPath}],[]} =
couch_key_tree:get_full_key_paths(RevTree, [DocInfo#doc_info.rev]),
- [{revs_info, lists:map(
+
+ [{revs_info, Pos, lists:map(
fun({Rev, {true, _Sp}}) ->
{Rev, deleted};
({Rev, {false, _Sp}}) ->
@@ -670,13 +827,15 @@ doc_meta_info(DocInfo, RevTree, Options) ->
end.
-doc_to_tree(Doc) ->
- doc_to_tree(Doc, lists:reverse(Doc#doc.revs)).
+doc_to_tree(#doc{revs={Start, RevIds}}=Doc) ->
+ [Tree] = doc_to_tree_simple(Doc, lists:reverse(RevIds)),
+ {Start - length(RevIds) + 1, Tree}.
+
-doc_to_tree(Doc, [RevId]) ->
+doc_to_tree_simple(Doc, [RevId]) ->
[{RevId, Doc, []}];
-doc_to_tree(Doc, [RevId | Rest]) ->
- [{RevId, ?REV_MISSING, doc_to_tree(Doc, Rest)}].
+doc_to_tree_simple(Doc, [RevId | Rest]) ->
+ [{RevId, ?REV_MISSING, doc_to_tree_simple(Doc, Rest)}].
make_doc(Db, FullDocInfo) ->
{#doc_info{id=Id,deleted=Deleted,summary_pointer=Sp}, RevPath}
@@ -703,4 +862,4 @@ make_doc(#db{fd=Fd}=Db, Id, Deleted, BodySp, RevisionPath) ->
}.
- \ No newline at end of file
+