diff options
25 files changed, 126 insertions, 126 deletions
diff --git a/src/couchdb/couch_btree.erl b/src/couchdb/couch_btree.erl index 88fca338..a076b34c 100644 --- a/src/couchdb/couch_btree.erl +++ b/src/couchdb/couch_btree.erl @@ -147,10 +147,10 @@ fold(#btree{root=Root}=Bt, Fun, Acc, Options) ->      Result =      case couch_util:get_value(start_key, Options) of      undefined -> -        stream_node(Bt, [], Bt#btree.root, InRange, Dir,  +        stream_node(Bt, [], Bt#btree.root, InRange, Dir,                  convert_fun_arity(Fun), Acc);      StartKey -> -        stream_node(Bt, [], Bt#btree.root, StartKey, InRange, Dir,  +        stream_node(Bt, [], Bt#btree.root, StartKey, InRange, Dir,                  convert_fun_arity(Fun), Acc)      end,      case Result of diff --git a/src/couchdb/couch_changes.erl b/src/couchdb/couch_changes.erl index 0c4691d0..52ebea69 100644 --- a/src/couchdb/couch_changes.erl +++ b/src/couchdb/couch_changes.erl @@ -82,7 +82,7 @@ make_filter_fun(FilterName, Style, Req, Db) ->              main_only ->                  [{[{<<"rev">>, couch_doc:rev_to_str(Rev)}]}];              all_docs -> -                [{[{<<"rev">>, couch_doc:rev_to_str(R)}]}  +                [{[{<<"rev">>, couch_doc:rev_to_str(R)}]}                          || #rev_info{rev=R} <- Revs]              end          end; @@ -93,7 +93,7 @@ make_filter_fun(FilterName, Style, Req, Db) ->          #doc{body={Props}} = DDoc,          couch_util:get_nested_json_value({Props}, [<<"filters">>, FName]),          fun(DocInfo) -> -            DocInfos =  +            DocInfos =              case Style of              main_only ->                  [DocInfo]; diff --git a/src/couchdb/couch_config.erl b/src/couchdb/couch_config.erl index 0e0c3fcb..d2ffda69 100644 --- a/src/couchdb/couch_config.erl +++ b/src/couchdb/couch_config.erl @@ -231,7 +231,7 @@ parse_ini_file(IniFile) ->                      [[]] ->                          % empty line means delete this key                          ets:delete(?MODULE, {AccSectionName, ValueName}), -                        {AccSectionName, AccValues};                         +                        {AccSectionName, AccValues};                      [LineValue | _Rest] ->                          {AccSectionName,                              [{{AccSectionName, ValueName}, LineValue} | AccValues]} diff --git a/src/couchdb/couch_db.erl b/src/couchdb/couch_db.erl index cd077a2f..5222cb5e 100644 --- a/src/couchdb/couch_db.erl +++ b/src/couchdb/couch_db.erl @@ -138,7 +138,7 @@ apply_open_options2(#doc{atts=Atts,revs=Revs}=Doc,          [{atts_since, PossibleAncestors}|Rest]) ->      RevPos = find_ancestor_rev_pos(Revs, PossibleAncestors),      apply_open_options2(Doc#doc{atts=[A#att{data= -        if AttPos>RevPos -> Data; true -> stub end}  +        if AttPos>RevPos -> Data; true -> stub end}          || #att{revpos=AttPos,data=Data}=A <- Atts]}, Rest);  apply_open_options2(Doc,[_|Rest]) ->      apply_open_options2(Doc,Rest). @@ -180,9 +180,9 @@ find_missing([{Id, Revs}|RestIdRevs], [{ok, FullInfo} | RestLookupInfo]) ->          % Find the revs that are possible parents of this rev          PossibleAncestors =          lists:foldl(fun({LeafPos, LeafRevId}, Acc) -> -            % this leaf is a "possible ancenstor" of the missing  +            % this leaf is a "possible ancenstor" of the missing              % revs if this LeafPos lessthan any of the missing revs -            case lists:any(fun({MissingPos, _}) ->  +            case lists:any(fun({MissingPos, _}) ->                      LeafPos < MissingPos end, MissingRevs) of              true ->                  [{LeafPos, LeafRevId} | Acc]; @@ -290,7 +290,7 @@ check_is_reader(#db{user_ctx=#user_ctx{name=Name,roles=Roles}=UserCtx}=Db) ->          ReaderRoles = couch_util:get_value(<<"roles">>, Readers,[]),          WithAdminRoles = [<<"_admin">> | ReaderRoles],          ReaderNames = couch_util:get_value(<<"names">>, Readers,[]), -        case ReaderRoles ++ ReaderNames of  +        case ReaderRoles ++ ReaderNames of          [] -> ok; % no readers == public access          _Else ->              case WithAdminRoles -- Roles of @@ -458,12 +458,12 @@ prep_and_validate_update(Db, #doc{id=Id,revs={RevStart, Revs}}=Doc,  prep_and_validate_updates(_Db, [], [], _AllowConflict, AccPrepped,          AccFatalErrors) ->     {AccPrepped, AccFatalErrors}; -prep_and_validate_updates(Db, [DocBucket|RestBuckets], [not_found|RestLookups],  +prep_and_validate_updates(Db, [DocBucket|RestBuckets], [not_found|RestLookups],          AllowConflict, AccPrepped, AccErrors) ->      [#doc{id=Id}|_]=DocBucket,      % no existing revs are known,      {PreppedBucket, AccErrors3} = lists:foldl( -        fun(#doc{revs=Revs}=Doc, {AccBucket, AccErrors2}) ->        +        fun(#doc{revs=Revs}=Doc, {AccBucket, AccErrors2}) ->              case couch_doc:has_stubs(Doc) of              true ->                  couch_doc:merge_stubs(Doc, #doc{}); % will throw exception @@ -504,7 +504,7 @@ prep_and_validate_updates(Db, [DocBucket|RestBuckets],              end          end,          {[], AccErrors}, DocBucket), -    prep_and_validate_updates(Db, RestBuckets, RestLookups, AllowConflict,  +    prep_and_validate_updates(Db, RestBuckets, RestLookups, AllowConflict,              [PreppedBucket | AccPrepped], AccErrors3). @@ -520,7 +520,7 @@ prep_and_validate_replicated_updates(Db, [Bucket|RestBuckets], [OldInfo|RestOldI      case OldInfo of      not_found ->          {ValidatedBucket, AccErrors3} = lists:foldl( -            fun(Doc, {AccPrepped2, AccErrors2}) ->                 +            fun(Doc, {AccPrepped2, AccErrors2}) ->                  case couch_doc:has_stubs(Doc) of                  true ->                      couch_doc:merge_doc(Doc, #doc{}); % will throw exception @@ -923,7 +923,7 @@ changes_since(Db, Style, StartSeq, Fun, Options, Acc) ->              #doc_info{revs=Revs} = DocInfo,              DocInfo2 =              case Style of -            main_only ->     +            main_only ->                  DocInfo;              all_docs ->                  % remove revs before the seq @@ -932,7 +932,7 @@ changes_since(Db, Style, StartSeq, Fun, Options, Acc) ->              end,              Fun(DocInfo2, Acc2)          end, -    {ok, _LastReduction, AccOut} = couch_btree:fold(Db#db.docinfo_by_seq_btree,  +    {ok, _LastReduction, AccOut} = couch_btree:fold(Db#db.docinfo_by_seq_btree,          Wrapper, Acc, [{start_key, StartSeq + 1}] ++ Options),      {ok, AccOut}. diff --git a/src/couchdb/couch_db_updater.erl b/src/couchdb/couch_db_updater.erl index efa44895..df682f62 100644 --- a/src/couchdb/couch_db_updater.erl +++ b/src/couchdb/couch_db_updater.erl @@ -187,11 +187,11 @@ handle_cast({compact_done, CompactFilepath}, #db{filepath=Filepath}=Db) ->      end. -handle_info({update_docs, Client, GroupedDocs, NonRepDocs, MergeConflicts,  +handle_info({update_docs, Client, GroupedDocs, NonRepDocs, MergeConflicts,          FullCommit}, Db) ->      GroupedDocs2 = [[{Client, D} || D <- DocGroup] || DocGroup <- GroupedDocs],      if NonRepDocs == [] -> -        {GroupedDocs3, Clients, FullCommit2} = collect_updates(GroupedDocs2,  +        {GroupedDocs3, Clients, FullCommit2} = collect_updates(GroupedDocs2,                  [Client], MergeConflicts, FullCommit);      true ->          GroupedDocs3 = GroupedDocs2, @@ -199,7 +199,7 @@ handle_info({update_docs, Client, GroupedDocs, NonRepDocs, MergeConflicts,          Clients = [Client]      end,      NonRepDocs2 = [{Client, NRDoc} || NRDoc <- NonRepDocs], -    try update_docs_int(Db, GroupedDocs3, NonRepDocs2, MergeConflicts,  +    try update_docs_int(Db, GroupedDocs3, NonRepDocs2, MergeConflicts,                  FullCommit2) of      {ok, Db2} ->          ok = gen_server:call(Db#db.main_pid, {db_updated, Db2}), @@ -254,7 +254,7 @@ collect_updates(GroupedDocsAcc, ClientsAcc, MergeConflicts, FullCommit) ->          {update_docs, Client, GroupedDocs, [], MergeConflicts, FullCommit2} ->              GroupedDocs2 = [[{Client, Doc} || Doc <- DocGroup]                      || DocGroup <- GroupedDocs], -            GroupedDocsAcc2 =  +            GroupedDocsAcc2 =                  merge_updates(GroupedDocsAcc, GroupedDocs2, []),              collect_updates(GroupedDocsAcc2, [Client | ClientsAcc],                      MergeConflicts, (FullCommit or FullCommit2)) @@ -505,7 +505,7 @@ merge_rev_trees(MergeConflicts, [NewDocs|RestDocsList],                          % this means we are recreating a brand new document                          % into a state that already existed before.                          % put the rev into a subsequent edit of the deletion -                        #doc_info{revs=[#rev_info{rev={OldPos,OldRev}}|_]} =  +                        #doc_info{revs=[#rev_info{rev={OldPos,OldRev}}|_]} =                                  couch_doc:to_doc_info(OldDocInfo),                          NewRevId = couch_db:new_revid(                                  NewDoc#doc{revs={OldPos, [OldRev]}}), @@ -513,7 +513,7 @@ merge_rev_trees(MergeConflicts, [NewDocs|RestDocsList],                          {NewTree2, _} = couch_key_tree:merge(AccTree,                                  [couch_db:doc_to_tree(NewDoc2)]),                          % we changed the rev id, this tells the caller we did -                        send_result(Client, Id, {Pos-1,PrevRevs},  +                        send_result(Client, Id, {Pos-1,PrevRevs},                                  {ok, {OldPos + 1, NewRevId}}),                          NewTree2;                      true -> @@ -527,7 +527,7 @@ merge_rev_trees(MergeConflicts, [NewDocs|RestDocsList],                  {NewTree, _} = couch_key_tree:merge(AccTree,                              [couch_db:doc_to_tree(NewDoc)]),                  NewTree -            end  +            end          end,          OldTree, NewDocs),      if NewRevTree == OldTree -> @@ -823,7 +823,7 @@ copy_compact(Db, NewDb0, Retry) ->      couch_task_status:set_update_frequency(500),      {ok, _, {NewDb2, Uncopied, TotalChanges}} = -        couch_btree:foldl(Db#db.docinfo_by_seq_btree, EnumBySeqFun,  +        couch_btree:foldl(Db#db.docinfo_by_seq_btree, EnumBySeqFun,              {NewDb, [], 0},              [{start_key, NewDb#db.update_seq + 1}]), diff --git a/src/couchdb/couch_doc.erl b/src/couchdb/couch_doc.erl index 025d4e55..cb8a116c 100644 --- a/src/couchdb/couch_doc.erl +++ b/src/couchdb/couch_doc.erl @@ -382,7 +382,7 @@ len_doc_to_multi_part_stream(Boundary, JsonBytes, Atts,              stub ->                  AccAttsSize;              _ -> -                AccAttsSize +   +                AccAttsSize +                  4 + % "\r\n\r\n"                  case SendEncodedAtts of                  true -> @@ -424,7 +424,7 @@ atts_to_mp([], _Boundary, WriteFun, _SendEncAtts) ->  atts_to_mp([#att{data=stub} | RestAtts], Boundary, WriteFun,          SendEncodedAtts) ->      atts_to_mp(RestAtts, Boundary, WriteFun, SendEncodedAtts); -atts_to_mp([Att | RestAtts], Boundary, WriteFun,  +atts_to_mp([Att | RestAtts], Boundary, WriteFun,          SendEncodedAtts)  ->      WriteFun(<<"\r\n\r\n">>),      AttFun = case SendEncodedAtts of @@ -440,7 +440,7 @@ atts_to_mp([Att | RestAtts], Boundary, WriteFun,  doc_from_multi_part_stream(ContentType, DataFun) ->      Self = self(), -    Parser = spawn_link(fun() ->  +    Parser = spawn_link(fun() ->          couch_httpd:parse_multipart_request(ContentType, DataFun,                  fun(Next)-> mp_parse_doc(Next, []) end),          unlink(Self) diff --git a/src/couchdb/couch_file.erl b/src/couchdb/couch_file.erl index 0d79afb2..34d0c453 100644 --- a/src/couchdb/couch_file.erl +++ b/src/couchdb/couch_file.erl @@ -89,7 +89,7 @@ append_binary(Fd, Bin) ->  append_binary_md5(Fd, Bin) ->      Size = iolist_size(Bin), -    gen_server:call(Fd, {append_bin,  +    gen_server:call(Fd, {append_bin,              [<<1:1/integer,Size:31/integer>>, couch_util:md5(Bin), Bin]}, infinity). @@ -266,7 +266,7 @@ handle_call({pread_iolist, Pos}, _From, File) ->          end;      <<0:1/integer,Len:31/integer>> ->          {Iolist, _} = read_raw_iolist_int(File, NextPos, Len), -        {reply, {ok, Iolist}, File}  +        {reply, {ok, Iolist}, File}      end;  handle_call({pread, Pos, Bytes}, _From, #file{fd=Fd,tail_append_begin=TailAppendBegin}=File) ->      {ok, Bin} = file:pread(Fd, Pos, Bytes), diff --git a/src/couchdb/couch_httpd.erl b/src/couchdb/couch_httpd.erl index 6c08847c..d9e91e4b 100644 --- a/src/couchdb/couch_httpd.erl +++ b/src/couchdb/couch_httpd.erl @@ -371,7 +371,7 @@ absolute_uri(#httpd{mochi_req=MochiReq}, Path) ->      Host = case MochiReq:get_header_value(XHost) of          undefined ->              case MochiReq:get_header_value("Host") of -                undefined ->     +                undefined ->                      {ok, {Address, Port}} = inet:sockname(MochiReq:get(socket)),                      inet_parse:ntoa(Address) ++ ":" ++ integer_to_list(Port);                  Value1 -> @@ -601,11 +601,11 @@ start_jsonp(Req) ->              try                  % make sure jsonp is configured on (default off)                  case couch_config:get("httpd", "allow_jsonp", "false") of -                "true" ->  +                "true" ->                      validate_callback(CallBack),                      CallBack ++ "("; -                _Else ->  -                    % this could throw an error message, but instead we just ignore the  +                _Else -> +                    % this could throw an error message, but instead we just ignore the                      % jsonp parameter                      % throw({bad_request, <<"JSONP must be configured before using.">>})                      put(jsonp, no_jsonp), @@ -763,9 +763,9 @@ parse_multipart_request(ContentType, DataFun, Callback) ->              buffer= <<>>,              data_fun=DataFun,              callback=Callback}, -    {Mp2, _NilCallback} = read_until(Mp, <<"--", Boundary0/binary>>,  +    {Mp2, _NilCallback} = read_until(Mp, <<"--", Boundary0/binary>>,          fun(Next)-> nil_callback(Next) end), -    #mp{buffer=Buffer, data_fun=DataFun2, callback=Callback2} =  +    #mp{buffer=Buffer, data_fun=DataFun2, callback=Callback2} =              parse_part_header(Mp2),      {Buffer, DataFun2, Callback2}. diff --git a/src/couchdb/couch_httpd_auth.erl b/src/couchdb/couch_httpd_auth.erl index ea5e1b46..572edbcc 100644 --- a/src/couchdb/couch_httpd_auth.erl +++ b/src/couchdb/couch_httpd_auth.erl @@ -100,18 +100,18 @@ default_authentication_handler(Req) ->  null_authentication_handler(Req) ->      Req#httpd{user_ctx=#user_ctx{roles=[<<"_admin">>]}}. -%% @doc proxy auth handler.  +%% @doc proxy auth handler.  % -% This handler allows creation of a userCtx object from a user authenticated remotly.  -% The client just pass specific headers to CouchDB and the handler create the userCtx.  +% This handler allows creation of a userCtx object from a user authenticated remotly. +% The client just pass specific headers to CouchDB and the handler create the userCtx.  % Headers  name can be defined in local.ini. By thefault they are :  % -%   * X-Auth-CouchDB-UserName : contain the username, (x_auth_username in  +%   * X-Auth-CouchDB-UserName : contain the username, (x_auth_username in  %   couch_httpd_auth section) -%   * X-Auth-CouchDB-Roles : contain the user roles, list of roles separated by a  +%   * X-Auth-CouchDB-Roles : contain the user roles, list of roles separated by a  %   comma (x_auth_roles in couch_httpd_auth section) -%   * X-Auth-CouchDB-Token : token to authenticate the authorization (x_auth_token  -%   in couch_httpd_auth section). This token is an hmac-sha1 created from secret key  +%   * X-Auth-CouchDB-Token : token to authenticate the authorization (x_auth_token +%   in couch_httpd_auth section). This token is an hmac-sha1 created from secret key  %   and username. The secret key should be the same in the client and couchdb node. s  %   ecret key is the secret key in couch_httpd_auth section of ini. This token is optional  %   if value of proxy_use_secret key in couch_httpd_auth section of ini isn't true. @@ -127,14 +127,14 @@ proxy_auth_user(Req) ->                                  "X-Auth-CouchDB-UserName"),      XHeaderRoles = couch_config:get("couch_httpd_auth", "x_auth_roles",                                  "X-Auth-CouchDB-Roles"), -    XHeaderToken = couch_config:get("couch_httpd_auth", "x_auth_token",  +    XHeaderToken = couch_config:get("couch_httpd_auth", "x_auth_token",                                  "X-Auth-CouchDB-Token"),      case header_value(Req, XHeaderUserName) of          undefined -> nil;          UserName ->              Roles = case header_value(Req, XHeaderRoles) of                  undefined -> []; -                Else ->   +                Else ->                      [?l2b(R) || R <- string:tokens(Else, ",")]              end,              case couch_config:get("couch_httpd_auth", "proxy_use_secret", "false") of @@ -153,7 +153,7 @@ proxy_auth_user(Req) ->                      end;                  _ ->                      Req#httpd{user_ctx=#user_ctx{name=?l2b(UserName), roles=Roles}} -            end            +            end      end.  % maybe we can use hovercraft to simplify running this view query @@ -165,7 +165,7 @@ get_user(UserName) ->          % which has a matching name, salt, and password_sha          [HashedPwd, Salt] = string:tokens(HashedPwdAndSalt, ","),          case get_user_props_from_db(UserName) of -            nil ->         +            nil ->                  [{<<"roles">>, [<<"_admin">>]},                    {<<"salt">>, ?l2b(Salt)},                    {<<"password_sha">>, ?l2b(HashedPwd)}]; @@ -187,12 +187,12 @@ get_user_props_from_db(UserName) ->          #doc{meta=Meta}=Doc ->              %  check here for conflict state and throw error if conflicted              case couch_util:get_value(conflicts,Meta,[]) of -                [] ->  +                [] ->                      {DocProps} = couch_query_servers:json_doc(Doc),                      case couch_util:get_value(<<"type">>, DocProps) of                          <<"user">> ->                              DocProps; -                        _Else ->  +                        _Else ->                              ?LOG_ERROR("Invalid user doc. Id: ~p",[DocId]),                              nil                      end; @@ -212,17 +212,17 @@ ensure_users_db_exists(DbName) ->      {ok, Db} ->          ensure_auth_ddoc_exists(Db, <<"_design/_auth">>),          {ok, Db}; -    _Error ->  +    _Error ->          {ok, Db} = couch_db:create(DbName, [{user_ctx, #user_ctx{roles=[<<"_admin">>]}}]),          ensure_auth_ddoc_exists(Db, <<"_design/_auth">>),          {ok, Db}      end. -ensure_auth_ddoc_exists(Db, DDocId) ->  +ensure_auth_ddoc_exists(Db, DDocId) ->      try couch_httpd_db:couch_doc_open(Db, DDocId, nil, []) of          _Foo -> ok -    catch  -        _:_Error ->  +    catch +        _:_Error ->              % create the design document              {ok, AuthDesign} = auth_design_doc(DDocId),              {ok, _Rev} = couch_db:update_doc(Db, AuthDesign, []), @@ -241,7 +241,7 @@ auth_design_doc(DocId) ->                      throw({forbidden : 'doc.type must be user'});                  } // we only validate user docs for now                  if (newDoc._deleted === true) { -                    // allow deletes by admins and matching users  +                    // allow deletes by admins and matching users                      // without checking the other fields                      if ((userCtx.roles.indexOf('_admin') != -1) || (userCtx.name == oldDoc.name)) {                          return; @@ -304,7 +304,7 @@ cookie_authentication_handler(#httpd{mochi_req=MochiReq}=Req) ->      case MochiReq:get_cookie_value("AuthSession") of      undefined -> Req;      [] -> Req; -    Cookie ->  +    Cookie ->          [User, TimeStr | HashParts] = try              AuthSession = couch_util:decodeBase64Url(Cookie),              [_A, _B | _Cs] = string:tokens(?b2l(AuthSession), ":") @@ -316,7 +316,7 @@ cookie_authentication_handler(#httpd{mochi_req=MochiReq}=Req) ->          % Verify expiry and hash          CurrentTime = make_cookie_time(),          case couch_config:get("couch_httpd_auth", "secret", nil) of -        nil ->  +        nil ->              ?LOG_ERROR("cookie auth secret is not set",[]),              Req;          SecretStr -> @@ -454,7 +454,7 @@ handle_session_req(#httpd{method='GET', user_ctx=UserCtx}=Req) ->                      {authentication_db, ?l2b(couch_config:get("couch_httpd_auth", "authentication_db"))},                      {authentication_handlers, [auth_name(H) || H <- couch_httpd:make_fun_spec_strs(                              couch_config:get("httpd", "authentication_handlers"))]} -                ] ++ maybe_value(authenticated, UserCtx#user_ctx.handler, fun(Handler) ->  +                ] ++ maybe_value(authenticated, UserCtx#user_ctx.handler, fun(Handler) ->                          auth_name(?b2l(Handler))                      end)}}              ]}) @@ -473,7 +473,7 @@ handle_session_req(Req) ->      send_method_not_allowed(Req, "GET,HEAD,POST,DELETE").  maybe_value(_Key, undefined, _Fun) -> []; -maybe_value(Key, Else, Fun) ->  +maybe_value(Key, Else, Fun) ->      [{Key, Fun(Else)}].  auth_name(String) when is_list(String) -> @@ -481,7 +481,7 @@ auth_name(String) when is_list(String) ->      ?l2b(Name).  to_int(Value) when is_binary(Value) -> -    to_int(?b2l(Value));  +    to_int(?b2l(Value));  to_int(Value) when is_list(Value) ->      list_to_integer(Value);  to_int(Value) when is_integer(Value) -> diff --git a/src/couchdb/couch_httpd_db.erl b/src/couchdb/couch_httpd_db.erl index de434f32..1f16ffcc 100644 --- a/src/couchdb/couch_httpd_db.erl +++ b/src/couchdb/couch_httpd_db.erl @@ -162,7 +162,7 @@ create_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) ->      ok = couch_httpd:verify_is_server_admin(Req),      LDbName = ?b2l(DbName),      case couch_config:get("couch_httpd_auth", "authentication_db") of -        LDbName ->  +        LDbName ->              % make sure user's db always has the auth ddoc              {ok, Db} = couch_httpd_auth:ensure_users_db_exists(DbName),              couch_db:close(Db), @@ -192,7 +192,7 @@ do_db_req(#httpd{user_ctx=UserCtx,path_parts=[DbName|_]}=Req, Fun) ->      LDbName = ?b2l(DbName),      % I hope this lookup is cheap.      case couch_config:get("couch_httpd_auth", "authentication_db") of -        LDbName ->  +        LDbName ->              % make sure user's db always has the auth ddoc              {ok, ADb} = couch_httpd_auth:ensure_users_db_exists(DbName),              couch_db:close(ADb); @@ -328,7 +328,7 @@ db_req(#httpd{method='POST',path_parts=[_,<<"_bulk_docs">>]}=Req, Db) ->              send_json(Req, 417, ErrorsJson)          end;      false -> -        Docs = lists:map(fun(JsonObj) ->  +        Docs = lists:map(fun(JsonObj) ->                  Doc = couch_doc:from_json_obj(JsonObj),                  validate_attachment_names(Doc),                  Doc @@ -389,17 +389,17 @@ db_req(#httpd{path_parts=[_,<<"_missing_revs">>]}=Req, _Db) ->  db_req(#httpd{method='POST',path_parts=[_,<<"_revs_diff">>]}=Req, Db) ->      {JsonDocIdRevs} = couch_httpd:json_body_obj(Req), -    JsonDocIdRevs2 =  +    JsonDocIdRevs2 =          [{Id, couch_doc:parse_revs(RevStrs)} || {Id, RevStrs} <- JsonDocIdRevs],      {ok, Results} = couch_db:get_missing_revs(Db, JsonDocIdRevs2), -    Results2 =  +    Results2 =      lists:map(fun({Id, MissingRevs, PossibleAncestors}) ->          {Id, -            {[{missing, couch_doc:revs_to_strs(MissingRevs)}] ++  +            {[{missing, couch_doc:revs_to_strs(MissingRevs)}] ++                  if PossibleAncestors == [] ->                      []; -                true ->  -                    [{possible_ancestors,  +                true -> +                    [{possible_ancestors,                          couch_doc:revs_to_strs(PossibleAncestors)}]                  end}}      end, Results), @@ -547,11 +547,11 @@ db_doc_req(#httpd{method='DELETE'}=Req, Db, DocId) ->      couch_doc_open(Db, DocId, nil, []),      case couch_httpd:qs_value(Req, "rev") of      undefined -> -        update_doc(Req, Db, DocId,  +        update_doc(Req, Db, DocId,                  couch_doc_from_req(Req, DocId, {[{<<"_deleted">>,true}]}));      Rev -> -        update_doc(Req, Db, DocId,  -                couch_doc_from_req(Req, DocId,  +        update_doc(Req, Db, DocId, +                couch_doc_from_req(Req, DocId,                      {[{<<"_rev">>, ?l2b(Rev)},{<<"_deleted">>,true}]}))      end; @@ -563,7 +563,7 @@ db_doc_req(#httpd{method='GET'}=Req, Db, DocId) ->          atts_since = AttsSince      } = parse_doc_query(Req),      case Revs of -    [] ->     +    [] ->          Options2 =          if AttsSince /= nil ->              [{atts_since, AttsSince}, attachments | Options]; @@ -703,7 +703,7 @@ send_doc(Req, Doc, Options) ->      [] ->          DiskEtag = couch_httpd:doc_etag(Doc),          % output etag only when we have no meta -        couch_httpd:etag_respond(Req, DiskEtag, fun() ->  +        couch_httpd:etag_respond(Req, DiskEtag, fun() ->              send_doc_efficiently(Req, Doc, [{"Etag", DiskEtag}], Options)          end);      _ -> @@ -728,7 +728,7 @@ send_doc_efficiently(Req, #doc{atts=Atts}=Doc, Headers, Options) ->              JsonBytes = ?JSON_ENCODE(couch_doc:to_json_obj(Doc, [follows|Options])),              Len = couch_doc:len_doc_to_multi_part_stream(Boundary,JsonBytes,                      Atts,false), -            CType = {<<"Content-Type">>,  +            CType = {<<"Content-Type">>,                      <<"multipart/related; boundary=\"", Boundary/binary, "\"">>},              {ok, Resp} = start_response_length(Req, 200, [CType|Headers], Len),              couch_doc:doc_to_multi_part_stream(Boundary,JsonBytes,Atts, diff --git a/src/couchdb/couch_httpd_oauth.erl b/src/couchdb/couch_httpd_oauth.erl index 42112fa7..accdfaca 100644 --- a/src/couchdb/couch_httpd_oauth.erl +++ b/src/couchdb/couch_httpd_oauth.erl @@ -20,7 +20,7 @@ oauth_authentication_handler(#httpd{mochi_req=MochiReq}=Req) ->      serve_oauth(Req, fun(URL, Params, Consumer, Signature) ->          AccessToken = couch_util:get_value("oauth_token", Params),          case couch_config:get("oauth_token_secrets", AccessToken) of -            undefined ->  +            undefined ->                  couch_httpd:send_error(Req, 400, <<"invalid_token">>,                      <<"Invalid OAuth token.">>);              TokenSecret -> diff --git a/src/couchdb/couch_httpd_show.erl b/src/couchdb/couch_httpd_show.erl index a67a82f1..c098f5ef 100644 --- a/src/couchdb/couch_httpd_show.erl +++ b/src/couchdb/couch_httpd_show.erl @@ -24,7 +24,7 @@  % /db/_design/foo/_show/bar/docid -% show converts a json doc to a response of any content-type.  +% show converts a json doc to a response of any content-type.  % it looks up the doc an then passes it to the query server.  % then it sends the response from the query server to the http client. @@ -77,7 +77,7 @@ handle_doc_show(Req, Db, DDoc, ShowName, Doc, DocId) ->      couch_httpd:etag_respond(Req, CurrentEtag, fun() ->          JsonReq = couch_httpd_external:json_req_obj(Req, Db, DocId),          JsonDoc = couch_query_servers:json_doc(Doc), -        [<<"resp">>, ExternalResp] =  +        [<<"resp">>, ExternalResp] =              couch_query_servers:ddoc_prompt(DDoc, [<<"shows">>, ShowName], [JsonDoc, JsonReq]),          JsonResp = apply_etag(ExternalResp, CurrentEtag),          couch_httpd_external:send_external_response(Req, JsonResp) @@ -125,10 +125,10 @@ handle_doc_update_req(Req, _Db, _DDoc) ->  send_doc_update_response(Req, Db, DDoc, UpdateName, Doc, DocId) ->      JsonReq = couch_httpd_external:json_req_obj(Req, Db, DocId),      JsonDoc = couch_query_servers:json_doc(Doc), -    {Code, JsonResp1} = case couch_query_servers:ddoc_prompt(DDoc,  +    {Code, JsonResp1} = case couch_query_servers:ddoc_prompt(DDoc,                  [<<"updates">>, UpdateName], [JsonDoc, JsonReq]) of          [<<"up">>, {NewJsonDoc}, {JsonResp}] -> -            Options = case couch_httpd:header_value(Req, "X-Couch-Full-Commit",  +            Options = case couch_httpd:header_value(Req, "X-Couch-Full-Commit",                  "false") of              "true" ->                  [full_commit]; @@ -138,7 +138,7 @@ send_doc_update_response(Req, Db, DDoc, UpdateName, Doc, DocId) ->              NewDoc = couch_doc:from_json_obj({NewJsonDoc}),              {ok, NewRev} = couch_db:update_doc(Db, NewDoc, Options),              NewRevStr = couch_doc:rev_to_str(NewRev), -            JsonRespWithRev =  {[{<<"headers">>,  +            JsonRespWithRev =  {[{<<"headers">>,                  {[{<<"X-Couch-Update-NewRev">>, NewRevStr}]}} | JsonResp]},              {201, JsonRespWithRev};          [<<"up">>, _Other, JsonResp] -> @@ -188,10 +188,10 @@ handle_view_list_req(Req, _Db, _DDoc) ->  handle_view_list(Req, Db, DDoc, LName, {ViewDesignName, ViewName}, Keys) ->      ViewDesignId = <<"_design/", ViewDesignName/binary>>,      {ViewType, View, Group, QueryArgs} = couch_httpd_view:load_view(Req, Db, {ViewDesignId, ViewName}, Keys), -    Etag = list_etag(Req, Db, Group, {couch_httpd:doc_etag(DDoc), Keys}),     +    Etag = list_etag(Req, Db, Group, {couch_httpd:doc_etag(DDoc), Keys}),      couch_httpd:etag_respond(Req, Etag, fun() ->              output_list(ViewType, Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group) -        end).     +        end).  list_etag(#httpd{user_ctx=UserCtx}=Req, Db, Group, More) ->      Accept = couch_httpd:header_value(Req, "Accept"), @@ -226,7 +226,7 @@ output_map_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group) ->          {ok, _, FoldResult} = case Keys of              nil ->                  FoldlFun = couch_httpd_view:make_view_fold_fun(Req, QueryArgs, Etag, Db, CurrentSeq, RowCount, ListFoldHelpers), -                    couch_view:fold(View, FoldlFun, FoldAccInit,  +                    couch_view:fold(View, FoldlFun, FoldAccInit,                      couch_httpd_view:make_key_options(QueryArgs));              Keys ->                  lists:foldl( @@ -265,16 +265,16 @@ output_reduce_list(Req, Db, DDoc, LName, View, QueryArgs, Etag, Keys, Group) ->          FoldAccInit = {Limit, SkipCount, undefined, []},          {ok, FoldResult} = case Keys of              nil -> -                couch_view:fold_reduce(View, RespFun, FoldAccInit, [{key_group_fun, GroupRowsFun} |  +                couch_view:fold_reduce(View, RespFun, FoldAccInit, [{key_group_fun, GroupRowsFun} |                      couch_httpd_view:make_key_options(QueryArgs)]);              Keys ->                  lists:foldl(                      fun(Key, {ok, FoldAcc}) ->                          couch_view:fold_reduce(View, RespFun, FoldAcc, -                            [{key_group_fun, GroupRowsFun} |  +                            [{key_group_fun, GroupRowsFun} |                                  couch_httpd_view:make_key_options(                                  QueryArgs#view_query_args{start_key=Key, end_key=Key})] -                            )     +                            )                      end, {ok, FoldAccInit}, Keys)              end,          finish_list(Req, QServer, Etag, FoldResult, StartListRespFun, CurrentSeq, null) @@ -294,7 +294,7 @@ make_reduce_start_resp_fun(QueryServer, Db, LName) ->  start_list_resp(QServer, LName, Req, Db, Head, Etag) ->      JsonReq = couch_httpd_external:json_req_obj(Req, Db), -    [<<"start">>,Chunks,JsonResp] = couch_query_servers:ddoc_proc_prompt(QServer,  +    [<<"start">>,Chunks,JsonResp] = couch_query_servers:ddoc_proc_prompt(QServer,          [<<"lists">>, LName], [Head, JsonReq]),      JsonResp2 = apply_etag(JsonResp, Etag),      #extern_resp_args{ @@ -359,7 +359,7 @@ finish_list(Req, {Proc, _DDocId}, Etag, FoldResult, StartFun, CurrentSeq, TotalR          {_, _, undefined, _, _} ->              {ok, Resp, BeginBody} =                  render_head_for_empty_list(StartFun, Req, Etag, CurrentSeq, TotalRows), -            [<<"end">>, Chunks] = couch_query_servers:proc_prompt(Proc, [<<"list_end">>]),             +            [<<"end">>, Chunks] = couch_query_servers:proc_prompt(Proc, [<<"list_end">>]),              Chunk = BeginBody ++ ?b2l(?l2b(Chunks)),              send_non_empty_chunk(Resp, Chunk);          {_, _, Resp, stop, _} -> diff --git a/src/couchdb/couch_httpd_view.erl b/src/couchdb/couch_httpd_view.erl index 7f6104fd..0675ef43 100644 --- a/src/couchdb/couch_httpd_view.erl +++ b/src/couchdb/couch_httpd_view.erl @@ -116,9 +116,9 @@ output_map_view(Req, View, Group, Db, QueryArgs, nil) ->          {ok, RowCount} = couch_view:get_row_count(View),          FoldlFun = make_view_fold_fun(Req, QueryArgs, CurrentEtag, Db, Group#group.current_seq, RowCount, #view_fold_helper_funs{reduce_count=fun couch_view:reduce_to_count/1}),          FoldAccInit = {Limit, SkipCount, undefined, []}, -        {ok, LastReduce, FoldResult} = couch_view:fold(View,  +        {ok, LastReduce, FoldResult} = couch_view:fold(View,                  FoldlFun, FoldAccInit, make_key_options(QueryArgs)), -        finish_view_fold(Req, RowCount,  +        finish_view_fold(Req, RowCount,                  couch_view:reduce_to_count(LastReduce), FoldResult)      end); @@ -524,7 +524,7 @@ apply_default_helper_funs(      }.  make_key_options(#view_query_args{direction = Dir}=QueryArgs) -> -     [{dir,Dir} | make_start_key_option(QueryArgs) ++  +     [{dir,Dir} | make_start_key_option(QueryArgs) ++              make_end_key_option(QueryArgs)].  make_start_key_option( diff --git a/src/couchdb/couch_key_tree.erl b/src/couchdb/couch_key_tree.erl index d5944119..4fe09bf3 100644 --- a/src/couchdb/couch_key_tree.erl +++ b/src/couchdb/couch_key_tree.erl @@ -290,7 +290,7 @@ map(Fun, [{Pos, Tree}|Rest]) ->  map_simple(_Fun, _Pos, []) ->      [];  map_simple(Fun, Pos, [{Key, Value, SubTree} | RestTree]) -> -    Value2 = Fun({Pos, Key}, Value,  +    Value2 = Fun({Pos, Key}, Value,              if SubTree == [] -> leaf; true -> branch end),      [{Key, Value2, map_simple(Fun, Pos + 1, SubTree)} | map_simple(Fun, Pos, RestTree)]. diff --git a/src/couchdb/couch_native_process.erl b/src/couchdb/couch_native_process.erl index 5a33dc82..b512f712 100644 --- a/src/couchdb/couch_native_process.erl +++ b/src/couchdb/couch_native_process.erl @@ -1,16 +1,16 @@ -% Licensed under the Apache License, Version 2.0 (the "License");  -% you may not use this file except in compliance with the License.  +% Licensed under the Apache License, Version 2.0 (the "License"); +% you may not use this file except in compliance with the License.  %  % You may obtain a copy of the License at  % http://www.apache.org/licenses/LICENSE-2.0  % -% Unless required by applicable law or agreed to in writing,  -% software distributed under the License is distributed on an  -% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,  -% either express or implied.  +% Unless required by applicable law or agreed to in writing, +% software distributed under the License is distributed on an +% "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, +% either express or implied.  %  % See the License for the specific language governing permissions -% and limitations under the License.  +% and limitations under the License.  %  % This file drew much inspiration from erlview, which was written by and  % copyright Michael McDaniel [http://autosys.us], and is also under APL 2.0 @@ -32,7 +32,7 @@  % which should be roughly the same as the javascript:  %    emit(doc._id, null);  % -% This module exposes enough functions such that a native erlang server can  +% This module exposes enough functions such that a native erlang server can  % act as a fully-fleged view server, but no 'helper' functions specifically  % for simplifying your erlang view code.  It is expected other third-party  % extensions will evolve which offer useful layers on top of this view server @@ -370,7 +370,7 @@ start_list_resp(Self, Sig) ->                  undefined -> {[{<<"headers">>, {[]}}]};                  CurrHdrs -> CurrHdrs              end, -            Chunks =  +            Chunks =              case erlang:get(Sig) of                  undefined -> [];                  CurrChunks -> CurrChunks diff --git a/src/couchdb/couch_query_servers.erl b/src/couchdb/couch_query_servers.erl index fb681603..b2aa7a12 100644 --- a/src/couchdb/couch_query_servers.erl +++ b/src/couchdb/couch_query_servers.erl @@ -90,7 +90,7 @@ group_reductions_results(List) ->  rereduce(_Lang, [], _ReducedValues) ->      {ok, []};  rereduce(Lang, RedSrcs, ReducedValues) -> -    Grouped = group_reductions_results(ReducedValues),     +    Grouped = group_reductions_results(ReducedValues),      Results = lists:zipwith(          fun          (<<"_", _/binary>> = FunSrc, Values) -> @@ -186,7 +186,7 @@ builtin_stats(rereduce, [[_,First]|Rest]) ->  % use the function stored in ddoc.validate_doc_update to test an update.  validate_doc_update(DDoc, EditDoc, DiskDoc, Ctx, SecObj) ->      JsonEditDoc = couch_doc:to_json_obj(EditDoc, [revs]), -    JsonDiskDoc = json_doc(DiskDoc),   +    JsonDiskDoc = json_doc(DiskDoc),      case ddoc_prompt(DDoc, [<<"validate_doc_update">>], [JsonEditDoc, JsonDiskDoc, Ctx, SecObj]) of          1 ->              ok; @@ -211,7 +211,7 @@ filter_docs(Req, Db, DDoc, FName, Docs) ->      [true, Passes] = ddoc_prompt(DDoc, [<<"filters">>, FName], [JsonDocs, JsonReq]),      {ok, Passes}. -ddoc_proc_prompt({Proc, DDocId}, FunPath, Args) ->  +ddoc_proc_prompt({Proc, DDocId}, FunPath, Args) ->      proc_prompt(Proc, [<<"ddoc">>, DDocId, FunPath, Args]).  ddoc_prompt(DDoc, FunPath, Args) -> @@ -271,7 +271,7 @@ terminate(_Reason, {_Langs, PidProcs, _LangProcs}) ->  handle_call({get_proc, #doc{body={Props}}=DDoc, DDocKey}, _From, {Langs, PidProcs, LangProcs}=Server) ->      % Note to future self. Add max process limit. -    Lang = couch_util:get_value(<<"language">>, Props, <<"javascript">>),     +    Lang = couch_util:get_value(<<"language">>, Props, <<"javascript">>),      case ets:lookup(LangProcs, Lang) of      [{Lang, [P|Rest]}] ->          % find a proc in the set that has the DDoc @@ -279,7 +279,7 @@ handle_call({get_proc, #doc{body={Props}}=DDoc, DDocKey}, _From, {Langs, PidProc          {ok, Proc} ->              rem_from_list(LangProcs, Lang, Proc),              {reply, {ok, Proc, get_query_server_config()}, Server}; -        Error ->  +        Error ->              {reply, Error, Server}          end;      _ -> @@ -289,7 +289,7 @@ handle_call({get_proc, #doc{body={Props}}=DDoc, DDocKey}, _From, {Langs, PidProc              case proc_with_ddoc(DDoc, DDocKey, [Proc]) of              {ok, Proc2} ->                  {reply, {ok, Proc2, get_query_server_config()}, Server}; -            Error ->  +            Error ->                  {reply, Error, Server}              end;          Error -> @@ -311,7 +311,7 @@ handle_call({get_proc, Lang}, _From, {Langs, PidProcs, LangProcs}=Server) ->              {reply, Error, Server}          end      end; -handle_call({unlink_proc, Pid}, _From, {_, PidProcs, _}=Server) ->     +handle_call({unlink_proc, Pid}, _From, {_, PidProcs, _}=Server) ->      rem_value(PidProcs, Pid),      unlink(Pid),      {reply, ok, Server}; @@ -370,7 +370,7 @@ new_process(Langs, Lang) ->      end.  proc_with_ddoc(DDoc, DDocKey, LangProcs) -> -    DDocProcs = lists:filter(fun(#proc{ddoc_keys=Keys}) ->  +    DDocProcs = lists:filter(fun(#proc{ddoc_keys=Keys}) ->              lists:any(fun(Key) ->                  Key == DDocKey              end, Keys) diff --git a/src/couchdb/couch_rep.erl b/src/couchdb/couch_rep.erl index 6be37f0b..96bed646 100644 --- a/src/couchdb/couch_rep.erl +++ b/src/couchdb/couch_rep.erl @@ -12,7 +12,7 @@  -module(couch_rep).  -behaviour(gen_server). --export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,  +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,      code_change/3]).  -export([replicate/2, checkpoint/1]). @@ -442,7 +442,7 @@ has_session_id(SessionId, [{Props} | Rest]) ->  maybe_append_options(Options, Props) ->      lists:foldl(fun(Option, Acc) -> -        Acc ++  +        Acc ++          case couch_util:get_value(Option, Props, false) of          true ->              "+" ++ ?b2l(Option); @@ -456,7 +456,7 @@ make_replication_id({Props}, UserCtx) ->      {ok, HostName} = inet:gethostname(),      % Port = mochiweb_socket_server:get(couch_httpd, port),      Src = get_rep_endpoint(UserCtx, couch_util:get_value(<<"source">>, Props)), -    Tgt = get_rep_endpoint(UserCtx, couch_util:get_value(<<"target">>, Props)),     +    Tgt = get_rep_endpoint(UserCtx, couch_util:get_value(<<"target">>, Props)),      Base = [HostName, Src, Tgt] ++          case couch_util:get_value(<<"filter">>, Props) of          undefined -> @@ -536,7 +536,7 @@ open_db(<<"https://",_/binary>>=Url, _, CreateTarget) ->      open_db({[{<<"url">>,Url}]}, [], CreateTarget);  open_db(<<DbName/binary>>, UserCtx, CreateTarget) ->      case CreateTarget of -    true ->  +    true ->          ok = couch_httpd:verify_is_server_admin(UserCtx),          couch_server:create(DbName, [{user_ctx, UserCtx}]);      false -> ok @@ -591,7 +591,7 @@ do_checkpoint(State) ->              {<<"missing_found">>, ets:lookup_element(Stats, missing_revs, 2)},              {<<"docs_read">>, ets:lookup_element(Stats, docs_read, 2)},              {<<"docs_written">>, ets:lookup_element(Stats, docs_written, 2)}, -            {<<"doc_write_failures">>,  +            {<<"doc_write_failures">>,                  ets:lookup_element(Stats, doc_write_failures, 2)}          ]},          % limit history to 50 entries diff --git a/src/couchdb/couch_rep_att.erl b/src/couchdb/couch_rep_att.erl index 3527df00..be10acc8 100644 --- a/src/couchdb/couch_rep_att.erl +++ b/src/couchdb/couch_rep_att.erl @@ -29,11 +29,11 @@ convert_stub(#att{data=stub, name=Name} = Attachment,      Attachment#att{data=RcvFun}.  cleanup() -> -    receive  +    receive      {ibrowse_async_response, _, _} ->          %% TODO maybe log, didn't expect to have data here          cleanup(); -    {ibrowse_async_response_end, _} ->  +    {ibrowse_async_response_end, _} ->          cleanup();      {ibrowse_async_headers, _, _, _} ->          cleanup() diff --git a/src/couchdb/couch_rep_changes_feed.erl b/src/couchdb/couch_rep_changes_feed.erl index c492fcd9..b1d3960d 100644 --- a/src/couchdb/couch_rep_changes_feed.erl +++ b/src/couchdb/couch_rep_changes_feed.erl @@ -12,7 +12,7 @@  -module(couch_rep_changes_feed).  -behaviour(gen_server). --export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,  +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,      code_change/3]).  -export([start_link/4, next/1, stop/1]). @@ -383,6 +383,6 @@ start_http_request(RawUrl) ->          {inactivity_timeout, 31000},          {response_format, binary}      ], -    {ibrowse_req_id, Id} =  +    {ibrowse_req_id, Id} =          ibrowse:send_req_direct(Pid, RawUrl, [], get, [], Opts, infinity),      {Pid, Id}. diff --git a/src/couchdb/couch_rep_httpc.erl b/src/couchdb/couch_rep_httpc.erl index 7c6409c5..53e318d3 100644 --- a/src/couchdb/couch_rep_httpc.erl +++ b/src/couchdb/couch_rep_httpc.erl @@ -46,7 +46,7 @@ do_request(Req) ->      nil ->          [];      _Else -> -        iolist_to_binary(?JSON_ENCODE(B))  +        iolist_to_binary(?JSON_ENCODE(B))      end,      Resp = case Conn of      nil -> @@ -115,7 +115,7 @@ full_url(Req) ->          resource = Resource,          qs = QS      } = Req, -    QStr = lists:map(fun({K,V}) -> io_lib:format("~s=~s",  +    QStr = lists:map(fun({K,V}) -> io_lib:format("~s=~s",          [couch_util:to_list(K), couch_util:to_list(V)]) end, QS),      lists:flatten([Url, Resource, "?", string:join(QStr, "&")]). diff --git a/src/couchdb/couch_rep_missing_revs.erl b/src/couchdb/couch_rep_missing_revs.erl index 58b37d08..1eff6774 100644 --- a/src/couchdb/couch_rep_missing_revs.erl +++ b/src/couchdb/couch_rep_missing_revs.erl @@ -12,7 +12,7 @@  -module(couch_rep_missing_revs).  -behaviour(gen_server). --export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,  +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,      code_change/3]).  -export([start_link/4, next/1, stop/1]). diff --git a/src/couchdb/couch_rep_reader.erl b/src/couchdb/couch_rep_reader.erl index b01c2ada..8b75258a 100644 --- a/src/couchdb/couch_rep_reader.erl +++ b/src/couchdb/couch_rep_reader.erl @@ -12,7 +12,7 @@  -module(couch_rep_reader).  -behaviour(gen_server). --export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,  +-export([init/1, handle_call/3, handle_cast/2, handle_info/2, terminate/2,      code_change/3]).  -export([start_link/4, next/1]). diff --git a/src/couchdb/couch_server.erl b/src/couchdb/couch_server.erl index 081babd1..f2403ef7 100644 --- a/src/couchdb/couch_server.erl +++ b/src/couchdb/couch_server.erl @@ -108,7 +108,7 @@ hash_admin_passwords(Persist) ->          ({User, ClearPassword}) ->              Salt = ?b2l(couch_uuids:random()),              Hashed = couch_util:to_hex(crypto:sha(ClearPassword ++ Salt)), -            couch_config:set("admins",  +            couch_config:set("admins",                  User, "-hashed-" ++ Hashed ++ "," ++ Salt, Persist)          end, couch_config:get("admins")). @@ -148,7 +148,7 @@ init([]) ->                  start_time=httpd_util:rfc1123_date()}}.  terminate(_Reason, _Srv) -> -    [couch_util:shutdown_sync(Pid) || {_, {Pid, _LruTime}} <-  +    [couch_util:shutdown_sync(Pid) || {_, {Pid, _LruTime}} <-              ets:tab2list(couch_dbs_by_name)],      ok. @@ -232,7 +232,7 @@ handle_call({open_result, DbName, {ok, OpenedDbPid}}, _From, Server) ->      link(OpenedDbPid),      [{DbName, {opening,Opener,Froms}}] = ets:lookup(couch_dbs_by_name, DbName),      lists:foreach(fun({FromPid,_}=From) -> -        gen_server:reply(From,  +        gen_server:reply(From,                  catch couch_db:open_ref_counted(OpenedDbPid, FromPid))      end, Froms),      LruTime = now(), @@ -242,11 +242,11 @@ handle_call({open_result, DbName, {ok, OpenedDbPid}}, _From, Server) ->      true = ets:insert(couch_dbs_by_pid, {OpenedDbPid, DbName}),      true = ets:insert(couch_dbs_by_lru, {LruTime, DbName}),      {reply, ok, Server}; -handle_call({open_result, DbName, Error}, _From, Server) ->     +handle_call({open_result, DbName, Error}, _From, Server) ->      [{DbName, {opening,Opener,Froms}}] = ets:lookup(couch_dbs_by_name, DbName),      lists:foreach(fun(From) ->          gen_server:reply(From, Error) -    end, Froms),     +    end, Froms),      true = ets:delete(couch_dbs_by_name, DbName),      true = ets:delete(couch_dbs_by_pid, Opener),      {reply, ok, Server#server{dbs_open=Server#server.dbs_open - 1}}; diff --git a/src/couchdb/couch_view_group.erl b/src/couchdb/couch_view_group.erl index 275d18d5..9c116759 100644 --- a/src/couchdb/couch_view_group.erl +++ b/src/couchdb/couch_view_group.erl @@ -178,7 +178,7 @@ handle_cast({compact_done, #group{current_seq=NewSeq} = NewGroup},          when NewSeq >= OldSeq ->      #group_state{          group = #group{name=GroupId, fd=OldFd, sig=GroupSig} = Group, -        init_args = {RootDir, DbName, _},  +        init_args = {RootDir, DbName, _},          updater_pid = UpdaterPid,          ref_counter = RefCounter      } = State, diff --git a/src/couchdb/couch_work_queue.erl b/src/couchdb/couch_work_queue.erl index 3581aaf7..decfcad8 100644 --- a/src/couchdb/couch_work_queue.erl +++ b/src/couchdb/couch_work_queue.erl @@ -83,7 +83,7 @@ handle_call({dequeue, Max}, _From, #q{queue=Queue, max_size=MaxSize,              {reply, {ok, queue:to_list(Queue)}, Q2}          end;      true -> -        {DequeuedItems, Queue2, Blocked2} =  +        {DequeuedItems, Queue2, Blocked2} =                  dequeue_items(Max, Queue, Q#q.blocked, []),          {reply, {ok, DequeuedItems},                  Q#q{items=Items-Max,blocked=Blocked2,queue=Queue2}}  | 
