summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--src/couchdb/couch_btree.erl39
-rw-r--r--src/couchdb/couch_db.erl12
-rw-r--r--src/couchdb/couch_db_updater.erl9
-rw-r--r--src/couchdb/couch_doc.erl8
-rw-r--r--src/couchdb/couch_file.erl10
-rw-r--r--src/couchdb/couch_httpd.erl8
-rw-r--r--src/couchdb/couch_httpd_auth.erl43
-rw-r--r--src/couchdb/couch_httpd_db.erl2
-rw-r--r--src/couchdb/couch_httpd_external.erl2
-rw-r--r--src/couchdb/couch_httpd_show.erl14
-rw-r--r--src/couchdb/couch_httpd_view.erl5
-rw-r--r--src/couchdb/couch_key_tree.erl4
-rw-r--r--src/couchdb/couch_native_process.erl11
-rw-r--r--src/couchdb/couch_os_process.erl5
-rw-r--r--src/couchdb/couch_query_servers.erl10
-rw-r--r--src/couchdb/couch_rep.erl12
-rw-r--r--src/couchdb/couch_rep_att.erl4
-rw-r--r--src/couchdb/couch_rep_httpc.erl2
-rw-r--r--src/couchdb/couch_rep_writer.erl22
-rw-r--r--src/couchdb/couch_stats_aggregator.erl8
-rw-r--r--src/couchdb/couch_util.erl31
-rw-r--r--src/couchdb/couch_view.erl16
-rw-r--r--src/couchdb/couch_view_compactor.erl4
-rw-r--r--src/couchdb/couch_view_updater.erl2
-rw-r--r--src/couchdb/couch_work_queue.erl2
25 files changed, 127 insertions, 158 deletions
diff --git a/src/couchdb/couch_btree.erl b/src/couchdb/couch_btree.erl
index 0dad13f8..73d50805 100644
--- a/src/couchdb/couch_btree.erl
+++ b/src/couchdb/couch_btree.erl
@@ -217,16 +217,12 @@ lookup(Bt, {Pointer, _Reds}, Keys) ->
lookup_kvnode(Bt, list_to_tuple(NodeList), 1, Keys, [])
end.
-
-
lookup_kpnode(_Bt, _NodeTuple, _LowerBound, [], Output) ->
{ok, lists:reverse(Output)};
-
-lookup_kpnode(_Bt, NodeTuple, LowerBound, Keys, Output) when size(NodeTuple) < LowerBound ->
+lookup_kpnode(_Bt, NodeTuple, LowerBound, Keys, Output) when tuple_size(NodeTuple) < LowerBound ->
{ok, lists:reverse(Output, [{Key, not_found} || Key <- Keys])};
-
lookup_kpnode(Bt, NodeTuple, LowerBound, [FirstLookupKey | _] = LookupKeys, Output) ->
- N = find_first_gteq(Bt, NodeTuple, LowerBound, size(NodeTuple), FirstLookupKey),
+ N = find_first_gteq(Bt, NodeTuple, LowerBound, tuple_size(NodeTuple), FirstLookupKey),
{Key, PointerInfo} = element(N, NodeTuple),
SplitFun = fun(LookupKey) -> not less(Bt, Key, LookupKey) end,
case lists:splitwith(SplitFun, LookupKeys) of
@@ -240,11 +236,11 @@ lookup_kpnode(Bt, NodeTuple, LowerBound, [FirstLookupKey | _] = LookupKeys, Outp
lookup_kvnode(_Bt, _NodeTuple, _LowerBound, [], Output) ->
{ok, lists:reverse(Output)};
-lookup_kvnode(_Bt, NodeTuple, LowerBound, Keys, Output) when size(NodeTuple) < LowerBound ->
+lookup_kvnode(_Bt, NodeTuple, LowerBound, Keys, Output) when tuple_size(NodeTuple) < LowerBound ->
% keys not found
{ok, lists:reverse(Output, [{Key, not_found} || Key <- Keys])};
lookup_kvnode(Bt, NodeTuple, LowerBound, [LookupKey | RestLookupKeys], Output) ->
- N = find_first_gteq(Bt, NodeTuple, LowerBound, size(NodeTuple), LookupKey),
+ N = find_first_gteq(Bt, NodeTuple, LowerBound, tuple_size(NodeTuple), LookupKey),
{Key, Value} = element(N, NodeTuple),
case less(Bt, LookupKey, Key) of
true ->
@@ -272,14 +268,14 @@ complete_root(Bt, KPs) ->
%%%%%%%%%%%%% The chunkify function sucks! %%%%%%%%%%%%%
% It is inaccurate as it does not account for compression when blocks are
-% written. Plus with the "case size(term_to_binary(InList)) of" code it's
-% probably really inefficient.
+% written. Plus with the "case byte_size(term_to_binary(InList)) of" code
+% it's probably really inefficient.
% dialyzer says this pattern is never matched
% chunkify(_Bt, []) ->
% [];
chunkify(Bt, InList) ->
- case size(term_to_binary(InList)) of
+ case byte_size(term_to_binary(InList)) of
Size when Size > ?CHUNK_THRESHOLD ->
NumberOfChunksLikely = ((Size div ?CHUNK_THRESHOLD) + 1),
ChunkThreshold = Size div NumberOfChunksLikely,
@@ -293,7 +289,7 @@ chunkify(_Bt, [], _ChunkThreshold, [], 0, OutputChunks) ->
chunkify(_Bt, [], _ChunkThreshold, OutList, _OutListSize, OutputChunks) ->
lists:reverse([lists:reverse(OutList) | OutputChunks]);
chunkify(Bt, [InElement | RestInList], ChunkThreshold, OutList, OutListSize, OutputChunks) ->
- case size(term_to_binary(InElement)) of
+ case byte_size(term_to_binary(InElement)) of
Size when (Size + OutListSize) > ChunkThreshold andalso OutList /= [] ->
chunkify(Bt, RestInList, ChunkThreshold, [], 0, [lists:reverse([InElement | OutList]) | OutputChunks]);
Size ->
@@ -319,7 +315,7 @@ modify_node(Bt, RootPointerInfo, Actions, QueryOutput) ->
[] -> % no nodes remain
{ok, [], QueryOutput2, Bt2};
NodeList -> % nothing changed
- {LastKey, _LastValue} = element(size(NodeTuple), NodeTuple),
+ {LastKey, _LastValue} = element(tuple_size(NodeTuple), NodeTuple),
{ok, [{LastKey, RootPointerInfo}], QueryOutput2, Bt2};
_Else2 ->
{ok, ResultList, Bt3} = write_node(Bt2, NodeType, NewNodeList),
@@ -357,18 +353,19 @@ modify_kpnode(Bt, {}, _LowerBound, Actions, [], QueryOutput) ->
modify_node(Bt, nil, Actions, QueryOutput);
modify_kpnode(Bt, NodeTuple, LowerBound, [], ResultNode, QueryOutput) ->
{ok, lists:reverse(ResultNode, bounded_tuple_to_list(NodeTuple, LowerBound,
- size(NodeTuple), [])), QueryOutput, Bt};
+ tuple_size(NodeTuple), [])), QueryOutput, Bt};
modify_kpnode(Bt, NodeTuple, LowerBound,
[{_, FirstActionKey, _}|_]=Actions, ResultNode, QueryOutput) ->
- N = find_first_gteq(Bt, NodeTuple, LowerBound, size(NodeTuple), FirstActionKey),
- case N == size(NodeTuple) of
+ Sz = tuple_size(NodeTuple),
+ N = find_first_gteq(Bt, NodeTuple, LowerBound, Sz, FirstActionKey),
+ case N =:= Sz of
true ->
% perform remaining actions on last node
- {_, PointerInfo} = element(size(NodeTuple), NodeTuple),
+ {_, PointerInfo} = element(Sz, NodeTuple),
{ok, ChildKPs, QueryOutput2, Bt2} =
modify_node(Bt, PointerInfo, Actions, QueryOutput),
NodeList = lists:reverse(ResultNode, bounded_tuple_to_list(NodeTuple, LowerBound,
- size(NodeTuple) - 1, ChildKPs)),
+ Sz - 1, ChildKPs)),
{ok, NodeList, QueryOutput2, Bt2};
false ->
{NodeKey, PointerInfo} = element(N, NodeTuple),
@@ -409,8 +406,8 @@ find_first_gteq(Bt, Tuple, Start, End, Key) ->
end.
modify_kvnode(Bt, NodeTuple, LowerBound, [], ResultNode, QueryOutput) ->
- {ok, lists:reverse(ResultNode, bounded_tuple_to_list(NodeTuple, LowerBound, size(NodeTuple), [])), QueryOutput, Bt};
-modify_kvnode(Bt, NodeTuple, LowerBound, [{ActionType, ActionKey, ActionValue} | RestActions], ResultNode, QueryOutput) when LowerBound > size(NodeTuple) ->
+ {ok, lists:reverse(ResultNode, bounded_tuple_to_list(NodeTuple, LowerBound, tuple_size(NodeTuple), [])), QueryOutput, Bt};
+modify_kvnode(Bt, NodeTuple, LowerBound, [{ActionType, ActionKey, ActionValue} | RestActions], ResultNode, QueryOutput) when LowerBound > tuple_size(NodeTuple) ->
case ActionType of
insert ->
modify_kvnode(Bt, NodeTuple, LowerBound, RestActions, [{ActionKey, ActionValue} | ResultNode], QueryOutput);
@@ -422,7 +419,7 @@ modify_kvnode(Bt, NodeTuple, LowerBound, [{ActionType, ActionKey, ActionValue} |
modify_kvnode(Bt, NodeTuple, LowerBound, RestActions, ResultNode, [{not_found, {ActionKey, nil}} | QueryOutput])
end;
modify_kvnode(Bt, NodeTuple, LowerBound, [{ActionType, ActionKey, ActionValue} | RestActions], AccNode, QueryOutput) ->
- N = find_first_gteq(Bt, NodeTuple, LowerBound, size(NodeTuple), ActionKey),
+ N = find_first_gteq(Bt, NodeTuple, LowerBound, tuple_size(NodeTuple), ActionKey),
{Key, Value} = element(N, NodeTuple),
ResultNode = bounded_tuple_to_revlist(NodeTuple, LowerBound, N - 1, AccNode),
case less(Bt, ActionKey, Key) of
diff --git a/src/couchdb/couch_db.erl b/src/couchdb/couch_db.erl
index 7630989a..a729f215 100644
--- a/src/couchdb/couch_db.erl
+++ b/src/couchdb/couch_db.erl
@@ -290,7 +290,7 @@ validate_doc_update(#db{user_ctx=UserCtx, admins=Admins},
#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
+ case length(UserNames -- [<<"_admin">> | Admins]) =:= length(UserNames) of
true ->
% not an admin
{unauthorized, <<"You are not a server or database admin.">>};
@@ -478,8 +478,8 @@ prep_and_validate_replicated_updates(Db, [Bucket|RestBuckets], [OldInfo|RestOldI
new_revid(#doc{body=Body,revs={OldStart,OldRevs},
atts=Atts,deleted=Deleted}) ->
- case [{N, T, M} || #att{name=N,type=T,md5=M} <- Atts, M /= <<>>] of
- Atts2 when length(Atts) /= length(Atts2) ->
+ case [{N, T, M} || #att{name=N,type=T,md5=M} <- Atts, M =/= <<>>] of
+ Atts2 when length(Atts) =/= length(Atts2) ->
% We must have old style non-md5 attachments
?l2b(integer_to_list(couch_util:rand32()));
Atts2 ->
@@ -503,7 +503,7 @@ check_dup_atts(#doc{atts=Atts}=Doc) ->
check_dup_atts2(Atts2),
Doc.
-check_dup_atts2([#att{name=N1}, #att{name=N2} | _]) when N1 == N2 ->
+check_dup_atts2([#att{name=N}, #att{name=N} | _]) ->
throw({bad_request, <<"Duplicate attachments">>});
check_dup_atts2([_ | Rest]) ->
check_dup_atts2(Rest);
@@ -803,10 +803,10 @@ handle_call(is_idle, _From, #db{fd_ref_counter=RefCntr, compactor_pid=Compact,
waiting_delayed_commit=Delay}=Db) ->
% Idle means no referrers. Unless in the middle of a compaction file switch,
% there are always at least 2 referrers, couch_db_updater and us.
- {reply, (Delay == nil) and (Compact == nil) and (couch_ref_counter:count(RefCntr) == 2), Db};
+ {reply, (Delay == nil) andalso (Compact == nil) andalso (couch_ref_counter:count(RefCntr) == 2), Db};
handle_call({db_updated, NewDb}, _From, #db{fd_ref_counter=OldRefCntr}) ->
#db{fd_ref_counter=NewRefCntr}=NewDb,
- case NewRefCntr == OldRefCntr of
+ case NewRefCntr =:= OldRefCntr of
true -> ok;
false ->
couch_ref_counter:add(NewRefCntr),
diff --git a/src/couchdb/couch_db_updater.erl b/src/couchdb/couch_db_updater.erl
index 31366a84..d9951cf6 100644
--- a/src/couchdb/couch_db_updater.erl
+++ b/src/couchdb/couch_db_updater.erl
@@ -318,11 +318,12 @@ btree_by_seq_reduce(reduce, DocInfos) ->
btree_by_seq_reduce(rereduce, Reds) ->
lists:sum(Reds).
-simple_upgrade_record(Old, New) when size(Old) == size(New)->
+simple_upgrade_record(Old, New) when tuple_size(Old) =:= tuple_size(New) ->
Old;
simple_upgrade_record(Old, New) ->
+ OldSz = tuple_size(Old),
NewValuesTail =
- lists:sublist(tuple_to_list(New), size(Old) + 1, size(New)-size(Old)),
+ lists:sublist(tuple_to_list(New), OldSz + 1, tuple_size(New) - OldSz),
list_to_tuple(tuple_to_list(Old) ++ NewValuesTail).
@@ -751,9 +752,9 @@ copy_compact(Db, NewDb0, Retry) ->
fun(#doc_info{high_seq=Seq}=DocInfo, _Offset, {AccNewDb, AccUncopied, TotalCopied}) ->
couch_task_status:update("Copied ~p of ~p changes (~p%)",
[TotalCopied, TotalChanges, (TotalCopied*100) div TotalChanges]),
- if TotalCopied rem 1000 == 0 ->
+ if TotalCopied rem 1000 =:= 0 ->
NewDb2 = copy_docs(Db, AccNewDb, lists:reverse([DocInfo | AccUncopied]), Retry),
- if TotalCopied rem 10000 == 0 ->
+ if TotalCopied rem 10000 =:= 0 ->
{ok, {commit_data(NewDb2#db{update_seq=Seq}), [], TotalCopied + 1}};
true ->
{ok, {NewDb2#db{update_seq=Seq}, [], TotalCopied + 1}}
diff --git a/src/couchdb/couch_doc.erl b/src/couchdb/couch_doc.erl
index eb07a9e6..d0c2918a 100644
--- a/src/couchdb/couch_doc.erl
+++ b/src/couchdb/couch_doc.erl
@@ -40,7 +40,7 @@ to_json_revisions(Options, Start, RevIds) ->
{<<"ids">>, [revid_to_str(R) ||R <- RevIds]}]}}]
end.
-revid_to_str(RevId) when size(RevId) == 16 ->
+revid_to_str(RevId) when size(RevId) =:= 16 ->
?l2b(couch_util:to_hex(RevId));
revid_to_str(RevId) ->
RevId.
@@ -122,10 +122,10 @@ from_json_obj({Props}) ->
from_json_obj(_Other) ->
throw({bad_request, "Document must be a JSON object"}).
-parse_revid(RevId) when size(RevId) == 32 ->
+parse_revid(RevId) when size(RevId) =:= 32 ->
RevInt = erlang:list_to_integer(?b2l(RevId), 16),
<<RevInt:128>>;
-parse_revid(RevId) when length(RevId) == 32 ->
+parse_revid(RevId) when length(RevId) =:= 32 ->
RevInt = erlang:list_to_integer(RevId, 16),
<<RevInt:128>>;
parse_revid(RevId) when is_binary(RevId) ->
@@ -222,7 +222,7 @@ transfer_fields([{<<"_revisions">>, {Props}} | Rest], Doc) ->
RevIds2 = [parse_revid(RevId) || RevId <- RevIds],
transfer_fields(Rest, Doc#doc{revs={Start, RevIds2}});
-transfer_fields([{<<"_deleted">>, B} | Rest], Doc) when (B==true) or (B==false) ->
+transfer_fields([{<<"_deleted">>, B} | Rest], Doc) when is_boolean(B) ->
transfer_fields(Rest, Doc#doc{deleted=B});
% ignored fields
diff --git a/src/couchdb/couch_file.erl b/src/couchdb/couch_file.erl
index 2dfc5403..5904260c 100644
--- a/src/couchdb/couch_file.erl
+++ b/src/couchdb/couch_file.erl
@@ -402,14 +402,14 @@ extract_header(Prefix, Bin) ->
write_old_header(Fd, Prefix, Data) ->
TermBin = term_to_binary(Data),
% the size of all the bytes written to the header, including the md5 signature (16 bytes)
- FilledSize = size(Prefix) + size(TermBin) + 16,
+ FilledSize = byte_size(Prefix) + byte_size(TermBin) + 16,
{TermBin2, FilledSize2} =
case FilledSize > ?HEADER_SIZE of
true ->
% too big!
{ok, Pos} = append_binary(Fd, TermBin),
PtrBin = term_to_binary({pointer_to_header_data, Pos}),
- {PtrBin, size(Prefix) + size(PtrBin) + 16};
+ {PtrBin, byte_size(Prefix) + byte_size(PtrBin) + 16};
false ->
{TermBin, FilledSize}
end,
@@ -465,7 +465,7 @@ calculate_total_read_len(BlockOffset, FinalLen) ->
FinalLen;
BlockLeft ->
FinalLen + ((FinalLen - BlockLeft) div (?SIZE_BLOCK -1)) +
- if ((FinalLen - BlockLeft) rem (?SIZE_BLOCK -1)) == 0 -> 0;
+ if ((FinalLen - BlockLeft) rem (?SIZE_BLOCK -1)) =:= 0 -> 0;
true -> 1 end
end.
@@ -499,8 +499,8 @@ split_iolist(List, 0, BeginAcc) ->
{lists:reverse(BeginAcc), List};
split_iolist([], SplitAt, _BeginAcc) ->
SplitAt;
-split_iolist([<<Bin/binary>> | Rest], SplitAt, BeginAcc) when SplitAt > size(Bin) ->
- split_iolist(Rest, SplitAt - size(Bin), [Bin | BeginAcc]);
+split_iolist([<<Bin/binary>> | Rest], SplitAt, BeginAcc) when SplitAt > byte_size(Bin) ->
+ split_iolist(Rest, SplitAt - byte_size(Bin), [Bin | BeginAcc]);
split_iolist([<<Bin/binary>> | Rest], SplitAt, BeginAcc) ->
<<Begin:SplitAt/binary,End/binary>> = Bin,
split_iolist([End | Rest], 0, [Begin | BeginAcc]);
diff --git a/src/couchdb/couch_httpd.erl b/src/couchdb/couch_httpd.erl
index caffdd51..a3e486f1 100644
--- a/src/couchdb/couch_httpd.erl
+++ b/src/couchdb/couch_httpd.erl
@@ -97,17 +97,17 @@ start_link() ->
make_arity_1_fun(SpecStr) ->
case couch_util:parse_term(SpecStr) of
{ok, {Mod, Fun, SpecArg}} ->
- fun(Arg) -> apply(Mod, Fun, [Arg, SpecArg]) end;
+ fun(Arg) -> Mod:Fun(Arg, SpecArg) end;
{ok, {Mod, Fun}} ->
- fun(Arg) -> apply(Mod, Fun, [Arg]) end
+ fun(Arg) -> Mod:Fun(Arg) end
end.
make_arity_2_fun(SpecStr) ->
case couch_util:parse_term(SpecStr) of
{ok, {Mod, Fun, SpecArg}} ->
- fun(Arg1, Arg2) -> apply(Mod, Fun, [Arg1, Arg2, SpecArg]) end;
+ fun(Arg1, Arg2) -> Mod:Fun(Arg1, Arg2, SpecArg) end;
{ok, {Mod, Fun}} ->
- fun(Arg1, Arg2) -> apply(Mod, Fun, [Arg1, Arg2]) end
+ fun(Arg1, Arg2) -> Mod:Fun(Arg1, Arg2) end
end.
% SpecStr is "{my_module, my_fun}, {my_module2, my_fun2}"
diff --git a/src/couchdb/couch_httpd_auth.erl b/src/couchdb/couch_httpd_auth.erl
index fb76c72a..b244e16e 100644
--- a/src/couchdb/couch_httpd_auth.erl
+++ b/src/couchdb/couch_httpd_auth.erl
@@ -377,7 +377,7 @@ create_user_req(#httpd{method='POST', mochi_req=MochiReq}=Req, Db) ->
?LOG_DEBUG("Can't create ~s: already exists", [?b2l(UserName)]),
throw({forbidden, <<"User already exists.">>})
end.
-
+
update_user_req(#httpd{method='PUT', mochi_req=MochiReq, user_ctx=UserCtx}=Req, Db, UserName) ->
Name = UserCtx#user_ctx.name,
UserRoles = UserCtx#user_ctx.roles,
@@ -413,16 +413,14 @@ update_user_req(#httpd{method='PUT', mochi_req=MochiReq, user_ctx=UserCtx}=Req,
PasswordHash = case lists:member(<<"_admin">>, UserRoles) of
true ->
- Hash = case Password of
+ case Password of
<<>> -> CurrentPasswordHash;
_Else ->
- H = hash_password(Password, UserSalt),
- H
- end,
- Hash;
- false when Name == UserName ->
+ hash_password(Password, UserSalt)
+ end;
+ false when Name =:= UserName ->
%% for user we test old password before allowing change
- Hash = case Password of
+ case Password of
<<>> ->
CurrentPasswordHash;
_P when OldPassword =:= [] ->
@@ -430,16 +428,13 @@ update_user_req(#httpd{method='PUT', mochi_req=MochiReq, user_ctx=UserCtx}=Req,
_Else ->
OldPasswordHash = hash_password(OldPassword1, UserSalt),
?LOG_DEBUG("~p == ~p", [CurrentPasswordHash, OldPasswordHash]),
- Hash1 = case CurrentPasswordHash of
- ExpectedHash when ExpectedHash == OldPasswordHash ->
- H = hash_password(Password, UserSalt),
- H;
+ case CurrentPasswordHash of
+ ExpectedHash when ExpectedHash =:= OldPasswordHash ->
+ hash_password(Password, UserSalt);
_ ->
throw({forbidden, <<"Old password is incorrect.">>})
- end,
- Hash1
- end,
- Hash;
+ end
+ end;
_ ->
throw({forbidden, <<"You aren't allowed to change this password.">>})
end,
@@ -457,24 +452,24 @@ update_user_req(#httpd{method='PUT', mochi_req=MochiReq, user_ctx=UserCtx}=Req,
handle_user_req(#httpd{method='POST'}=Req) ->
DbName = couch_config:get("couch_httpd_auth", "authentication_db"),
ensure_users_db_exists(?l2b(DbName)),
- case couch_db:open(?l2b(DbName), [{user_ctx, #user_ctx{roles=[<<"_admin">>]}}]) of
- {ok, Db} -> create_user_req(Req, Db)
- end;
+ {ok, Db} = couch_db:open(?l2b(DbName),
+ [{user_ctx, #user_ctx{roles=[<<"_admin">>]}}]),
+ create_user_req(Req, Db);
handle_user_req(#httpd{method='PUT', path_parts=[_]}=_Req) ->
throw({bad_request, <<"Username is missing">>});
handle_user_req(#httpd{method='PUT', path_parts=[_, UserName]}=Req) ->
DbName = couch_config:get("couch_httpd_auth", "authentication_db"),
ensure_users_db_exists(?l2b(DbName)),
- case couch_db:open(?l2b(DbName), [{user_ctx, #user_ctx{roles=[<<"_admin">>]}}]) of
- {ok, Db} -> update_user_req(Req, Db, UserName)
- end;
+ {ok, Db} = couch_db:open(?l2b(DbName),
+ [{user_ctx, #user_ctx{roles=[<<"_admin">>]}}]),
+ update_user_req(Req, Db, UserName);
handle_user_req(Req) ->
- send_method_not_allowed(Req, "POST,PUT").
+ couch_httpd:send_method_not_allowed(Req, "POST,PUT").
to_int(Value) when is_binary(Value) ->
to_int(?b2l(Value));
to_int(Value) when is_list(Value) ->
- erlang:list_to_integer(Value);
+ list_to_integer(Value);
to_int(Value) when is_integer(Value) ->
Value.
diff --git a/src/couchdb/couch_httpd_db.erl b/src/couchdb/couch_httpd_db.erl
index 71fc3f48..5d23232e 100644
--- a/src/couchdb/couch_httpd_db.erl
+++ b/src/couchdb/couch_httpd_db.erl
@@ -1021,7 +1021,7 @@ db_attachment_req(#httpd{method=Method}=Req, Db, DocId, FileNameParts)
[];
_ ->
[#att{
- name=FileName,
+ name = FileName,
type = case couch_httpd:header_value(Req,"Content-Type") of
undefined ->
% We could throw an error here or guess by the FileName.
diff --git a/src/couchdb/couch_httpd_external.erl b/src/couchdb/couch_httpd_external.erl
index 0a26bcf7..86b2bfc6 100644
--- a/src/couchdb/couch_httpd_external.erl
+++ b/src/couchdb/couch_httpd_external.erl
@@ -86,6 +86,7 @@ json_req_obj(#httpd{mochi_req=Req,
to_json_terms(Data) ->
to_json_terms(Data, []).
+
to_json_terms([], Acc) ->
{lists:reverse(Acc)};
to_json_terms([{Key, Value} | Rest], Acc) when is_atom(Key) ->
@@ -93,7 +94,6 @@ to_json_terms([{Key, Value} | Rest], Acc) when is_atom(Key) ->
to_json_terms([{Key, Value} | Rest], Acc) ->
to_json_terms(Rest, [{list_to_binary(Key), list_to_binary(Value)} | Acc]).
-
send_external_response(#httpd{mochi_req=MochiReq}=Req, Response) ->
#extern_resp_args{
code = Code,
diff --git a/src/couchdb/couch_httpd_show.erl b/src/couchdb/couch_httpd_show.erl
index 4ed11c2a..14b9b415 100644
--- a/src/couchdb/couch_httpd_show.erl
+++ b/src/couchdb/couch_httpd_show.erl
@@ -50,11 +50,10 @@ handle_doc_update_req(#httpd{
#doc{body={Props}} = couch_httpd_db:couch_doc_open(Db, DesignId, nil, []),
Lang = proplists:get_value(<<"language">>, Props, <<"javascript">>),
UpdateSrc = couch_util:get_nested_json_value({Props}, [<<"updates">>, UpdateName]),
- Doc = try couch_httpd_db:couch_doc_open(Db, DocId, nil, [conflicts]) of
- FoundDoc -> FoundDoc
- catch
- _ -> nil
- end,
+ Doc = try couch_httpd_db:couch_doc_open(Db, DocId, nil, [conflicts])
+ catch
+ _ -> nil
+ end,
send_doc_update_response(Lang, UpdateSrc, DocId, Doc, Req, Db);
handle_doc_update_req(#httpd{
@@ -66,13 +65,11 @@ handle_doc_update_req(#httpd{
UpdateSrc = couch_util:get_nested_json_value({Props}, [<<"updates">>, UpdateName]),
send_doc_update_response(Lang, UpdateSrc, nil, nil, Req, Db);
-
handle_doc_update_req(Req, _Db) ->
send_error(Req, 404, <<"update_error">>, <<"Invalid path.">>).
-
handle_doc_show(Req, DesignName, ShowName, DocId, Db) ->
DesignId = <<"_design/", DesignName/binary>>,
#doc{body={Props}} = couch_httpd_db:couch_doc_open(Db, DesignId, nil, []),
@@ -81,8 +78,7 @@ handle_doc_show(Req, DesignName, ShowName, DocId, Db) ->
Doc = case DocId of
nil -> nil;
_ ->
- try couch_httpd_db:couch_doc_open(Db, DocId, nil, [conflicts]) of
- FoundDoc -> FoundDoc
+ try couch_httpd_db:couch_doc_open(Db, DocId, nil, [conflicts])
catch
_ -> nil
end
diff --git a/src/couchdb/couch_httpd_view.erl b/src/couchdb/couch_httpd_view.erl
index 4c9902ed..cb9f8687 100644
--- a/src/couchdb/couch_httpd_view.erl
+++ b/src/couchdb/couch_httpd_view.erl
@@ -236,10 +236,7 @@ parse_view_params(Req, Keys, ViewType) ->
lists:foldl(fun({K, V}, Acc) ->
parse_view_param(K, V) ++ Acc
end, [], QueryList),
- IsMultiGet = case Keys of
- nil -> false;
- _ -> true
- end,
+ IsMultiGet = (Keys =/= nil),
Args = #view_query_args{
view_type=ViewType,
multi_get=IsMultiGet
diff --git a/src/couchdb/couch_key_tree.erl b/src/couchdb/couch_key_tree.erl
index 830820f3..d5944119 100644
--- a/src/couchdb/couch_key_tree.erl
+++ b/src/couchdb/couch_key_tree.erl
@@ -34,7 +34,7 @@ merge(A, B) ->
end,
{A, false}, B),
if HasConflicts or
- ((length(Merged) /= length(A)) and (length(Merged) /= length(B))) ->
+ ((length(Merged) =/= length(A)) and (length(Merged) =/= length(B))) ->
Conflicts = conflicts;
true ->
Conflicts = no_conflicts
@@ -221,7 +221,7 @@ get_full_key_paths(_Pos, [], KeysToGet, _KeyPathAcc) ->
get_full_key_paths(Pos, [{KeyId, Value, SubTree} | RestTree], KeysToGet, KeyPathAcc) ->
KeysToGet2 = KeysToGet -- [{Pos, KeyId}],
CurrentNodeResult =
- case length(KeysToGet2) == length(KeysToGet) of
+ case length(KeysToGet2) =:= length(KeysToGet) of
true -> % not in the key list.
[];
false -> % this node is the key list. return it
diff --git a/src/couchdb/couch_native_process.erl b/src/couchdb/couch_native_process.erl
index 65b733bf..2b74073c 100644
--- a/src/couchdb/couch_native_process.erl
+++ b/src/couchdb/couch_native_process.erl
@@ -120,19 +120,14 @@ run(State, [<<"validate">>, BFun, NDoc, ODoc, Ctx]) ->
{State, catch Fun(NDoc, ODoc, Ctx)};
run(State, [<<"filter">>, Docs, Req]) ->
{_Sig, Fun} = hd(State#evstate.funs),
- Resp = lists:map(fun(Doc) ->
- case (catch Fun(Doc, Req)) of
- true -> true;
- _ -> false
- end
- end, Docs),
+ Resp = lists:map(fun(Doc) -> (catch Fun(Doc, Req)) =:= true end, Docs),
{State, [true, Resp]};
run(State, [<<"show">>, BFun, Doc, Req]) ->
{_Sig, Fun} = makefun(State, BFun),
Resp = case (catch Fun(Doc, Req)) of
FunResp when is_list(FunResp) ->
FunResp;
- FunResp when is_tuple(FunResp), size(FunResp) == 1 ->
+ FunResp when tuple_size(FunResp) =:= 1 ->
[<<"resp">>, FunResp];
FunResp ->
FunResp
@@ -352,7 +347,7 @@ to_binary({Data}) ->
end,
{lists:map(Pred, Data)};
to_binary(Data) when is_list(Data) ->
- lists:map(fun to_binary/1, Data);
+ [to_binary(D) || D <- Data];
to_binary(null) ->
null;
to_binary(true) ->
diff --git a/src/couchdb/couch_os_process.erl b/src/couchdb/couch_os_process.erl
index f42e99a8..244a59f5 100644
--- a/src/couchdb/couch_os_process.erl
+++ b/src/couchdb/couch_os_process.erl
@@ -62,10 +62,9 @@ prompt(Pid, Data) ->
writeline(OsProc, Data) when is_record(OsProc, os_proc) ->
port_command(OsProc#os_proc.port, Data ++ "\n").
-readline(OsProc) when is_record(OsProc, os_proc) ->
+readline(#os_proc{} = OsProc) ->
readline(OsProc, []).
-readline(OsProc, Acc) when is_record(OsProc, os_proc) ->
- #os_proc{port=Port} = OsProc,
+readline(#os_proc{port = Port} = OsProc, Acc) ->
receive
{Port, {data, {noeol, Data}}} ->
readline(OsProc, [Data|Acc]);
diff --git a/src/couchdb/couch_query_servers.erl b/src/couchdb/couch_query_servers.erl
index f94cc28b..9ab4ff5a 100644
--- a/src/couchdb/couch_query_servers.erl
+++ b/src/couchdb/couch_query_servers.erl
@@ -190,10 +190,7 @@ render_doc_show(Lang, ShowSrc, DocId, Doc, Req, Db) ->
{DocId, nil} -> {{append_docid(DocId, JsonReqIn)}, null};
_ -> {{append_docid(DocId, JsonReqIn)}, couch_doc:to_json_obj(Doc, [revs])}
end,
- try proc_prompt(Proc,
- [<<"show">>, ShowSrc, JsonDoc, JsonReq]) of
- FormResp ->
- FormResp
+ try proc_prompt(Proc, [<<"show">>, ShowSrc, JsonDoc, JsonReq])
after
ok = ret_os_process(Proc)
end.
@@ -207,10 +204,7 @@ render_doc_update(Lang, UpdateSrc, DocId, Doc, Req, Db) ->
{DocId, nil} -> {{append_docid(DocId, JsonReqIn)}, null};
_ -> {{append_docid(DocId, JsonReqIn)}, couch_doc:to_json_obj(Doc, [revs])}
end,
- try proc_prompt(Proc,
- [<<"update">>, UpdateSrc, JsonDoc, JsonReq]) of
- FormResp ->
- FormResp
+ try proc_prompt(Proc, [<<"update">>, UpdateSrc, JsonDoc, JsonReq])
after
ok = ret_os_process(Proc)
end.
diff --git a/src/couchdb/couch_rep.erl b/src/couchdb/couch_rep.erl
index 23a44069..24b1557f 100644
--- a/src/couchdb/couch_rep.erl
+++ b/src/couchdb/couch_rep.erl
@@ -308,15 +308,15 @@ compare_rep_history([{S}|SourceRest], [{T}|TargetRest]=Target) ->
end
end.
-close_db(#http_db{})->
+close_db(#http_db{}) ->
ok;
-close_db(Db)->
+close_db(Db) ->
couch_db:close(Db).
-dbname(#http_db{} = Db) ->
- Db#http_db.url;
-dbname(Db) ->
- Db#db.name.
+dbname(#http_db{url = Url}) ->
+ Url;
+dbname(#db{name = Name}) ->
+ Name.
dbinfo(#http_db{} = Db) ->
{DbProps} = couch_rep_httpc:request(Db),
diff --git a/src/couchdb/couch_rep_att.erl b/src/couchdb/couch_rep_att.erl
index 04674fe2..6b576a01 100644
--- a/src/couchdb/couch_rep_att.erl
+++ b/src/couchdb/couch_rep_att.erl
@@ -16,9 +16,9 @@
-include("couch_db.hrl").
-convert_stub(#att{data=stub} = Attachment, {#http_db{} = Db, Id, Rev}) ->
+convert_stub(#att{data=stub, name=Name} = Attachment,
+ {#http_db{} = Db, Id, Rev}) ->
{Pos, [RevId|_]} = Rev,
- Name = Attachment#att.name,
Request = Db#http_db{
resource = lists:flatten([couch_util:url_encode(Id), "/",
couch_util:url_encode(Name)]),
diff --git a/src/couchdb/couch_rep_httpc.erl b/src/couchdb/couch_rep_httpc.erl
index 60d44cab..4944f554 100644
--- a/src/couchdb/couch_rep_httpc.erl
+++ b/src/couchdb/couch_rep_httpc.erl
@@ -17,7 +17,7 @@
-export([db_exists/1, db_exists/2, full_url/1, request/1, redirected_request/2,
spawn_worker_process/1, spawn_link_worker_process/1]).
-request(Req) when is_record(Req, http_db) ->
+request(#http_db{} = Req) ->
do_request(Req).
do_request(#http_db{url=Url} = Req) when is_binary(Url) ->
diff --git a/src/couchdb/couch_rep_writer.erl b/src/couchdb/couch_rep_writer.erl
index ffe9768f..b86028ce 100644
--- a/src/couchdb/couch_rep_writer.erl
+++ b/src/couchdb/couch_rep_writer.erl
@@ -44,13 +44,13 @@ writer_loop(Parent, Reader, Target) ->
writer_loop(Parent, Reader, Target)
end.
-write_docs(#http_db{} = Db, Docs) ->
+write_docs(#http_db{headers = Headers} = Db, Docs) ->
JsonDocs = [couch_doc:to_json_obj(Doc, [revs,attachments]) || Doc <- Docs],
Request = Db#http_db{
resource = "_bulk_docs",
method = post,
body = {[{new_edits, false}, {docs, JsonDocs}]},
- headers = [{"x-couch-full-commit", "false"} | Db#http_db.headers]
+ headers = [{"x-couch-full-commit", "false"} | Headers]
},
ErrorsJson = case couch_rep_httpc:request(Request) of
{FailProps} ->
@@ -58,16 +58,14 @@ write_docs(#http_db{} = Db, Docs) ->
List when is_list(List) ->
List
end,
- ErrorsList =
- lists:map(
- fun({Props}) ->
- Id = proplists:get_value(<<"id">>, Props),
- Rev = couch_doc:parse_rev(proplists:get_value(<<"rev">>, Props)),
- ErrId = couch_util:to_existing_atom(
- proplists:get_value(<<"error">>, Props)),
- Reason = proplists:get_value(<<"reason">>, Props),
- {{Id, Rev}, {ErrId, Reason}}
- end, ErrorsJson),
+ ErrorsList = [write_docs_1(V) || V <- ErrorsJson],
{ok, ErrorsList};
write_docs(Db, Docs) ->
couch_db:update_docs(Db, Docs, [delay_commit], replicated_changes).
+
+write_docs_1({Props}) ->
+ Id = proplists:get_value(<<"id">>, Props),
+ Rev = couch_doc:parse_rev(proplists:get_value(<<"rev">>, Props)),
+ ErrId = couch_util:to_existing_atom(proplists:get_value(<<"error">>, Props)),
+ Reason = proplists:get_value(<<"reason">>, Props),
+ {{Id, Rev}, {ErrId, Reason}}.
diff --git a/src/couchdb/couch_stats_aggregator.erl b/src/couchdb/couch_stats_aggregator.erl
index d067f354..76620093 100644
--- a/src/couchdb/couch_stats_aggregator.erl
+++ b/src/couchdb/couch_stats_aggregator.erl
@@ -173,8 +173,8 @@ new_value(incremental, Value, Current) ->
new_value(absolute, Value, _Current) ->
Value.
-add_value(Time, Value, #aggregate{count=Count}=Agg) when Count < 1 ->
- Samples = case Agg#aggregate.seconds of
+add_value(Time, Value, #aggregate{count=Count, seconds=Secs}=Agg) when Count < 1 ->
+ Samples = case Secs of
0 -> [];
_ -> [{Time, Value}]
end,
@@ -233,8 +233,8 @@ rem_values(Time, Agg) ->
end, Agg, Remove),
Agg2#aggregate{samples=Keep}.
-rem_value(_Value, #aggregate{count=Count}=Agg) when Count =< 1 ->
- #aggregate{seconds=Agg#aggregate.seconds};
+rem_value(_Value, #aggregate{count=Count, seconds=Secs}) when Count =< 1 ->
+ #aggregate{seconds=Secs};
rem_value(Value, Agg) ->
#aggregate{
count=Count,
diff --git a/src/couchdb/couch_util.erl b/src/couchdb/couch_util.erl
index cdae0d4a..d02be8d9 100644
--- a/src/couchdb/couch_util.erl
+++ b/src/couchdb/couch_util.erl
@@ -49,11 +49,11 @@ start_driver(LibDir) ->
% works like list_to_existing_atom, except can be list or binary and it
% gives you the original value instead of an error if no existing atom.
-to_existing_atom(V) when is_list(V)->
+to_existing_atom(V) when is_list(V) ->
try list_to_existing_atom(V) catch _ -> V end;
-to_existing_atom(V) when is_binary(V)->
+to_existing_atom(V) when is_binary(V) ->
try list_to_existing_atom(?b2l(V)) catch _ -> V end;
-to_existing_atom(V) when is_atom(V)->
+to_existing_atom(V) when is_atom(V) ->
V.
@@ -76,7 +76,7 @@ to_digit(N) when N < 10 -> $0 + N;
to_digit(N) -> $a + N-10.
-parse_term(Bin) when is_binary(Bin)->
+parse_term(Bin) when is_binary(Bin) ->
parse_term(binary_to_list(Bin));
parse_term(List) ->
{ok, Tokens, _} = erl_scan:string(List ++ "."),
@@ -148,14 +148,14 @@ ascii_lower([], Acc) ->
lists:reverse(Acc);
ascii_lower([Char | RestString], Acc) when Char >= $A, Char =< $B ->
ascii_lower(RestString, [Char + ($a-$A) | Acc]);
-ascii_lower([Char | RestString], Acc)->
+ascii_lower([Char | RestString], Acc) ->
ascii_lower(RestString, [Char | Acc]).
% Is a character whitespace?
-is_whitespace($\s)-> true;
-is_whitespace($\t)-> true;
-is_whitespace($\n)-> true;
-is_whitespace($\r)-> true;
+is_whitespace($\s) -> true;
+is_whitespace($\t) -> true;
+is_whitespace($\n) -> true;
+is_whitespace($\r) -> true;
is_whitespace(_Else) -> false.
@@ -206,9 +206,8 @@ collate(A, B, Options) when is_binary(A), is_binary(B) ->
true -> 1; % Case insensitive
false -> 0 % Case sensitive
end,
-
- SizeA = size(A),
- SizeB = size(B),
+ SizeA = byte_size(A),
+ SizeB = byte_size(B),
Bin = <<SizeA:32/native, A/binary, SizeB:32/native, B/binary>>,
[Result] = erlang:port_control(drv_port(), Operation, Bin),
% Result is 0 for lt, 1 for eq and 2 for gt. Subtract 1 to return the
@@ -227,9 +226,7 @@ should_flush(MemThreshHold) ->
{memory, ProcMem2} = process_info(self(), memory),
BinMem2 = lists:foldl(fun({_Id, Size, _NRefs}, Acc) -> Size+Acc end,
0, element(2,process_info(self(), binary))),
- if ProcMem2+BinMem2 > MemThreshHold ->
- true;
- true -> false end;
+ ProcMem2+BinMem2 > MemThreshHold;
true -> false end.
@@ -287,7 +284,7 @@ encodeBase64Url(<<>>, Acc) ->
%%
%% decodeBase64(BinaryChars) -> Binary
%%
-decodeBase64(Cs) when is_list(Cs)->
+decodeBase64(Cs) when is_list(Cs) ->
decodeBase64(list_to_binary(Cs));
decodeBase64(Cs) ->
decode1(Cs, <<>>).
@@ -304,7 +301,7 @@ decode1(<<C1, C2, C3, C4, Cs/binary>>, Acc) ->
decode1(<<>>, Acc) ->
Acc.
-decodeBase64Url(Cs) when is_list(Cs)->
+decodeBase64Url(Cs) when is_list(Cs) ->
decodeBase64Url(list_to_binary(Cs));
decodeBase64Url(Cs) ->
decode1Url(Cs, <<>>).
diff --git a/src/couchdb/couch_view.erl b/src/couchdb/couch_view.erl
index 778756ef..f80ce434 100644
--- a/src/couchdb/couch_view.erl
+++ b/src/couchdb/couch_view.erl
@@ -89,9 +89,9 @@ cleanup_index_files(Db) ->
RegExp = "("++ string:join(Sigs, "|") ++")",
% filter out the ones in use
- DeleteFiles = lists:filter(fun(FilePath) ->
- re:run(FilePath, RegExp, [{capture, none}]) == nomatch
- end, FileList),
+ DeleteFiles = [FilePath
+ || FilePath <- FileList,
+ re:run(FilePath, RegExp, [{capture, none}]) =:= nomatch],
% delete unused files
?LOG_DEBUG("deleting unused view index files: ~p",[DeleteFiles]),
[file:delete(File)||File <- DeleteFiles],
@@ -369,12 +369,10 @@ less_json_ids({JsonA, IdA}, {JsonB, IdB}) ->
less_json(A, B) ->
TypeA = type_sort(A),
TypeB = type_sort(B),
- if
- TypeA == TypeB ->
- Less = less_same_type(A,B),
- Less;
- true ->
- TypeA < TypeB
+ if TypeA == TypeB ->
+ less_same_type(A, B);
+ true ->
+ TypeA < TypeB
end.
type_sort(V) when is_atom(V) -> 0;
diff --git a/src/couchdb/couch_view_compactor.erl b/src/couchdb/couch_view_compactor.erl
index ad217d97..895556bf 100644
--- a/src/couchdb/couch_view_compactor.erl
+++ b/src/couchdb/couch_view_compactor.erl
@@ -49,7 +49,7 @@ compact_group(Group, EmptyGroup) ->
couch_task_status:add_task(<<"View Group Compaction">>, TaskName, <<"">>),
Fun = fun(KV, {Bt, Acc, TotalCopied}) ->
- if TotalCopied rem 10000 == 0 ->
+ if TotalCopied rem 10000 =:= 0 ->
couch_task_status:update("Copied ~p of ~p Ids (~p%)",
[TotalCopied, Count, (TotalCopied*100) div Count]),
{ok, Bt2} = couch_btree:add(Bt, lists:reverse([KV|Acc])),
@@ -81,7 +81,7 @@ compact_view(View, EmptyView) ->
%% Key is {Key,DocId}
Fun = fun(KV, {Bt, Acc, TotalCopied}) ->
- if TotalCopied rem 10000 == 0 ->
+ if TotalCopied rem 10000 =:= 0 ->
couch_task_status:update("View #~p: copied ~p of ~p KVs (~p%)",
[View#view.id_num, TotalCopied, Count, (TotalCopied*100) div Count]),
{ok, Bt2} = couch_btree:add(Bt, lists:reverse([KV|Acc])),
diff --git a/src/couchdb/couch_view_updater.erl b/src/couchdb/couch_view_updater.erl
index 475f59ac..aa7d98fc 100644
--- a/src/couchdb/couch_view_updater.erl
+++ b/src/couchdb/couch_view_updater.erl
@@ -16,6 +16,8 @@
-include("couch_db.hrl").
+-spec update(_, #group{}) -> no_return().
+
update(Owner, Group) ->
#group{
db = #db{name=DbName} = Db,
diff --git a/src/couchdb/couch_work_queue.erl b/src/couchdb/couch_work_queue.erl
index e00b52a3..ca9445d3 100644
--- a/src/couchdb/couch_work_queue.erl
+++ b/src/couchdb/couch_work_queue.erl
@@ -52,7 +52,7 @@ terminate(_Reason, #q{work_waiter=WW}) ->
gen_server:reply(WW, closed).
handle_call({queue, Item}, From, #q{work_waiter=nil}=Q0) ->
- Q = Q0#q{size=Q0#q.size + size(term_to_binary(Item)),
+ Q = Q0#q{size=Q0#q.size + byte_size(term_to_binary(Item)),
items=Q0#q.items + 1,
buffer=[Item | Q0#q.buffer]},
case (Q#q.size >= Q#q.max_size) orelse