summaryrefslogtreecommitdiff
path: root/apps/chttpd/src
diff options
context:
space:
mode:
Diffstat (limited to 'apps/chttpd/src')
-rw-r--r--apps/chttpd/src/chttpd.app.src7
-rw-r--r--apps/chttpd/src/chttpd.erl585
-rw-r--r--apps/chttpd/src/chttpd_app.erl21
-rw-r--r--apps/chttpd/src/chttpd_auth.erl473
-rw-r--r--apps/chttpd/src/chttpd_db.erl1082
-rw-r--r--apps/chttpd/src/chttpd_external.erl173
-rw-r--r--apps/chttpd/src/chttpd_misc.erl222
-rw-r--r--apps/chttpd/src/chttpd_oauth.erl168
-rw-r--r--apps/chttpd/src/chttpd_rewrite.erl418
-rw-r--r--apps/chttpd/src/chttpd_show.erl311
-rw-r--r--apps/chttpd/src/chttpd_sup.erl25
-rw-r--r--apps/chttpd/src/chttpd_view.erl306
12 files changed, 0 insertions, 3791 deletions
diff --git a/apps/chttpd/src/chttpd.app.src b/apps/chttpd/src/chttpd.app.src
deleted file mode 100644
index 95ef107a..00000000
--- a/apps/chttpd/src/chttpd.app.src
+++ /dev/null
@@ -1,7 +0,0 @@
-{application, chttpd, [
- {description, "HTTP interface for CouchDB cluster"},
- {vsn, "1.0.3"},
- {registered, [chttpd_sup, chttpd]},
- {applications, [kernel, stdlib, couch, fabric]},
- {mod, {chttpd_app,[]}}
-]}. \ No newline at end of file
diff --git a/apps/chttpd/src/chttpd.erl b/apps/chttpd/src/chttpd.erl
deleted file mode 100644
index 4d3425b2..00000000
--- a/apps/chttpd/src/chttpd.erl
+++ /dev/null
@@ -1,585 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd).
--include_lib("couch/include/couch_db.hrl").
-
--export([start_link/0, stop/0, handle_request/1, config_change/2,
- primary_header_value/2, header_value/2, header_value/3, qs_value/2,
- qs_value/3, qs/1, path/1, absolute_uri/2, body_length/1,
- verify_is_server_admin/1, unquote/1, quote/1, recv/2,recv_chunked/4,
- error_info/1, parse_form/1, json_body/1, json_body_obj/1, body/1,
- doc_etag/1, make_etag/1, etag_respond/3, partition/1, serve_file/3,
- server_header/0, start_chunked_response/3,send_chunk/2,
- start_response_length/4, send/2, start_json_response/2,
- start_json_response/3, end_json_response/1, send_response/4,
- send_method_not_allowed/2, send_error/2, send_error/4, send_redirect/2,
- send_chunked_error/2, send_json/2,send_json/3,send_json/4]).
-
-start_link() ->
- Options = [
- {loop, fun ?MODULE:handle_request/1},
- {name, ?MODULE},
- {ip, couch_config:get("chttpd", "bind_address", any)},
- {port, couch_config:get("chttpd", "port", "5984")},
- {backlog, list_to_integer(couch_config:get("chttpd", "backlog", "128"))}
- ],
- case mochiweb_http:start(Options) of
- {ok, Pid} ->
- ok = couch_config:register(fun ?MODULE:config_change/2, Pid),
- {ok, Pid};
- {error, Reason} ->
- io:format("Failure to start Mochiweb: ~s~n", [Reason]),
- {error, Reason}
- end.
-
-config_change("chttpd", "bind_address") ->
- ?MODULE:stop();
-config_change("chttpd", "port") ->
- ?MODULE:stop();
-config_change("chttpd", "backlog") ->
- ?MODULE:stop().
-
-stop() ->
- mochiweb_http:stop(?MODULE).
-
-handle_request(MochiReq) ->
- Begin = now(),
-
- AuthenticationFuns = [
- fun chttpd_auth:cookie_authentication_handler/1,
- fun chttpd_auth:default_authentication_handler/1
- ],
-
- % for the path, use the raw path with the query string and fragment
- % removed, but URL quoting left intact
- RawUri = MochiReq:get(raw_path),
- {"/" ++ Path, _, _} = mochiweb_util:urlsplit_path(RawUri),
- {HandlerKey, _, _} = mochiweb_util:partition(Path, "/"),
-
- LogForClosedSocket = io_lib:format("mochiweb_recv_error for ~s - ~p ~s", [
- MochiReq:get(peer),
- MochiReq:get(method),
- RawUri
- ]),
-
- Method1 =
- case MochiReq:get(method) of
- % already an atom
- Meth when is_atom(Meth) -> Meth;
-
- % Non standard HTTP verbs aren't atoms (COPY, MOVE etc) so convert when
- % possible (if any module references the atom, then it's existing).
- Meth -> couch_util:to_existing_atom(Meth)
- end,
- increment_method_stats(Method1),
- % alias HEAD to GET as mochiweb takes care of stripping the body
- Method = case Method1 of
- 'HEAD' -> 'GET';
- Other -> Other
- end,
-
- HttpReq = #httpd{
- mochi_req = MochiReq,
- method = Method,
- path_parts = [list_to_binary(chttpd:unquote(Part))
- || Part <- string:tokens(Path, "/")],
- db_url_handlers = db_url_handlers(),
- design_url_handlers = design_url_handlers()
- },
-
- {ok, Resp} =
- try
- case authenticate_request(HttpReq, AuthenticationFuns) of
- #httpd{} = Req ->
- HandlerFun = url_handler(HandlerKey),
- HandlerFun(Req);
- Response ->
- Response
- end
- catch
- throw:{http_head_abort, Resp0} ->
- {ok, Resp0};
- throw:{invalid_json, S} ->
- ?LOG_ERROR("attempted upload of invalid JSON ~s", [S]),
- send_error(HttpReq, {bad_request, "invalid UTF-8 JSON"});
- exit:{mochiweb_recv_error, E} ->
- ?LOG_INFO(LogForClosedSocket ++ " - ~p", [E]),
- exit(normal);
- throw:Error ->
- send_error(HttpReq, Error);
- error:database_does_not_exist ->
- send_error(HttpReq, database_does_not_exist);
- error:badarg ->
- ?LOG_ERROR("Badarg error in HTTP request",[]),
- ?LOG_INFO("Stacktrace: ~p",[erlang:get_stacktrace()]),
- send_error(HttpReq, badarg);
- error:function_clause ->
- ?LOG_ERROR("function_clause error in HTTP request",[]),
- ?LOG_INFO("Stacktrace: ~p",[erlang:get_stacktrace()]),
- send_error(HttpReq, function_clause);
- Tag:Error ->
- ?LOG_ERROR("Uncaught error in HTTP request: ~p",[{Tag, Error}]),
- ?LOG_INFO("Stacktrace: ~p",[erlang:get_stacktrace()]),
- send_error(HttpReq, Error)
- end,
-
- RequestTime = timer:now_diff(now(), Begin)/1000,
- Peer = MochiReq:get(peer),
- Code = Resp:get(code),
- Host = MochiReq:get_header_value("Host"),
- ?LOG_INFO("~s ~s ~s ~s ~B ~B", [Peer, Host,
- atom_to_list(Method1), RawUri, Code, round(RequestTime)]),
- couch_stats_collector:record({couchdb, request_time}, RequestTime),
- couch_stats_collector:increment({httpd, requests}),
- {ok, Resp}.
-
-% Try authentication handlers in order until one returns a result
-authenticate_request(#httpd{user_ctx=#user_ctx{}} = Req, _AuthFuns) ->
- Req;
-authenticate_request(#httpd{} = Req, [AuthFun|Rest]) ->
- authenticate_request(AuthFun(Req), Rest);
-authenticate_request(#httpd{} = Req, []) ->
- case couch_config:get("chttpd", "require_valid_user", "false") of
- "true" ->
- throw({unauthorized, <<"Authentication required.">>});
- "false" ->
- case couch_config:get("admins") of
- [] ->
- Ctx = #user_ctx{roles=[<<"_reader">>, <<"_writer">>, <<"_admin">>]},
- Req#httpd{user_ctx = Ctx};
- _ ->
- Req#httpd{user_ctx=#user_ctx{}}
- end
- end;
-authenticate_request(Response, _AuthFuns) ->
- Response.
-
-increment_method_stats(Method) ->
- couch_stats_collector:increment({httpd_request_methods, Method}).
-
-url_handler("") -> fun chttpd_misc:handle_welcome_req/1;
-url_handler("favicon.ico") -> fun chttpd_misc:handle_favicon_req/1;
-url_handler("_utils") -> fun chttpd_misc:handle_utils_dir_req/1;
-url_handler("_all_dbs") -> fun chttpd_misc:handle_all_dbs_req/1;
-url_handler("_active_tasks") -> fun chttpd_misc:handle_task_status_req/1;
-url_handler("_config") -> fun chttpd_misc:handle_config_req/1;
-url_handler("_replicate") -> fun chttpd_misc:handle_replicate_req/1;
-url_handler("_uuids") -> fun chttpd_misc:handle_uuids_req/1;
-url_handler("_log") -> fun chttpd_misc:handle_log_req/1;
-url_handler("_sleep") -> fun chttpd_misc:handle_sleep_req/1;
-url_handler("_session") -> fun chttpd_auth:handle_session_req/1;
-url_handler("_user") -> fun chttpd_auth:handle_user_req/1;
-url_handler("_oauth") -> fun chttpd_oauth:handle_oauth_req/1;
-url_handler("_restart") -> fun showroom_http:handle_restart_req/1;
-url_handler("_membership") -> fun mem3_httpd:handle_membership_req/1;
-url_handler(_) -> fun chttpd_db:handle_request/1.
-
-db_url_handlers() ->
- [
- {<<"_view_cleanup">>, fun chttpd_db:handle_view_cleanup_req/2},
- {<<"_compact">>, fun chttpd_db:handle_compact_req/2},
- {<<"_design">>, fun chttpd_db:handle_design_req/2},
- {<<"_temp_view">>, fun chttpd_view:handle_temp_view_req/2},
- {<<"_changes">>, fun chttpd_db:handle_changes_req/2},
- {<<"_search">>, fun chttpd_external:handle_search_req/2}
- ].
-
-design_url_handlers() ->
- [
- {<<"_view">>, fun chttpd_view:handle_view_req/3},
- {<<"_show">>, fun chttpd_show:handle_doc_show_req/3},
- {<<"_list">>, fun chttpd_show:handle_view_list_req/3},
- {<<"_update">>, fun chttpd_show:handle_doc_update_req/3},
- {<<"_info">>, fun chttpd_db:handle_design_info_req/3},
- {<<"_rewrite">>, fun chttpd_rewrite:handle_rewrite_req/3}
- ].
-
-% Utilities
-
-partition(Path) ->
- mochiweb_util:partition(Path, "/").
-
-header_value(#httpd{mochi_req=MochiReq}, Key) ->
- MochiReq:get_header_value(Key).
-
-header_value(#httpd{mochi_req=MochiReq}, Key, Default) ->
- case MochiReq:get_header_value(Key) of
- undefined -> Default;
- Value -> Value
- end.
-
-primary_header_value(#httpd{mochi_req=MochiReq}, Key) ->
- MochiReq:get_primary_header_value(Key).
-
-serve_file(#httpd{mochi_req=MochiReq}=Req, RelativePath, DocumentRoot) ->
- {ok, MochiReq:serve_file(RelativePath, DocumentRoot,
- server_header() ++ chttpd_auth:cookie_auth_header(Req, []))}.
-
-qs_value(Req, Key) ->
- qs_value(Req, Key, undefined).
-
-qs_value(Req, Key, Default) ->
- couch_util:get_value(Key, qs(Req), Default).
-
-qs(#httpd{mochi_req=MochiReq}) ->
- MochiReq:parse_qs().
-
-path(#httpd{mochi_req=MochiReq}) ->
- MochiReq:get(path).
-
-absolute_uri(#httpd{mochi_req=MochiReq}, Path) ->
- XHost = couch_config:get("httpd", "x_forwarded_host", "X-Forwarded-Host"),
- Host = case MochiReq:get_header_value(XHost) of
- undefined ->
- case MochiReq:get_header_value("Host") of
- undefined ->
- {ok, {Address, Port}} = inet:sockname(MochiReq:get(socket)),
- inet_parse:ntoa(Address) ++ ":" ++ integer_to_list(Port);
- Value1 ->
- Value1
- end;
- Value -> Value
- end,
- XSsl = couch_config:get("httpd", "x_forwarded_ssl", "X-Forwarded-Ssl"),
- Scheme = case MochiReq:get_header_value(XSsl) of
- "on" -> "https";
- _ ->
- XProto = couch_config:get("httpd", "x_forwarded_proto",
- "X-Forwarded-Proto"),
- case MochiReq:get_header_value(XProto) of
- % Restrict to "https" and "http" schemes only
- "https" -> "https";
- _ -> "http"
- end
- end,
- Scheme ++ "://" ++ Host ++ Path.
-
-unquote(UrlEncodedString) ->
- mochiweb_util:unquote(UrlEncodedString).
-
-quote(UrlDecodedString) ->
- mochiweb_util:quote_plus(UrlDecodedString).
-
-parse_form(#httpd{mochi_req=MochiReq}) ->
- mochiweb_multipart:parse_form(MochiReq).
-
-recv(#httpd{mochi_req=MochiReq}, Len) ->
- MochiReq:recv(Len).
-
-recv_chunked(#httpd{mochi_req=MochiReq}, MaxChunkSize, ChunkFun, InitState) ->
- % Fun is called once with each chunk
- % Fun({Length, Binary}, State)
- % called with Length == 0 on the last time.
- MochiReq:stream_body(MaxChunkSize, ChunkFun, InitState).
-
-body_length(Req) ->
- case header_value(Req, "Transfer-Encoding") of
- undefined ->
- case header_value(Req, "Content-Length") of
- undefined -> undefined;
- Length -> list_to_integer(Length)
- end;
- "chunked" -> chunked;
- Unknown -> {unknown_transfer_encoding, Unknown}
- end.
-
-body(#httpd{mochi_req=MochiReq, req_body=ReqBody}) ->
- case ReqBody of
- undefined ->
- % Maximum size of document PUT request body (4GB)
- MaxSize = list_to_integer(
- couch_config:get("couchdb", "max_document_size", "4294967296")),
- MochiReq:recv_body(MaxSize);
- _Else ->
- ReqBody
- end.
-
-json_body(Httpd) ->
- ?JSON_DECODE(body(Httpd)).
-
-json_body_obj(Httpd) ->
- case json_body(Httpd) of
- {Props} -> {Props};
- _Else ->
- throw({bad_request, "Request body must be a JSON object"})
- end.
-
-
-doc_etag(#doc{revs={Start, [DiskRev|_]}}) ->
- "\"" ++ ?b2l(couch_doc:rev_to_str({Start, DiskRev})) ++ "\"".
-
-make_etag(Term) ->
- <<SigInt:128/integer>> = erlang:md5(term_to_binary(Term)),
- list_to_binary(io_lib:format("\"~.36B\"",[SigInt])).
-
-etag_match(Req, CurrentEtag) when is_binary(CurrentEtag) ->
- etag_match(Req, binary_to_list(CurrentEtag));
-
-etag_match(Req, CurrentEtag) ->
- EtagsToMatch = string:tokens(
- chttpd:header_value(Req, "If-None-Match", ""), ", "),
- lists:member(CurrentEtag, EtagsToMatch).
-
-etag_respond(Req, CurrentEtag, RespFun) ->
- case etag_match(Req, CurrentEtag) of
- true ->
- % the client has this in their cache.
- chttpd:send_response(Req, 304, [{"Etag", CurrentEtag}], <<>>);
- false ->
- % Run the function.
- RespFun()
- end.
-
-verify_is_server_admin(#httpd{user_ctx=#user_ctx{roles=Roles}}) ->
- case lists:member(<<"_admin">>, Roles) of
- true -> ok;
- false -> throw({unauthorized, <<"You are not a server admin.">>})
- end.
-
-start_response_length(#httpd{mochi_req=MochiReq}=Req, Code, Headers, Length) ->
- couch_stats_collector:increment({httpd_status_codes, Code}),
- Resp = MochiReq:start_response_length({Code, Headers ++ server_header() ++
- chttpd_auth:cookie_auth_header(Req, Headers), Length}),
- case MochiReq:get(method) of
- 'HEAD' -> throw({http_head_abort, Resp});
- _ -> ok
- end,
- {ok, Resp}.
-
-send(Resp, Data) ->
- Resp:send(Data),
- {ok, Resp}.
-
-start_chunked_response(#httpd{mochi_req=MochiReq}=Req, Code, Headers) ->
- couch_stats_collector:increment({httpd_status_codes, Code}),
- Resp = MochiReq:respond({Code, Headers ++ server_header() ++
- chttpd_auth:cookie_auth_header(Req, Headers), chunked}),
- case MochiReq:get(method) of
- 'HEAD' -> throw({http_head_abort, Resp});
- _ -> ok
- end,
- {ok, Resp}.
-
-send_chunk(Resp, Data) ->
- Resp:write_chunk(Data),
- {ok, Resp}.
-
-send_response(#httpd{mochi_req=MochiReq}=Req, Code, Headers, Body) ->
- couch_stats_collector:increment({httpd_status_codes, Code}),
- if Code >= 400 ->
- ?LOG_DEBUG("httpd ~p error response:~n ~s", [Code, Body]);
- true -> ok
- end,
- {ok, MochiReq:respond({Code, Headers ++ server_header() ++
- chttpd_auth:cookie_auth_header(Req, Headers), Body})}.
-
-send_method_not_allowed(Req, Methods) ->
- send_error(Req, 405, [{"Allow", Methods}], <<"method_not_allowed">>,
- ?l2b("Only " ++ Methods ++ " allowed")).
-
-send_json(Req, Value) ->
- send_json(Req, 200, Value).
-
-send_json(Req, Code, Value) ->
- send_json(Req, Code, [], Value).
-
-send_json(Req, Code, Headers, Value) ->
- DefaultHeaders = [
- {"Content-Type", negotiate_content_type(Req)},
- {"Cache-Control", "must-revalidate"}
- ],
- Body = list_to_binary(
- [start_jsonp(Req), ?JSON_ENCODE(Value), end_jsonp(), $\n]
- ),
- send_response(Req, Code, DefaultHeaders ++ Headers, Body).
-
-start_json_response(Req, Code) ->
- start_json_response(Req, Code, []).
-
-start_json_response(Req, Code, Headers) ->
- DefaultHeaders = [
- {"Content-Type", negotiate_content_type(Req)},
- {"Cache-Control", "must-revalidate"}
- ],
- start_jsonp(Req), % Validate before starting chunked.
- %start_chunked_response(Req, Code, DefaultHeaders ++ Headers).
- {ok, Resp} = start_chunked_response(Req, Code, DefaultHeaders ++ Headers),
- case start_jsonp(Req) of
- [] -> ok;
- Start -> send_chunk(Resp, Start)
- end,
- {ok, Resp}.
-
-end_json_response(Resp) ->
- send_chunk(Resp, end_jsonp() ++ [$\r,$\n]),
- %send_chunk(Resp, [$\n]),
- send_chunk(Resp, []).
-
-start_jsonp(Req) ->
- case get(jsonp) of
- undefined -> put(jsonp, qs_value(Req, "callback", no_jsonp));
- _ -> ok
- end,
- case get(jsonp) of
- no_jsonp -> [];
- [] -> [];
- CallBack ->
- try
- validate_callback(CallBack),
- CallBack ++ "("
- catch
- Error ->
- put(jsonp, no_jsonp),
- throw(Error)
- end
- end.
-
-end_jsonp() ->
- Resp = case get(jsonp) of
- no_jsonp -> [];
- [] -> [];
- _ -> ");"
- end,
- put(jsonp, undefined),
- Resp.
-
-validate_callback(CallBack) when is_binary(CallBack) ->
- validate_callback(binary_to_list(CallBack));
-validate_callback([]) ->
- ok;
-validate_callback([Char | Rest]) ->
- case Char of
- _ when Char >= $a andalso Char =< $z -> ok;
- _ when Char >= $A andalso Char =< $Z -> ok;
- _ when Char >= $0 andalso Char =< $9 -> ok;
- _ when Char == $. -> ok;
- _ when Char == $_ -> ok;
- _ when Char == $[ -> ok;
- _ when Char == $] -> ok;
- _ ->
- throw({bad_request, invalid_callback})
- end,
- validate_callback(Rest).
-
-
-error_info({Error, Reason}) when is_list(Reason) ->
- error_info({Error, couch_util:to_binary(Reason)});
-error_info(bad_request) ->
- {400, <<"bad_request">>, <<>>};
-error_info({bad_request, Reason}) ->
- {400, <<"bad_request">>, Reason};
-error_info({query_parse_error, Reason}) ->
- {400, <<"query_parse_error">>, Reason};
-error_info(database_does_not_exist) ->
- {404, <<"not_found">>, <<"Database does not exist.">>};
-error_info(not_found) ->
- {404, <<"not_found">>, <<"missing">>};
-error_info({not_found, Reason}) ->
- {404, <<"not_found">>, Reason};
-error_info({not_acceptable, Reason}) ->
- {406, <<"not_acceptable">>, Reason};
-error_info(conflict) ->
- {409, <<"conflict">>, <<"Document update conflict.">>};
-error_info({conflict, _}) ->
- {409, <<"conflict">>, <<"Document update conflict.">>};
-error_info({forbidden, Msg}) ->
- {403, <<"forbidden">>, Msg};
-error_info({forbidden, Error, Msg}) ->
- {403, Error, Msg};
-error_info({unauthorized, Msg}) ->
- {401, <<"unauthorized">>, Msg};
-error_info(file_exists) ->
- {412, <<"file_exists">>, <<"The database could not be "
- "created, the file already exists.">>};
-error_info({r_quorum_not_met, Reason}) ->
- {412, <<"read_quorum_not_met">>, Reason};
-error_info({w_quorum_not_met, Reason}) ->
- {500, <<"write_quorum_not_met">>, Reason};
-error_info({bad_ctype, Reason}) ->
- {415, <<"bad_content_type">>, Reason};
-error_info({error, illegal_database_name}) ->
- {400, <<"illegal_database_name">>, <<"Only lowercase characters (a-z), "
- "digits (0-9), and any of the characters _, $, (, ), +, -, and / "
- "are allowed">>};
-error_info({missing_stub, Reason}) ->
- {412, <<"missing_stub">>, Reason};
-error_info(not_implemented) ->
- {501, <<"not_implemented">>, <<"this feature is not yet implemented">>};
-error_info({Error, Reason}) ->
- {500, couch_util:to_binary(Error), couch_util:to_binary(Reason)};
-error_info(Error) ->
- {500, <<"unknown_error">>, couch_util:to_binary(Error)}.
-
-send_error(_Req, {already_sent, Resp, _Error}) ->
- {ok, Resp};
-
-send_error(#httpd{mochi_req=MochiReq}=Req, Error) ->
- {Code, ErrorStr, ReasonStr} = error_info(Error),
- Headers = if Code == 401 ->
- case MochiReq:get_header_value("X-CouchDB-WWW-Authenticate") of
- undefined ->
- case couch_config:get("httpd", "WWW-Authenticate", nil) of
- nil ->
- [];
- Type ->
- [{"WWW-Authenticate", Type}]
- end;
- Type ->
- [{"WWW-Authenticate", Type}]
- end;
- true ->
- []
- end,
- send_error(Req, Code, Headers, ErrorStr, ReasonStr).
-
-send_error(Req, Code, ErrorStr, ReasonStr) ->
- send_error(Req, Code, [], ErrorStr, ReasonStr).
-
-send_error(Req, Code, Headers, ErrorStr, ReasonStr) ->
- send_json(Req, Code, Headers,
- {[{<<"error">>, ErrorStr},
- {<<"reason">>, ReasonStr}]}).
-
-% give the option for list functions to output html or other raw errors
-send_chunked_error(Resp, {_Error, {[{<<"body">>, Reason}]}}) ->
- send_chunk(Resp, Reason),
- send_chunk(Resp, []);
-
-send_chunked_error(Resp, Error) ->
- {Code, ErrorStr, ReasonStr} = error_info(Error),
- JsonError = {[{<<"code">>, Code},
- {<<"error">>, ErrorStr},
- {<<"reason">>, ReasonStr}]},
- send_chunk(Resp, ?l2b([$\n,?JSON_ENCODE(JsonError),$\n])),
- send_chunk(Resp, []).
-
-send_redirect(Req, Path) ->
- Headers = [{"Location", chttpd:absolute_uri(Req, Path)}],
- send_response(Req, 301, Headers, <<>>).
-
-negotiate_content_type(#httpd{mochi_req=MochiReq}) ->
- %% Determine the appropriate Content-Type header for a JSON response
- %% depending on the Accept header in the request. A request that explicitly
- %% lists the correct JSON MIME type will get that type, otherwise the
- %% response will have the generic MIME type "text/plain"
- AcceptedTypes = case MochiReq:get_header_value("Accept") of
- undefined -> [];
- AcceptHeader -> string:tokens(AcceptHeader, ", ")
- end,
- case lists:member("application/json", AcceptedTypes) of
- true -> "application/json";
- false -> "text/plain;charset=utf-8"
- end.
-
-server_header() ->
- couch_httpd:server_header().
diff --git a/apps/chttpd/src/chttpd_app.erl b/apps/chttpd/src/chttpd_app.erl
deleted file mode 100644
index d7a5aef8..00000000
--- a/apps/chttpd/src/chttpd_app.erl
+++ /dev/null
@@ -1,21 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_app).
--behaviour(application).
--export([start/2, stop/1]).
-
-start(_Type, StartArgs) ->
- chttpd_sup:start_link(StartArgs).
-
-stop(_State) ->
- ok.
diff --git a/apps/chttpd/src/chttpd_auth.erl b/apps/chttpd/src/chttpd_auth.erl
deleted file mode 100644
index 24fe8c05..00000000
--- a/apps/chttpd/src/chttpd_auth.erl
+++ /dev/null
@@ -1,473 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_auth).
--include_lib("couch/include/couch_db.hrl").
-
--export([special_test_authentication_handler/1, null_authentication_handler/1,
- cookie_authentication_handler/1, default_authentication_handler/1,
- handle_session_req/1, handle_user_req/1, cookie_auth_header/2]).
-
-% used by OAuth handler
--export([get_user/1, ensure_users_db_exists/1]).
-
--import(chttpd, [send_json/2, send_json/4, send_method_not_allowed/2]).
-
-special_test_authentication_handler(Req) ->
- case chttpd:header_value(Req, "WWW-Authenticate") of
- "X-Couch-Test-Auth " ++ NamePass ->
- % NamePass is a colon separated string: "joe schmoe:a password".
- [Name, Pass] = re:split(NamePass, ":", [{return, list}]),
- case {Name, Pass} of
- {"Jan Lehnardt", "apple"} -> ok;
- {"Christopher Lenz", "dog food"} -> ok;
- {"Noah Slater", "biggiesmalls endian"} -> ok;
- {"Chris Anderson", "mp3"} -> ok;
- {"Damien Katz", "pecan pie"} -> ok;
- {_, _} ->
- throw({unauthorized, <<"Name or password is incorrect.">>})
- end,
- Req#httpd{user_ctx=#user_ctx{name=?l2b(Name)}};
- _ ->
- % No X-Couch-Test-Auth credentials sent, give admin access so the
- % previous authentication can be restored after the test
- Req#httpd{user_ctx=#user_ctx{roles=[<<"_admin">>]}}
- end.
-
-null_authentication_handler(Req) ->
- Ctx = #user_ctx{roles=[<<"_reader">>, <<"writer">>, <<"_admin">>]},
- Req#httpd{user_ctx=Ctx}.
-
-default_authentication_handler(Req) ->
- case basic_username_pw(Req) of
- {Username, Password} ->
- case get_user(Username) of
- nil ->
- throw({unauthorized, <<"unknown username">>});
- Props ->
- ExpectedHash = couch_util:get_value(<<"password_sha">>, Props),
- Salt = couch_util:get_value(<<"salt">>, Props),
- PasswordHash = hash_password(?l2b(Password), Salt),
- case couch_util:verify(ExpectedHash, PasswordHash) of
- true ->
- Ctx = #user_ctx{
- name = couch_util:get_value(<<"username">>, Props),
- roles = couch_util:get_value(<<"roles">>, Props)
- },
- Req#httpd{user_ctx=Ctx};
- _ ->
- throw({unauthorized, <<"password is incorrect">>})
- end
- end;
- nil ->
- Req
- end.
-
-cookie_authentication_handler(#httpd{path_parts=[<<"_session">>],
- method='POST'} = Req) ->
- % ignore any cookies sent with login request
- Req;
-cookie_authentication_handler(Req) ->
- try cookie_auth_user(Req) of
- nil ->
- Req;
- {cookie_auth_failed, _} = X ->
- Req#httpd{auth=X};
- Req2 ->
- Req2
- catch error:_ ->
- Req#httpd{auth={cookie_auth_failed, {invalid_cookie, null}}}
- end.
-
-cookie_auth_header(#httpd{auth={cookie_auth_failed, _}}, Headers) ->
- % check for an AuthSession cookie from login handler
- CookieHeader = couch_util:get_value("Set-Cookie", Headers, ""),
- Cookies = mochiweb_cookies:parse_cookie(CookieHeader),
- AuthSession = couch_util:get_value("AuthSession", Cookies),
- if AuthSession == undefined ->
- [generate_cookie_buster()];
- true ->
- []
- end;
-cookie_auth_header(#httpd{user_ctx=#user_ctx{name=null}}, _Headers) ->
- [];
-cookie_auth_header(#httpd{user_ctx=Ctx, auth={Secret,true}}, Headers) ->
- % Note: we only set the AuthSession cookie if:
- % * a valid AuthSession cookie has been received
- % * we are outside a 10% timeout window
- % * and if an AuthSession cookie hasn't already been set e.g. by a login
- % or logout handler.
- % The login and logout handlers set the AuthSession cookie themselves.
- CookieHeader = couch_util:get_value("Set-Cookie", Headers, ""),
- Cookies = mochiweb_cookies:parse_cookie(CookieHeader),
- AuthSession = couch_util:get_value("AuthSession", Cookies),
- if AuthSession == undefined ->
- [generate_cookie(Ctx#user_ctx.name, Secret, timestamp())];
- true ->
- []
- end;
-cookie_auth_header(_Req, _Headers) ->
- [].
-
-handle_session_req(#httpd{method='POST', mochi_req=MochiReq, user_ctx=Ctx}=Req) ->
- % login
- Form = parse_form(MochiReq),
- UserName = extract_username(Form),
- case get_user(UserName) of
- nil ->
- throw({forbidden, <<"unknown username">>});
- User ->
- UserSalt = couch_util:get_value(<<"salt">>, User),
- case lists:member(<<"_admin">>, Ctx#user_ctx.roles) of
- true ->
- ok;
- false ->
- Password = extract_password(Form),
- ExpectedHash = couch_util:get_value(<<"password_sha">>, User),
- PasswordHash = hash_password(Password, UserSalt),
- case couch_util:verify(ExpectedHash, PasswordHash) of
- true ->
- ok;
- _Else ->
- throw({forbidden, <<"Name or password is incorrect.">>})
- end
- end,
- Secret = ?l2b(couch_config:get("couch_httpd_auth", "secret")),
- SecretAndSalt = <<Secret/binary, UserSalt/binary>>,
- Cookie = generate_cookie(UserName, SecretAndSalt, timestamp()),
- send_response(Req, [Cookie])
- end;
-handle_session_req(#httpd{method='GET', user_ctx=UserCtx}=Req) ->
- % whoami
- #user_ctx{name = Name, roles = Roles} = UserCtx,
- ForceLogin = chttpd:qs_value(Req, "basic", "false"),
- case {Name, ForceLogin} of
- {null, "true"} ->
- throw({unauthorized, <<"Please login.">>});
- _False ->
- Props = [{name,Name}, {roles,Roles}],
- send_json(Req, {[{ok,true}, {userCtx, {Props}} | Props]})
- end;
-handle_session_req(#httpd{method='DELETE'}=Req) ->
- % logout
- send_response(Req, [generate_cookie_buster()]);
-handle_session_req(Req) ->
- send_method_not_allowed(Req, "GET,HEAD,POST,DELETE").
-
-handle_user_req(#httpd{method='POST'}=Req) ->
- DbName = couch_config:get("couch_httpd_auth", "authentication_db", "users"),
- ensure_users_db_exists(DbName),
- create_user(Req, DbName);
-handle_user_req(#httpd{method=Method, path_parts=[_]}=_Req) when
- Method == 'PUT' orelse Method == 'DELETE' ->
- 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", "users"),
- ensure_users_db_exists(DbName),
- update_user(Req, DbName, UserName);
-handle_user_req(#httpd{method='DELETE', path_parts=[_, UserName]}=Req) ->
- DbName = couch_config:get("couch_httpd_auth", "authentication_db", "users"),
- ensure_users_db_exists(DbName),
- delete_user(Req, DbName, UserName);
-handle_user_req(Req) ->
- send_method_not_allowed(Req, "DELETE,POST,PUT").
-
-get_user(UserName) when is_list(UserName) ->
- get_user(?l2b(UserName));
-get_user(UserName) ->
- case couch_config:get("admins", ?b2l(UserName)) of
- "-hashed-" ++ HashedPwdAndSalt ->
- [HashedPwd, Salt] = string:tokens(HashedPwdAndSalt, ","),
- [
- {<<"username">>, UserName},
- {<<"roles">>, [<<"_reader">>, <<"_writer">>, <<"_admin">>]},
- {<<"salt">>, ?l2b(Salt)},
- {<<"password_sha">>, ?l2b(HashedPwd)}
- ];
- _ ->
- try ets:lookup(users_cache, UserName) of
- [{UserName, Props}] ->
- Props;
- [] ->
- load_user_from_db(UserName)
- catch error:badarg ->
- load_user_from_db(UserName)
- end
- end.
-
-load_user_from_db(UserName) ->
- DbName = couch_config:get("couch_httpd_auth", "authentication_db", "users"),
- try fabric:open_doc(DbName, UserName, []) of
- {ok, Doc} ->
- ?LOG_INFO("cache miss on username ~s", [UserName]),
- {Props} = couch_doc:to_json_obj(Doc, []),
- Props;
- _Else ->
- ?LOG_INFO("no record of user ~s", [UserName]),
- nil
- catch error:database_does_not_exist ->
- nil
- end.
-
-ensure_users_db_exists(DbName) ->
- try fabric:get_doc_count(DbName) of
- {ok, N} when is_integer(N) ->
- ok;
- {error, _} ->
- fabric:create_db(DbName, [])
- catch error:database_does_not_exist ->
- fabric:create_db(DbName, [])
- end.
-
-% internal functions
-
-basic_username_pw(Req) ->
- case chttpd:header_value(Req, "Authorization") of
- "Basic " ++ Base64Value ->
- case string:tokens(?b2l(base64:decode(Base64Value)),":") of
- [User, Pass] ->
- {User, Pass};
- [User] ->
- {User, ""};
- _ ->
- nil
- end;
- _ ->
- nil
- end.
-
-cookie_auth_user(#httpd{mochi_req=MochiReq}=Req) ->
- case MochiReq:get_cookie_value("AuthSession") of
- undefined ->
- nil;
- Cookie ->
- AuthSession = couch_util:decodeBase64Url(Cookie),
- [User, TimeStr | HashParts] = string:tokens(?b2l(AuthSession), ":"),
- % Verify expiry and hash
- case couch_config:get("couch_httpd_auth", "secret") of
- undefined ->
- ?LOG_DEBUG("AuthSession cookie, but no secret in config!", []),
- {cookie_auth_failed, {internal_server_error, null}};
- SecretStr ->
- case get_user(User) of
- nil ->
- Msg = io_lib:format("no record of user ~s", [User]),
- {cookie_auth_failed, {bad_user, ?l2b(Msg)}};
- Result ->
- Secret = ?l2b(SecretStr),
- UserSalt = couch_util:get_value(<<"salt">>, Result),
- FullSecret = <<Secret/binary, UserSalt/binary>>,
- ExpectedHash = crypto:sha_mac(FullSecret, [User, ":", TimeStr]),
- PasswordHash = ?l2b(string:join(HashParts, ":")),
- case couch_util:verify(ExpectedHash, PasswordHash) of
- true ->
- TimeStamp = erlang:list_to_integer(TimeStr, 16),
- Timeout = erlang:list_to_integer(couch_config:get(
- "couch_httpd_auth", "timeout", "600")),
- CurrentTime = timestamp(),
- if CurrentTime < TimeStamp + Timeout ->
- TimeLeft = TimeStamp + Timeout - CurrentTime,
- Req#httpd{user_ctx=#user_ctx{
- name=?l2b(User),
- roles=couch_util:get_value(<<"roles">>, Result, [])
- }, auth={FullSecret, TimeLeft < Timeout*0.9}};
- true ->
- ?LOG_DEBUG("cookie for ~s was expired", [User]),
- Msg = lists:concat(["Your session has expired after ",
- Timeout div 60, " minutes of inactivity"]),
- {cookie_auth_failed, {credentials_expired, ?l2b(Msg)}}
- end;
- _Else ->
- Msg = <<"cookie password hash was incorrect">>,
- {cookie_auth_failed, {bad_password, Msg}}
- end
- end
- end
- end.
-
-create_user(#httpd{method='POST', mochi_req=MochiReq}=Req, Db) ->
- Form = parse_form(MochiReq),
- {UserName, Password} = extract_username_password(Form),
- case get_user(UserName) of
- nil ->
- Roles = [?l2b(R) || R <- proplists:get_all_values("roles", Form)],
- if Roles /= [] ->
- chttpd:verify_is_server_admin(Req);
- true -> ok end,
- Active = chttpd_view:parse_bool_param(couch_util:get_value("active",
- Form, "true")),
- UserSalt = couch_uuids:random(),
- UserDoc = #doc{
- id = UserName,
- body = {[
- {<<"active">>, Active},
- {<<"email">>, ?l2b(couch_util:get_value("email", Form, ""))},
- {<<"password_sha">>, hash_password(Password, UserSalt)},
- {<<"roles">>, Roles},
- {<<"salt">>, UserSalt},
- {<<"type">>, <<"user">>},
- {<<"username">>, UserName}
- ]}
- },
- {ok, _Rev} = fabric:update_doc(Db, UserDoc, []),
- ?LOG_DEBUG("User ~s (~s) with password, ~s created.", [UserName,
- UserName, Password]),
- send_response(Req);
- _Result ->
- ?LOG_DEBUG("Can't create ~s: already exists", [UserName]),
- throw({forbidden, <<"User already exists.">>})
- end.
-
-delete_user(#httpd{user_ctx=UserCtx}=Req, Db, UserName) ->
- case get_user(UserName) of
- nil ->
- throw({not_found, <<"User doesn't exist">>});
- User ->
- case lists:member(<<"_admin">>,UserCtx#user_ctx.roles) of
- true ->
- ok;
- false when UserCtx#user_ctx.name == UserName ->
- ok;
- false ->
- throw({forbidden, <<"You aren't allowed to delete the user">>})
- end,
- {Pos,Rev} = couch_doc:parse_rev(couch_util:get_value(<<"_rev">>,User)),
- UserDoc = #doc{
- id = UserName,
- revs = {Pos, [Rev]},
- deleted = true
- },
- {ok, _Rev} = fabric:update_doc(Db, UserDoc, []),
- send_response(Req)
- end.
-
-extract_username(Form) ->
- CouchFormat = couch_util:get_value("name", Form),
- try ?l2b(couch_util:get_value("username", Form, CouchFormat))
- catch error:badarg ->
- throw({bad_request, <<"user accounts must have a username">>})
- end.
-
-extract_password(Form) ->
- try ?l2b(couch_util:get_value("password", Form))
- catch error:badarg ->
- throw({bad_request, <<"user accounts must have a password">>})
- end.
-
-extract_username_password(Form) ->
- CouchFormat = couch_util:get_value("name", Form),
- try
- {?l2b(couch_util:get_value("username", Form, CouchFormat)),
- ?l2b(couch_util:get_value("password", Form))}
- catch error:badarg ->
- Msg = <<"user accounts must have a username and password">>,
- throw({bad_request, Msg})
- end.
-
-generate_cookie_buster() ->
- T0 = calendar:now_to_datetime({0,86400,0}),
- Opts = [{max_age,0}, {path,"/"}, {local_time,T0}],
- mochiweb_cookies:cookie("AuthSession", "", Opts).
-
-generate_cookie(User, Secret, TimeStamp) ->
- SessionData = ?b2l(User) ++ ":" ++ erlang:integer_to_list(TimeStamp, 16),
- Hash = crypto:sha_mac(Secret, SessionData),
- Cookie = couch_util:encodeBase64Url(SessionData ++ ":" ++ ?b2l(Hash)),
- % TODO add {secure, true} to options when SSL is detected
- mochiweb_cookies:cookie("AuthSession", Cookie, [{path, "/"}]).
-
-hash_password(Password, Salt) ->
- ?l2b(couch_util:to_hex(crypto:sha(<<Password/binary, Salt/binary>>))).
-
-parse_form(MochiReq) ->
- case MochiReq:get_primary_header_value("content-type") of
- "application/x-www-form-urlencoded" ++ _ ->
- ReqBody = MochiReq:recv_body(),
- mochiweb_util:parse_qs(ReqBody);
- _ ->
- throw({bad_request, <<"you must specify "
- "application/x-www-form-urlencoded as the primary content-type">>})
- end.
-
-send_response(Req) ->
- send_response(Req, []).
-
-send_response(Req, ExtraHeaders) ->
- {Code, Headers} = case chttpd:qs_value(Req, "next", nil) of
- nil -> {200, []};
- Redirect ->
- {302, [{"Location", chttpd:absolute_uri(Req, Redirect)}]}
- end,
- send_json(Req, Code, Headers ++ ExtraHeaders, {[{ok, true}]}).
-
-timestamp() ->
- {MegaSeconds, Seconds, _} = erlang:now(),
- MegaSeconds * 1000000 + Seconds.
-
-update_user(#httpd{mochi_req=MochiReq, user_ctx=UserCtx}=Req, Db, UserName) ->
- case get_user(UserName) of
- nil ->
- throw({not_found, <<"User doesn't exist">>});
- User ->
- Form = parse_form(MochiReq),
- NewPassword = ?l2b(couch_util:get_value("password", Form, "")),
- OldPassword = ?l2b(couch_util:get_value("old_password", Form, "")),
-
- UserSalt = couch_util:get_value(<<"salt">>, User),
- CurrentPasswordHash = couch_util:get_value(<<"password_sha">>, User),
-
- Roles = [?l2b(R) || R <- proplists:get_all_values("roles", Form)],
- if Roles /= [] ->
- chttpd:verify_is_server_admin(Req);
- true -> ok end,
-
- PasswordHash = case NewPassword of
- <<>> ->
- CurrentPasswordHash;
- _Else ->
- case lists:member(<<"_admin">>,UserCtx#user_ctx.roles) of
- true ->
- hash_password(NewPassword, UserSalt);
- false when UserCtx#user_ctx.name == UserName ->
- %% for user we test old password before allowing change
- case hash_password(OldPassword, UserSalt) of
- CurrentPasswordHash ->
- hash_password(NewPassword, UserSalt);
- _ ->
- throw({forbidden, <<"Old password is incorrect.">>})
- end;
- _ ->
- Msg = <<"You aren't allowed to change this password.">>,
- throw({forbidden, Msg})
- end
- end,
-
- Active = chttpd_view:parse_bool_param(couch_util:get_value("active",
- Form, "true")),
- {Pos,Rev} = couch_doc:parse_rev(couch_util:get_value(<<"_rev">>,User)),
- UserDoc = #doc{
- id = UserName,
- revs = {Pos,[Rev]},
- body = {[
- {<<"active">>, Active},
- {<<"email">>, ?l2b(couch_util:get_value("email", Form, ""))},
- {<<"password_sha">>, PasswordHash},
- {<<"roles">>, Roles},
- {<<"salt">>, UserSalt},
- {<<"type">>, <<"user">>},
- {<<"username">>, UserName}
- ]}
- },
- {ok, _Rev} = fabric:update_doc(Db, UserDoc, []),
- ?LOG_DEBUG("User ~s updated.", [UserName]),
- send_response(Req)
- end.
diff --git a/apps/chttpd/src/chttpd_db.erl b/apps/chttpd/src/chttpd_db.erl
deleted file mode 100644
index fc219621..00000000
--- a/apps/chttpd/src/chttpd_db.erl
+++ /dev/null
@@ -1,1082 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_db).
--include_lib("couch/include/couch_db.hrl").
-
--export([handle_request/1, handle_compact_req/2, handle_design_req/2,
- db_req/2, couch_doc_open/4,handle_changes_req/2,
- update_doc_result_to_json/1, update_doc_result_to_json/2,
- handle_design_info_req/3, handle_view_cleanup_req/2]).
-
--import(chttpd,
- [send_json/2,send_json/3,send_json/4,send_method_not_allowed/2,
- start_json_response/2,send_chunk/2,end_json_response/1,
- start_chunked_response/3, absolute_uri/2, send/2,
- start_response_length/4]).
-
--record(doc_query_args, {
- options = [],
- rev = nil,
- open_revs = [],
- update_type = interactive_edit,
- atts_since = nil
-}).
-
-% Database request handlers
-handle_request(#httpd{path_parts=[DbName|RestParts],method=Method,
- db_url_handlers=DbUrlHandlers}=Req)->
- case {Method, RestParts} of
- {'PUT', []} ->
- create_db_req(Req, DbName);
- {'DELETE', []} ->
- delete_db_req(Req, DbName);
- {_, []} ->
- do_db_req(Req, fun db_req/2);
- {_, [SecondPart|_]} ->
- Handler = couch_util:get_value(SecondPart, DbUrlHandlers, fun db_req/2),
- do_db_req(Req, Handler)
- end.
-
-handle_changes_req(#httpd{method='GET'}=Req, Db) ->
- #changes_args{filter=Raw, style=Style} = Args0 = parse_changes_query(Req),
- ChangesArgs = Args0#changes_args{
- filter = couch_changes:configure_filter(Raw, Style, Req, Db)
- },
- case ChangesArgs#changes_args.feed of
- "normal" ->
- T0 = now(),
- {ok, Info} = fabric:get_db_info(Db),
- Etag = chttpd:make_etag(Info),
- DeltaT = timer:now_diff(now(), T0) / 1000,
- couch_stats_collector:record({couchdb, dbinfo}, DeltaT),
- chttpd:etag_respond(Req, Etag, fun() ->
- {ok, Resp} = chttpd:start_json_response(Req, 200, [{"Etag",Etag}]),
- fabric:changes(Db, fun changes_callback/2, {"normal", Resp},
- ChangesArgs)
- end);
- Feed ->
- % "longpoll" or "continuous"
- {ok, Resp} = chttpd:start_json_response(Req, 200),
- fabric:changes(Db, fun changes_callback/2, {Feed, Resp}, ChangesArgs)
- end;
-handle_changes_req(#httpd{path_parts=[_,<<"_changes">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "GET,HEAD").
-
-% callbacks for continuous feed (newline-delimited JSON Objects)
-changes_callback(start, {"continuous", _} = Acc) ->
- {ok, Acc};
-changes_callback({change, Change}, {"continuous", Resp} = Acc) ->
- send_chunk(Resp, [?JSON_ENCODE(Change) | "\n"]),
- {ok, Acc};
-changes_callback({stop, EndSeq0}, {"continuous", Resp}) ->
- EndSeq = case is_old_couch(Resp) of true -> 0; false -> EndSeq0 end,
- send_chunk(Resp, [?JSON_ENCODE({[{<<"last_seq">>, EndSeq}]}) | "\n"]),
- end_json_response(Resp);
-
-% callbacks for longpoll and normal (single JSON Object)
-changes_callback(start, {_, Resp}) ->
- send_chunk(Resp, "{\"results\":[\n"),
- {ok, {"", Resp}};
-changes_callback({change, Change}, {Prepend, Resp}) ->
- send_chunk(Resp, [Prepend, ?JSON_ENCODE(Change)]),
- {ok, {",\r\n", Resp}};
-changes_callback({stop, EndSeq}, {_, Resp}) ->
- case is_old_couch(Resp) of
- true ->
- send_chunk(Resp, "\n],\n\"last_seq\":0}\n");
- false ->
- send_chunk(Resp, io_lib:format("\n],\n\"last_seq\":\"~s\"}\n",[EndSeq]))
- end,
- end_json_response(Resp);
-
-changes_callback(timeout, {Prepend, Resp}) ->
- send_chunk(Resp, "\n"),
- {ok, {Prepend, Resp}};
-changes_callback({error, Reason}, Resp) ->
- chttpd:send_chunked_error(Resp, {error, Reason}).
-
-is_old_couch(Resp) ->
- MochiReq = Resp:get(request),
- case MochiReq:get_header_value("user-agent") of
- undefined ->
- false;
- UserAgent ->
- string:str(UserAgent, "CouchDB/0") > 0
- end.
-
-handle_compact_req(Req, _) ->
- Msg = <<"Compaction must be triggered on a per-shard basis in BigCouch">>,
- couch_httpd:send_error(Req, 403, forbidden, Msg).
-
-handle_view_cleanup_req(Req, Db) ->
- ok = fabric:cleanup_index_files(Db),
- send_json(Req, 202, {[{ok, true}]}).
-
-handle_design_req(#httpd{
- path_parts=[_DbName, _Design, Name, <<"_",_/binary>> = Action | _Rest],
- design_url_handlers = DesignUrlHandlers
- }=Req, Db) ->
- case fabric:open_doc(Db, <<"_design/", Name/binary>>, []) of
- {ok, DDoc} ->
- % TODO we'll trigger a badarity here if ddoc attachment starts with "_",
- % or if user tries an unknown Action
- Handler = couch_util:get_value(Action, DesignUrlHandlers, fun db_req/2),
- Handler(Req, Db, DDoc);
- Error ->
- throw(Error)
- end;
-
-handle_design_req(Req, Db) ->
- db_req(Req, Db).
-
-handle_design_info_req(#httpd{method='GET'}=Req, Db, #doc{id=Id} = DDoc) ->
- {ok, GroupInfoList} = fabric:get_view_group_info(Db, DDoc),
- send_json(Req, 200, {[
- {name, Id},
- {view_index, {GroupInfoList}}
- ]});
-
-handle_design_info_req(Req, _Db, _DDoc) ->
- send_method_not_allowed(Req, "GET").
-
-create_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) ->
- N = couch_httpd:qs_value(Req, "n", couch_config:get("cluster", "n", "3")),
- Q = couch_httpd:qs_value(Req, "q", couch_config:get("cluster", "q", "8")),
- case fabric:create_db(DbName, [{user_ctx,UserCtx}, {n,N}, {q,Q}]) of
- ok ->
- DocUrl = absolute_uri(Req, "/" ++ couch_util:url_encode(DbName)),
- send_json(Req, 201, [{"Location", DocUrl}], {[{ok, true}]});
- {error, file_exists} ->
- chttpd:send_error(Req, file_exists);
- Error ->
- throw(Error)
- end.
-
-delete_db_req(#httpd{user_ctx=UserCtx}=Req, DbName) ->
- case fabric:delete_db(DbName, [{user_ctx, UserCtx}]) of
- ok ->
- send_json(Req, 200, {[{ok, true}]});
- Error ->
- throw(Error)
- end.
-
-do_db_req(#httpd{path_parts=[DbName|_]}=Req, Fun) ->
- Fun(Req, #db{name=DbName}).
-
-db_req(#httpd{method='GET',path_parts=[DbName]}=Req, _Db) ->
- % measure the time required to generate the etag, see if it's worth it
- T0 = now(),
- {ok, DbInfo} = fabric:get_db_info(DbName),
- DeltaT = timer:now_diff(now(), T0) / 1000,
- couch_stats_collector:record({couchdb, dbinfo}, DeltaT),
- send_json(Req, {DbInfo});
-
-db_req(#httpd{method='POST', path_parts=[DbName], user_ctx=Ctx}=Req, Db) ->
- couch_httpd:validate_ctype(Req, "application/json"),
- Doc = couch_doc:from_json_obj(couch_httpd:json_body(Req)),
- Doc2 = case Doc#doc.id of
- <<"">> ->
- Doc#doc{id=couch_uuids:new(), revs={0, []}};
- _ ->
- Doc
- end,
- DocId = Doc2#doc.id,
- case couch_httpd:qs_value(Req, "batch") of
- "ok" ->
- % async_batching
- spawn(fun() ->
- case catch(fabric:update_doc(Db, Doc2, [{user_ctx, Ctx}])) of
- {ok, _} -> ok;
- Error ->
- ?LOG_INFO("Batch doc error (~s): ~p",[DocId, Error])
- end
- end),
-
- send_json(Req, 202, [], {[
- {ok, true},
- {id, DocId}
- ]});
- _Normal ->
- % normal
- {ok, NewRev} = fabric:update_doc(Db, Doc2, [{user_ctx, Ctx}]),
- DocUrl = absolute_uri(Req, [$/, DbName, $/, DocId]),
- send_json(Req, 201, [{"Location", DocUrl}], {[
- {ok, true},
- {id, DocId},
- {rev, couch_doc:rev_to_str(NewRev)}
- ]})
- end;
-
-db_req(#httpd{path_parts=[_DbName]}=Req, _Db) ->
- send_method_not_allowed(Req, "DELETE,GET,HEAD,POST");
-
-db_req(#httpd{method='POST',path_parts=[_,<<"_ensure_full_commit">>]}=Req, _Db) ->
- send_json(Req, 201, {[
- {ok, true},
- {instance_start_time, <<"0">>}
- ]});
-
-db_req(#httpd{path_parts=[_,<<"_ensure_full_commit">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "POST");
-
-db_req(#httpd{method='POST',path_parts=[_,<<"_bulk_docs">>], user_ctx=Ctx}=Req, Db) ->
- couch_stats_collector:increment({httpd, bulk_requests}),
- couch_httpd:validate_ctype(Req, "application/json"),
- {JsonProps} = chttpd:json_body_obj(Req),
- DocsArray = couch_util:get_value(<<"docs">>, JsonProps),
- case chttpd:header_value(Req, "X-Couch-Full-Commit") of
- "true" ->
- Options = [full_commit, {user_ctx,Ctx}];
- "false" ->
- Options = [delay_commit, {user_ctx,Ctx}];
- _ ->
- Options = [{user_ctx,Ctx}]
- end,
- case couch_util:get_value(<<"new_edits">>, JsonProps, true) of
- true ->
- Docs = lists:map(
- fun({ObjProps} = JsonObj) ->
- Doc = couch_doc:from_json_obj(JsonObj),
- validate_attachment_names(Doc),
- Id = case Doc#doc.id of
- <<>> -> couch_uuids:new();
- Id0 -> Id0
- end,
- case couch_util:get_value(<<"_rev">>, ObjProps) of
- undefined ->
- Revs = {0, []};
- Rev ->
- {Pos, RevId} = couch_doc:parse_rev(Rev),
- Revs = {Pos, [RevId]}
- end,
- Doc#doc{id=Id,revs=Revs}
- end,
- DocsArray),
- Options2 =
- case couch_util:get_value(<<"all_or_nothing">>, JsonProps) of
- true -> [all_or_nothing|Options];
- _ -> Options
- end,
- case fabric:update_docs(Db, Docs, Options2) of
- {ok, Results} ->
- % output the results
- DocResults = lists:zipwith(fun update_doc_result_to_json/2,
- Docs, Results),
- send_json(Req, 201, DocResults);
- {aborted, Errors} ->
- ErrorsJson =
- lists:map(fun update_doc_result_to_json/1, Errors),
- send_json(Req, 417, ErrorsJson)
- end;
- false ->
- Docs = [couch_doc:from_json_obj(JsonObj) || JsonObj <- DocsArray],
- [validate_attachment_names(D) || D <- Docs],
- {ok, Errors} = fabric:update_docs(Db, Docs, [replicated_changes|Options]),
- ErrorsJson = lists:map(fun update_doc_result_to_json/1, Errors),
- send_json(Req, 201, ErrorsJson)
- end;
-
-db_req(#httpd{path_parts=[_,<<"_bulk_docs">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "POST");
-
-db_req(#httpd{method='POST',path_parts=[_,<<"_purge">>]}=Req, Db) ->
- couch_httpd:validate_ctype(Req, "application/json"),
- {IdsRevs} = chttpd:json_body_obj(Req),
- IdsRevs2 = [{Id, couch_doc:parse_revs(Revs)} || {Id, Revs} <- IdsRevs],
- case fabric:purge_docs(Db, IdsRevs2) of
- {ok, PurgeSeq, PurgedIdsRevs} ->
- PurgedIdsRevs2 = [{Id, couch_doc:revs_to_strs(Revs)} || {Id, Revs}
- <- PurgedIdsRevs],
- send_json(Req, 200, {[
- {<<"purge_seq">>, PurgeSeq},
- {<<"purged">>, {PurgedIdsRevs2}}
- ]});
- Error ->
- throw(Error)
- end;
-
-db_req(#httpd{path_parts=[_,<<"_purge">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "POST");
-
-db_req(#httpd{method='GET',path_parts=[_,<<"_all_docs">>]}=Req, Db) ->
- all_docs_view(Req, Db, nil);
-
-db_req(#httpd{method='POST',path_parts=[_,<<"_all_docs">>]}=Req, Db) ->
- {Fields} = chttpd:json_body_obj(Req),
- case couch_util:get_value(<<"keys">>, Fields, nil) of
- Keys when is_list(Keys) ->
- all_docs_view(Req, Db, Keys);
- nil ->
- all_docs_view(Req, Db, nil);
- _ ->
- throw({bad_request, "`keys` body member must be an array."})
- end;
-
-db_req(#httpd{path_parts=[_,<<"_all_docs">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "GET,HEAD,POST");
-
-db_req(#httpd{method='POST',path_parts=[_,<<"_missing_revs">>]}=Req, Db) ->
- {JsonDocIdRevs} = couch_httpd:json_body_obj(Req),
- {ok, Results} = fabric:get_missing_revs(Db, JsonDocIdRevs),
- Results2 = [{Id, couch_doc:revs_to_strs(Revs)} || {Id, Revs} <- Results],
- send_json(Req, {[
- {missing_revs, {Results2}}
- ]});
-
-db_req(#httpd{path_parts=[_,<<"_missing_revs">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "POST");
-
-db_req(#httpd{method='POST',path_parts=[_,<<"_revs_diff">>]}=Req, Db) ->
- {JsonDocIdRevs} = couch_httpd:json_body_obj(Req),
- {ok, Results} = fabric:get_missing_revs(Db, JsonDocIdRevs),
- Results2 =
- lists:map(fun({Id, MissingRevs, PossibleAncestors}) ->
- {Id,
- {[{missing, couch_doc:revs_to_strs(MissingRevs)}] ++
- if PossibleAncestors == [] ->
- [];
- true ->
- [{possible_ancestors,
- couch_doc:revs_to_strs(PossibleAncestors)}]
- end}}
- end, Results),
- send_json(Req, {Results2});
-
-db_req(#httpd{path_parts=[_,<<"_revs_diff">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "POST");
-
-db_req(#httpd{method='PUT',path_parts=[_,<<"_security">>],user_ctx=Ctx}=Req,
- Db) ->
- SecObj = couch_httpd:json_body(Req),
- ok = fabric:set_security(Db, SecObj, [{user_ctx,Ctx}]),
- send_json(Req, {[{<<"ok">>, true}]});
-
-db_req(#httpd{method='GET',path_parts=[_,<<"_security">>]}=Req, Db) ->
- send_json(Req, fabric:get_security(Db));
-
-db_req(#httpd{path_parts=[_,<<"_security">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "PUT,GET");
-
-db_req(#httpd{method='PUT',path_parts=[_,<<"_revs_limit">>],user_ctx=Ctx}=Req,
- Db) ->
- Limit = chttpd:json_body(Req),
- ok = fabric:set_revs_limit(Db, Limit, [{user_ctx,Ctx}]),
- send_json(Req, {[{<<"ok">>, true}]});
-
-db_req(#httpd{method='GET',path_parts=[_,<<"_revs_limit">>]}=Req, Db) ->
- send_json(Req, fabric:get_revs_limit(Db));
-
-db_req(#httpd{path_parts=[_,<<"_revs_limit">>]}=Req, _Db) ->
- send_method_not_allowed(Req, "PUT,GET");
-
-% vanilla CouchDB sends a 301 here, but we just handle the request
-db_req(#httpd{path_parts=[DbName,<<"_design/",Name/binary>>|Rest]}=Req, Db) ->
- db_req(Req#httpd{path_parts=[DbName, <<"_design">>, Name | Rest]}, Db);
-
-% Special case to enable using an unencoded slash in the URL of design docs,
-% as slashes in document IDs must otherwise be URL encoded.
-db_req(#httpd{path_parts=[_DbName,<<"_design">>,Name]}=Req, Db) ->
- db_doc_req(Req, Db, <<"_design/",Name/binary>>);
-
-db_req(#httpd{path_parts=[_DbName,<<"_design">>,Name|FileNameParts]}=Req, Db) ->
- db_attachment_req(Req, Db, <<"_design/",Name/binary>>, FileNameParts);
-
-
-% Special case to allow for accessing local documents without %2F
-% encoding the docid. Throws out requests that don't have the second
-% path part or that specify an attachment name.
-db_req(#httpd{path_parts=[_DbName, <<"_local">>]}, _Db) ->
- throw({bad_request, <<"Invalid _local document id.">>});
-
-db_req(#httpd{path_parts=[_DbName, <<"_local/">>]}, _Db) ->
- throw({bad_request, <<"Invalid _local document id.">>});
-
-db_req(#httpd{path_parts=[_DbName, <<"_local">>, Name]}=Req, Db) ->
- db_doc_req(Req, Db, <<"_local/", Name/binary>>);
-
-db_req(#httpd{path_parts=[_DbName, <<"_local">> | _Rest]}, _Db) ->
- throw({bad_request, <<"_local documents do not accept attachments.">>});
-
-db_req(#httpd{path_parts=[_, DocId]}=Req, Db) ->
- db_doc_req(Req, Db, DocId);
-
-db_req(#httpd{path_parts=[_, DocId | FileNameParts]}=Req, Db) ->
- db_attachment_req(Req, Db, DocId, FileNameParts).
-
-all_docs_view(Req, Db, Keys) ->
- % measure the time required to generate the etag, see if it's worth it
- T0 = now(),
- {ok, Info} = fabric:get_db_info(Db),
- Etag = couch_httpd:make_etag(Info),
- DeltaT = timer:now_diff(now(), T0) / 1000,
- couch_stats_collector:record({couchdb, dbinfo}, DeltaT),
- QueryArgs = chttpd_view:parse_view_params(Req, Keys, map),
- chttpd:etag_respond(Req, Etag, fun() ->
- {ok, Resp} = chttpd:start_json_response(Req, 200, [{"Etag",Etag}]),
- fabric:all_docs(Db, fun all_docs_callback/2, {nil, Resp}, QueryArgs)
- end).
-
-all_docs_callback({total_and_offset, Total, Offset}, {_, Resp}) ->
- Chunk = "{\"total_rows\":~p,\"offset\":~p,\"rows\":[\r\n",
- send_chunk(Resp, io_lib:format(Chunk, [Total, Offset])),
- {ok, {"", Resp}};
-all_docs_callback({row, Row}, {Prepend, Resp}) ->
- send_chunk(Resp, [Prepend, ?JSON_ENCODE(Row)]),
- {ok, {",\r\n", Resp}};
-all_docs_callback(complete, {_, Resp}) ->
- send_chunk(Resp, "\r\n]}"),
- end_json_response(Resp);
-all_docs_callback({error, Reason}, {_, Resp}) ->
- chttpd:send_chunked_error(Resp, {error, Reason}).
-
-db_doc_req(#httpd{method='DELETE'}=Req, Db, DocId) ->
- % check for the existence of the doc to handle the 404 case.
- couch_doc_open(Db, DocId, nil, []),
- case chttpd:qs_value(Req, "rev") of
- undefined ->
- Body = {[{<<"_deleted">>,true}]};
- Rev ->
- Body = {[{<<"_rev">>, ?l2b(Rev)},{<<"_deleted">>,true}]}
- end,
- update_doc(Req, Db, DocId, couch_doc_from_req(Req, DocId, Body));
-
-db_doc_req(#httpd{method='GET'}=Req, Db, DocId) ->
- #doc_query_args{
- rev = Rev,
- open_revs = Revs,
- options = Options,
- atts_since = AttsSince
- } = parse_doc_query(Req),
- case Revs of
- [] ->
- Options2 =
- if AttsSince /= nil ->
- [{atts_since, AttsSince}, attachments | Options];
- true -> Options
- end,
- Doc = couch_doc_open(Db, DocId, Rev, Options2),
- send_doc(Req, Doc, Options2);
- _ ->
- {ok, Results} = fabric:open_revs(Db, DocId, Revs, Options),
- AcceptedTypes = case couch_httpd:header_value(Req, "Accept") of
- undefined -> [];
- AcceptHeader -> string:tokens(AcceptHeader, "; ")
- end,
- case lists:member("multipart/mixed", AcceptedTypes) of
- false ->
- {ok, Resp} = start_json_response(Req, 200),
- send_chunk(Resp, "["),
- % We loop through the docs. The first time through the separator
- % is whitespace, then a comma on subsequent iterations.
- lists:foldl(
- fun(Result, AccSeparator) ->
- case Result of
- {ok, Doc} ->
- JsonDoc = couch_doc:to_json_obj(Doc, Options),
- Json = ?JSON_ENCODE({[{ok, JsonDoc}]}),
- send_chunk(Resp, AccSeparator ++ Json);
- {{not_found, missing}, RevId} ->
- RevStr = couch_doc:rev_to_str(RevId),
- Json = ?JSON_ENCODE({[{"missing", RevStr}]}),
- send_chunk(Resp, AccSeparator ++ Json)
- end,
- "," % AccSeparator now has a comma
- end,
- "", Results),
- send_chunk(Resp, "]"),
- end_json_response(Resp);
- true ->
- send_docs_multipart(Req, Results, Options)
- end
- end;
-
-db_doc_req(#httpd{method='POST', user_ctx=Ctx}=Req, Db, DocId) ->
- couch_httpd:validate_referer(Req),
- couch_doc:validate_docid(DocId),
- couch_httpd:validate_ctype(Req, "multipart/form-data"),
- Form = couch_httpd:parse_form(Req),
- case proplists:is_defined("_doc", Form) of
- true ->
- Json = ?JSON_DECODE(couch_util:get_value("_doc", Form)),
- Doc = couch_doc_from_req(Req, DocId, Json);
- false ->
- Rev = couch_doc:parse_rev(list_to_binary(couch_util:get_value("_rev", Form))),
- {ok, [{ok, Doc}]} = fabric:open_revs(Db, DocId, [Rev], [])
- end,
- UpdatedAtts = [
- #att{name=validate_attachment_name(Name),
- type=list_to_binary(ContentType),
- data=Content} ||
- {Name, {ContentType, _}, Content} <-
- proplists:get_all_values("_attachments", Form)
- ],
- #doc{atts=OldAtts} = Doc,
- OldAtts2 = lists:flatmap(
- fun(#att{name=OldName}=Att) ->
- case [1 || A <- UpdatedAtts, A#att.name == OldName] of
- [] -> [Att]; % the attachment wasn't in the UpdatedAtts, return it
- _ -> [] % the attachment was in the UpdatedAtts, drop it
- end
- end, OldAtts),
- NewDoc = Doc#doc{
- atts = UpdatedAtts ++ OldAtts2
- },
- {ok, NewRev} = fabric:update_doc(Db, NewDoc, [{user_ctx,Ctx}]),
-
- send_json(Req, 201, [{"Etag", "\"" ++ ?b2l(couch_doc:rev_to_str(NewRev)) ++ "\""}], {[
- {ok, true},
- {id, DocId},
- {rev, couch_doc:rev_to_str(NewRev)}
- ]});
-
-db_doc_req(#httpd{method='PUT', user_ctx=Ctx}=Req, Db, DocId) ->
- #doc_query_args{
- update_type = UpdateType
- } = parse_doc_query(Req),
- couch_doc:validate_docid(DocId),
-
- W = couch_httpd:qs_value(Req, "w", couch_config:get("cluster", "w", "2")),
- Options = [{user_ctx,Ctx}, {w,W}],
-
- Loc = absolute_uri(Req, [$/, Db#db.name, $/, DocId]),
- RespHeaders = [{"Location", Loc}],
- case couch_util:to_list(couch_httpd:header_value(Req, "Content-Type")) of
- ("multipart/related;" ++ _) = ContentType ->
- {ok, Doc0} = couch_doc:doc_from_multi_part_stream(ContentType,
- fun() -> receive_request_data(Req) end),
- Doc = couch_doc_from_req(Req, DocId, Doc0),
- update_doc(Req, Db, DocId, Doc, RespHeaders, UpdateType);
- _Else ->
- case couch_httpd:qs_value(Req, "batch") of
- "ok" ->
- % batch
- Doc = couch_doc_from_req(Req, DocId, couch_httpd:json_body(Req)),
-
- spawn(fun() ->
- case catch(fabric:update_doc(Db, Doc, Options)) of
- {ok, _} -> ok;
- Error ->
- ?LOG_INFO("Batch doc error (~s): ~p",[DocId, Error])
- end
- end),
- send_json(Req, 202, [], {[
- {ok, true},
- {id, DocId}
- ]});
- _Normal ->
- % normal
- Body = couch_httpd:json_body(Req),
- Doc = couch_doc_from_req(Req, DocId, Body),
- update_doc(Req, Db, DocId, Doc, RespHeaders, UpdateType)
- end
- end;
-
-db_doc_req(#httpd{method='COPY', user_ctx=Ctx}=Req, Db, SourceDocId) ->
- SourceRev =
- case extract_header_rev(Req, couch_httpd:qs_value(Req, "rev")) of
- missing_rev -> nil;
- Rev -> Rev
- end,
- {TargetDocId, TargetRevs} = parse_copy_destination_header(Req),
- % open old doc
- Doc = couch_doc_open(Db, SourceDocId, SourceRev, []),
- % save new doc
- {ok, NewTargetRev} = fabric:update_doc(Db,
- Doc#doc{id=TargetDocId, revs=TargetRevs}, [{user_ctx,Ctx}]),
- % respond
- send_json(Req, 201,
- [{"Etag", "\"" ++ ?b2l(couch_doc:rev_to_str(NewTargetRev)) ++ "\""}],
- update_doc_result_to_json(TargetDocId, {ok, NewTargetRev}));
-
-db_doc_req(Req, _Db, _DocId) ->
- send_method_not_allowed(Req, "DELETE,GET,HEAD,POST,PUT,COPY").
-
-send_doc(Req, Doc, Options) ->
- case Doc#doc.meta of
- [] ->
- DiskEtag = couch_httpd:doc_etag(Doc),
- % output etag only when we have no meta
- couch_httpd:etag_respond(Req, DiskEtag, fun() ->
- send_doc_efficiently(Req, Doc, [{"Etag", DiskEtag}], Options)
- end);
- _ ->
- send_doc_efficiently(Req, Doc, [], Options)
- end.
-
-send_doc_efficiently(Req, #doc{atts=[]}=Doc, Headers, Options) ->
- send_json(Req, 200, Headers, couch_doc:to_json_obj(Doc, Options));
-send_doc_efficiently(Req, #doc{atts=Atts}=Doc, Headers, Options) ->
- case lists:member(attachments, Options) of
- true ->
- AcceptedTypes = case couch_httpd:header_value(Req, "Accept") of
- undefined -> [];
- AcceptHeader -> string:tokens(AcceptHeader, ", ")
- end,
- case lists:member("multipart/related", AcceptedTypes) of
- false ->
- send_json(Req, 200, Headers, couch_doc:to_json_obj(Doc, Options));
- true ->
- Boundary = couch_uuids:random(),
- JsonBytes = ?JSON_ENCODE(couch_doc:to_json_obj(Doc,
- [attachments, follows|Options])),
- {ContentType, Len} = couch_doc:len_doc_to_multi_part_stream(
- Boundary,JsonBytes, Atts,false),
- CType = {<<"Content-Type">>, ContentType},
- {ok, Resp} = start_response_length(Req, 200, [CType|Headers], Len),
- couch_doc:doc_to_multi_part_stream(Boundary,JsonBytes,Atts,
- fun(Data) -> couch_httpd:send(Resp, Data) end, false)
- end;
- false ->
- send_json(Req, 200, Headers, couch_doc:to_json_obj(Doc, Options))
- end.
-
-send_docs_multipart(Req, Results, Options) ->
- OuterBoundary = couch_uuids:random(),
- InnerBoundary = couch_uuids:random(),
- CType = {"Content-Type",
- "multipart/mixed; boundary=\"" ++ ?b2l(OuterBoundary) ++ "\""},
- {ok, Resp} = start_chunked_response(Req, 200, [CType]),
- couch_httpd:send_chunk(Resp, <<"--", OuterBoundary/binary>>),
- lists:foreach(
- fun({ok, #doc{atts=Atts}=Doc}) ->
- JsonBytes = ?JSON_ENCODE(couch_doc:to_json_obj(Doc,
- [attachments,follows|Options])),
- {ContentType, _Len} = couch_doc:len_doc_to_multi_part_stream(
- InnerBoundary, JsonBytes, Atts, false),
- couch_httpd:send_chunk(Resp, <<"\r\nContent-Type: ",
- ContentType/binary, "\r\n\r\n">>),
- couch_doc:doc_to_multi_part_stream(InnerBoundary, JsonBytes, Atts,
- fun(Data) -> couch_httpd:send_chunk(Resp, Data)
- end, false),
- couch_httpd:send_chunk(Resp, <<"\r\n--", OuterBoundary/binary>>);
- ({{not_found, missing}, RevId}) ->
- RevStr = couch_doc:rev_to_str(RevId),
- Json = ?JSON_ENCODE({[{"missing", RevStr}]}),
- couch_httpd:send_chunk(Resp,
- [<<"\r\nContent-Type: application/json; error=\"true\"\r\n\r\n">>,
- Json,
- <<"\r\n--", OuterBoundary/binary>>])
- end, Results),
- couch_httpd:send_chunk(Resp, <<"--">>),
- couch_httpd:last_chunk(Resp).
-
-receive_request_data(Req) ->
- {couch_httpd:recv(Req, 0), fun() -> receive_request_data(Req) end}.
-
-update_doc_result_to_json({{Id, Rev}, Error}) ->
- {_Code, Err, Msg} = chttpd:error_info(Error),
- {[{id, Id}, {rev, couch_doc:rev_to_str(Rev)},
- {error, Err}, {reason, Msg}]}.
-
-update_doc_result_to_json(#doc{id=DocId}, Result) ->
- update_doc_result_to_json(DocId, Result);
-update_doc_result_to_json(DocId, {ok, NewRev}) ->
- {[{id, DocId}, {rev, couch_doc:rev_to_str(NewRev)}]};
-update_doc_result_to_json(DocId, Error) ->
- {_Code, ErrorStr, Reason} = chttpd:error_info(Error),
- {[{id, DocId}, {error, ErrorStr}, {reason, Reason}]}.
-
-
-update_doc(Req, Db, DocId, Json) ->
- update_doc(Req, Db, DocId, Json, []).
-
-update_doc(Req, Db, DocId, Doc, Headers) ->
- update_doc(Req, Db, DocId, Doc, Headers, interactive_edit).
-
-update_doc(#httpd{user_ctx=Ctx} = Req, Db, DocId, #doc{deleted=Deleted}=Doc,
- Headers, UpdateType) ->
- W = couch_httpd:qs_value(Req, "w", couch_config:get("cluster", "w", "2")),
- case couch_httpd:header_value(Req, "X-Couch-Full-Commit") of
- "true" ->
- Options = [full_commit, UpdateType, {user_ctx,Ctx}, {w,W}];
- "false" ->
- Options = [delay_commit, UpdateType, {user_ctx,Ctx}, {w,W}];
- _ ->
- Options = [UpdateType, {user_ctx,Ctx}, {w,W}]
- end,
- {ok, NewRev} = fabric:update_doc(Db, Doc, Options),
- NewRevStr = couch_doc:rev_to_str(NewRev),
- ResponseHeaders = [{"Etag", <<"\"", NewRevStr/binary, "\"">>} | Headers],
- send_json(Req, if Deleted -> 200; true -> 201 end, ResponseHeaders, {[
- {ok, true},
- {id, DocId},
- {rev, NewRevStr}
- ]}).
-
-couch_doc_from_req(Req, DocId, #doc{revs=Revs} = Doc) ->
- validate_attachment_names(Doc),
- ExplicitDocRev =
- case Revs of
- {Start,[RevId|_]} -> {Start, RevId};
- _ -> undefined
- end,
- case extract_header_rev(Req, ExplicitDocRev) of
- missing_rev ->
- Revs2 = {0, []};
- ExplicitDocRev ->
- Revs2 = Revs;
- {Pos, Rev} ->
- Revs2 = {Pos, [Rev]}
- end,
- Doc#doc{id=DocId, revs=Revs2};
-couch_doc_from_req(Req, DocId, Json) ->
- couch_doc_from_req(Req, DocId, couch_doc:from_json_obj(Json)).
-
-
-% Useful for debugging
-% couch_doc_open(Db, DocId) ->
-% couch_doc_open(Db, DocId, nil, []).
-
-couch_doc_open(Db, DocId, Rev, Options) ->
- case Rev of
- nil -> % open most recent rev
- case fabric:open_doc(Db, DocId, Options) of
- {ok, Doc} ->
- Doc;
- Error ->
- throw(Error)
- end;
- _ -> % open a specific rev (deletions come back as stubs)
- case fabric:open_revs(Db, DocId, [Rev], Options) of
- {ok, [{ok, Doc}]} ->
- Doc;
- {ok, [{{not_found, missing}, Rev}]} ->
- throw(not_found);
- {ok, [Else]} ->
- throw(Else)
- end
- end.
-
-% Attachment request handlers
-
-db_attachment_req(#httpd{method='GET'}=Req, Db, DocId, FileNameParts) ->
- FileName = list_to_binary(mochiweb_util:join(lists:map(fun binary_to_list/1,
- FileNameParts),"/")),
- #doc_query_args{
- rev=Rev,
- options=Options
- } = parse_doc_query(Req),
- #doc{
- atts=Atts
- } = Doc = couch_doc_open(Db, DocId, Rev, Options),
- case [A || A <- Atts, A#att.name == FileName] of
- [] ->
- throw({not_found, "Document is missing attachment"});
- [#att{type=Type, encoding=Enc, disk_len=DiskLen, att_len=AttLen}=Att] ->
- Etag = chttpd:doc_etag(Doc),
- ReqAcceptsAttEnc = lists:member(
- atom_to_list(Enc),
- couch_httpd:accepted_encodings(Req)
- ),
- Headers = [
- {"ETag", Etag},
- {"Cache-Control", "must-revalidate"},
- {"Content-Type", binary_to_list(Type)}
- ] ++ case ReqAcceptsAttEnc of
- true ->
- [{"Content-Encoding", atom_to_list(Enc)}];
- _ ->
- []
- end,
- Len = case {Enc, ReqAcceptsAttEnc} of
- {identity, _} ->
- % stored and served in identity form
- DiskLen;
- {_, false} when DiskLen =/= AttLen ->
- % Stored encoded, but client doesn't accept the encoding we used,
- % so we need to decode on the fly. DiskLen is the identity length
- % of the attachment.
- DiskLen;
- {_, true} ->
- % Stored and served encoded. AttLen is the encoded length.
- AttLen;
- _ ->
- % We received an encoded attachment and stored it as such, so we
- % don't know the identity length. The client doesn't accept the
- % encoding, and since we cannot serve a correct Content-Length
- % header we'll fall back to a chunked response.
- undefined
- end,
- AttFun = case ReqAcceptsAttEnc of
- false ->
- fun couch_doc:att_foldl_decode/3;
- true ->
- fun couch_doc:att_foldl/3
- end,
- couch_httpd:etag_respond(
- Req,
- Etag,
- fun() ->
- case Len of
- undefined ->
- {ok, Resp} = start_chunked_response(Req, 200, Headers),
- AttFun(Att, fun(Seg, _) -> send_chunk(Resp, Seg) end, ok),
- couch_httpd:last_chunk(Resp);
- _ ->
- {ok, Resp} = start_response_length(Req, 200, Headers, Len),
- AttFun(Att, fun(Seg, _) -> send(Resp, Seg) end, ok)
- end
- end
- )
- end;
-
-
-db_attachment_req(#httpd{method=Method, user_ctx=Ctx}=Req, Db, DocId, FileNameParts)
- when (Method == 'PUT') or (Method == 'DELETE') ->
- FileName = validate_attachment_name(
- mochiweb_util:join(
- lists:map(fun binary_to_list/1,
- FileNameParts),"/")),
-
- NewAtt = case Method of
- 'DELETE' ->
- [];
- _ ->
- [#att{
- name=FileName,
- type = case couch_httpd:header_value(Req,"Content-Type") of
- undefined ->
- % We could throw an error here or guess by the FileName.
- % Currently, just giving it a default.
- <<"application/octet-stream">>;
- CType ->
- list_to_binary(CType)
- end,
- data = fabric:att_receiver(Req, chttpd:body_length(Req)),
- att_len = case couch_httpd:header_value(Req,"Content-Length") of
- undefined ->
- undefined;
- Length ->
- list_to_integer(Length)
- end,
- md5 = get_md5_header(Req),
- encoding = case string:to_lower(string:strip(
- couch_httpd:header_value(Req,"Content-Encoding","identity")
- )) of
- "identity" ->
- identity;
- "gzip" ->
- gzip;
- _ ->
- throw({
- bad_ctype,
- "Only gzip and identity content-encodings are supported"
- })
- end
- }]
- end,
-
- Doc = case extract_header_rev(Req, couch_httpd:qs_value(Req, "rev")) of
- missing_rev -> % make the new doc
- couch_doc:validate_docid(DocId),
- #doc{id=DocId};
- Rev ->
- case fabric:open_revs(Db, DocId, [Rev], []) of
- {ok, [{ok, Doc0}]} -> Doc0;
- {ok, [Error]} -> throw(Error)
- end
- end,
-
- #doc{atts=Atts} = Doc,
- DocEdited = Doc#doc{
- atts = NewAtt ++ [A || A <- Atts, A#att.name /= FileName]
- },
- {ok, UpdatedRev} = fabric:update_doc(Db, DocEdited, [{user_ctx,Ctx}]),
- #db{name=DbName} = Db,
-
- {Status, Headers} = case Method of
- 'DELETE' ->
- {200, []};
- _ ->
- {201, [{"Location", absolute_uri(Req, [$/, DbName, $/, DocId, $/,
- FileName])}]}
- end,
- send_json(Req,Status, Headers, {[
- {ok, true},
- {id, DocId},
- {rev, couch_doc:rev_to_str(UpdatedRev)}
- ]});
-
-db_attachment_req(Req, _Db, _DocId, _FileNameParts) ->
- send_method_not_allowed(Req, "DELETE,GET,HEAD,PUT").
-
-get_md5_header(Req) ->
- ContentMD5 = couch_httpd:header_value(Req, "Content-MD5"),
- Length = couch_httpd:body_length(Req),
- Trailer = couch_httpd:header_value(Req, "Trailer"),
- case {ContentMD5, Length, Trailer} of
- _ when is_list(ContentMD5) orelse is_binary(ContentMD5) ->
- base64:decode(ContentMD5);
- {_, chunked, undefined} ->
- <<>>;
- {_, chunked, _} ->
- case re:run(Trailer, "\\bContent-MD5\\b", [caseless]) of
- {match, _} ->
- md5_in_footer;
- _ ->
- <<>>
- end;
- _ ->
- <<>>
- end.
-
-parse_doc_query(Req) ->
- lists:foldl(fun({Key,Value}, Args) ->
- case {Key, Value} of
- {"attachments", "true"} ->
- Options = [attachments | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"meta", "true"} ->
- Options = [revs_info, conflicts, deleted_conflicts | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"revs", "true"} ->
- Options = [revs | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"local_seq", "true"} ->
- Options = [local_seq | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"revs_info", "true"} ->
- Options = [revs_info | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"conflicts", "true"} ->
- Options = [conflicts | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"deleted_conflicts", "true"} ->
- Options = [deleted_conflicts | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"rev", Rev} ->
- Args#doc_query_args{rev=couch_doc:parse_rev(Rev)};
- {"open_revs", "all"} ->
- Args#doc_query_args{open_revs=all};
- {"open_revs", RevsJsonStr} ->
- JsonArray = ?JSON_DECODE(RevsJsonStr),
- Args#doc_query_args{open_revs=[couch_doc:parse_rev(Rev) || Rev <- JsonArray]};
- {"atts_since", RevsJsonStr} ->
- JsonArray = ?JSON_DECODE(RevsJsonStr),
- Args#doc_query_args{atts_since = couch_doc:parse_revs(JsonArray)};
- {"new_edits", "false"} ->
- Args#doc_query_args{update_type=replicated_changes};
- {"new_edits", "true"} ->
- Args#doc_query_args{update_type=interactive_edit};
- {"att_encoding_info", "true"} ->
- Options = [att_encoding_info | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"r", R} ->
- Options = [{r,R} | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- {"w", W} ->
- Options = [{w,W} | Args#doc_query_args.options],
- Args#doc_query_args{options=Options};
- _Else -> % unknown key value pair, ignore.
- Args
- end
- end, #doc_query_args{}, chttpd:qs(Req)).
-
-parse_changes_query(Req) ->
- lists:foldl(fun({Key, Value}, Args) ->
- case {Key, Value} of
- {"feed", _} ->
- Args#changes_args{feed=Value};
- {"descending", "true"} ->
- Args#changes_args{dir=rev};
- {"since", _} ->
- Args#changes_args{since=Value};
- {"limit", _} ->
- Args#changes_args{limit=list_to_integer(Value)};
- {"style", _} ->
- Args#changes_args{style=list_to_existing_atom(Value)};
- {"heartbeat", "true"} ->
- Args#changes_args{heartbeat=true};
- {"heartbeat", _} ->
- Args#changes_args{heartbeat=list_to_integer(Value)};
- {"timeout", _} ->
- Args#changes_args{timeout=list_to_integer(Value)};
- {"include_docs", "true"} ->
- Args#changes_args{include_docs=true};
- {"filter", _} ->
- Args#changes_args{filter=Value};
- _Else -> % unknown key value pair, ignore.
- Args
- end
- end, #changes_args{}, couch_httpd:qs(Req)).
-
-extract_header_rev(Req, ExplicitRev) when is_binary(ExplicitRev) or is_list(ExplicitRev)->
- extract_header_rev(Req, couch_doc:parse_rev(ExplicitRev));
-extract_header_rev(Req, ExplicitRev) ->
- Etag = case chttpd:header_value(Req, "If-Match") of
- undefined -> undefined;
- Value -> couch_doc:parse_rev(string:strip(Value, both, $"))
- end,
- case {ExplicitRev, Etag} of
- {undefined, undefined} -> missing_rev;
- {_, undefined} -> ExplicitRev;
- {undefined, _} -> Etag;
- _ when ExplicitRev == Etag -> Etag;
- _ ->
- throw({bad_request, "Document rev and etag have different values"})
- end.
-
-
-parse_copy_destination_header(Req) ->
- Destination = chttpd:header_value(Req, "Destination"),
- case re:run(Destination, "\\?", [{capture, none}]) of
- nomatch ->
- {list_to_binary(Destination), {0, []}};
- match ->
- [DocId, RevQs] = re:split(Destination, "\\?", [{return, list}]),
- [_RevQueryKey, Rev] = re:split(RevQs, "=", [{return, list}]),
- {Pos, RevId} = couch_doc:parse_rev(Rev),
- {list_to_binary(DocId), {Pos, [RevId]}}
- end.
-
-validate_attachment_names(Doc) ->
- lists:foreach(fun(#att{name=Name}) ->
- validate_attachment_name(Name)
- end, Doc#doc.atts).
-
-validate_attachment_name(Name) when is_list(Name) ->
- validate_attachment_name(list_to_binary(Name));
-validate_attachment_name(<<"_",_/binary>>) ->
- throw({bad_request, <<"Attachment name can't start with '_'">>});
-validate_attachment_name(Name) ->
- case is_valid_utf8(Name) of
- true -> Name;
- false -> throw({bad_request, <<"Attachment name is not UTF-8 encoded">>})
- end.
-
-%% borrowed from mochijson2:json_bin_is_safe()
-is_valid_utf8(<<>>) ->
- true;
-is_valid_utf8(<<C, Rest/binary>>) ->
- case C of
- $\" ->
- false;
- $\\ ->
- false;
- $\b ->
- false;
- $\f ->
- false;
- $\n ->
- false;
- $\r ->
- false;
- $\t ->
- false;
- C when C >= 0, C < $\s; C >= 16#7f, C =< 16#10FFFF ->
- false;
- C when C < 16#7f ->
- is_valid_utf8(Rest);
- _ ->
- false
- end.
diff --git a/apps/chttpd/src/chttpd_external.erl b/apps/chttpd/src/chttpd_external.erl
deleted file mode 100644
index 51f32e10..00000000
--- a/apps/chttpd/src/chttpd_external.erl
+++ /dev/null
@@ -1,173 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_external).
-
--export([handle_external_req/2, handle_external_req/3]).
--export([send_external_response/2, json_req_obj/2, json_req_obj/3]).
--export([default_or_content_type/2, parse_external_response/1]).
--export([handle_search_req/2]).
-
--import(chttpd,[send_error/4]).
-
--include_lib("couch/include/couch_db.hrl").
-
-handle_search_req(Req, Db) ->
- process_external_req(Req, Db, <<"search">>).
-
-% handle_external_req/2
-% for the old type of config usage:
-% _external = {chttpd_external, handle_external_req}
-% with urls like
-% /db/_external/action/design/name
-handle_external_req(#httpd{
- path_parts=[_DbName, _External, UrlName | _Path]
- }=HttpReq, Db) ->
- process_external_req(HttpReq, Db, UrlName);
-handle_external_req(#httpd{path_parts=[_, _]}=Req, _Db) ->
- send_error(Req, 404, <<"external_server_error">>, <<"No server name specified.">>);
-handle_external_req(Req, _) ->
- send_error(Req, 404, <<"external_server_error">>, <<"Broken assumption">>).
-
-% handle_external_req/3
-% for this type of config usage:
-% _action = {chttpd_external, handle_external_req, <<"action">>}
-% with urls like
-% /db/_action/design/name
-handle_external_req(HttpReq, Db, Name) ->
- process_external_req(HttpReq, Db, Name).
-
-process_external_req(HttpReq, Db, Name) ->
-
- Response = couch_external_manager:execute(binary_to_list(Name),
- json_req_obj(HttpReq, Db)),
-
- case Response of
- {unknown_external_server, Msg} ->
- send_error(HttpReq, 404, <<"external_server_error">>, Msg);
- _ ->
- send_external_response(HttpReq, Response)
- end.
-
-json_req_obj(Req, Db) -> json_req_obj(Req, Db, null).
-json_req_obj(#httpd{mochi_req=Req,
- method=Method,
- path_parts=Path,
- req_body=ReqBody
- }, Db, DocId) ->
- Body = case ReqBody of
- undefined -> Req:recv_body();
- Else -> Else
- end,
- ParsedForm = case Req:get_primary_header_value("content-type") of
- "application/x-www-form-urlencoded" ++ _ ->
- mochiweb_util:parse_qs(Body);
- _ ->
- []
- end,
- Headers = Req:get(headers),
- Hlist = mochiweb_headers:to_list(Headers),
- {ok, Info} = fabric:get_db_info(Db),
-
- % add headers...
- {[{<<"info">>, {Info}},
- {<<"id">>, DocId},
- {<<"method">>, Method},
- {<<"path">>, Path},
- {<<"query">>, json_query_keys(to_json_terms(Req:parse_qs()))},
- {<<"headers">>, to_json_terms(Hlist)},
- {<<"body">>, Body},
- {<<"peer">>, ?l2b(Req:get(peer))},
- {<<"form">>, to_json_terms(ParsedForm)},
- {<<"cookie">>, to_json_terms(Req:parse_cookie())},
- {<<"userCtx">>, couch_util:json_user_ctx(Db)}]}.
-
-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) ->
- to_json_terms(Rest, [{list_to_binary(atom_to_list(Key)), list_to_binary(Value)} | Acc]);
-to_json_terms([{Key, Value} | Rest], Acc) ->
- to_json_terms(Rest, [{list_to_binary(Key), list_to_binary(Value)} | Acc]).
-
-json_query_keys({Json}) ->
- json_query_keys(Json, []).
-json_query_keys([], Acc) ->
- {lists:reverse(Acc)};
-json_query_keys([{<<"startkey">>, Value} | Rest], Acc) ->
- json_query_keys(Rest, [{<<"startkey">>, ?JSON_DECODE(Value)}|Acc]);
-json_query_keys([{<<"endkey">>, Value} | Rest], Acc) ->
- json_query_keys(Rest, [{<<"endkey">>, ?JSON_DECODE(Value)}|Acc]);
-json_query_keys([{<<"key">>, Value} | Rest], Acc) ->
- json_query_keys(Rest, [{<<"key">>, ?JSON_DECODE(Value)}|Acc]);
-json_query_keys([Term | Rest], Acc) ->
- json_query_keys(Rest, [Term|Acc]).
-
-send_external_response(#httpd{mochi_req=MochiReq}, Response) ->
- #extern_resp_args{
- code = Code,
- data = Data,
- ctype = CType,
- headers = Headers
- } = parse_external_response(Response),
- Resp = MochiReq:respond({Code,
- default_or_content_type(CType, Headers ++ chttpd:server_header()), Data}),
- {ok, Resp}.
-
-parse_external_response({Response}) ->
- lists:foldl(fun({Key,Value}, Args) ->
- case {Key, Value} of
- {"", _} ->
- Args;
- {<<"code">>, Value} ->
- Args#extern_resp_args{code=Value};
- {<<"stop">>, true} ->
- Args#extern_resp_args{stop=true};
- {<<"json">>, Value} ->
- Args#extern_resp_args{
- data=?JSON_ENCODE(Value),
- ctype="application/json"};
- {<<"body">>, Value} ->
- Args#extern_resp_args{data=Value, ctype="text/html; charset=utf-8"};
- {<<"base64">>, Value} ->
- Args#extern_resp_args{
- data=base64:decode(Value),
- ctype="application/binary"
- };
- {<<"headers">>, {Headers}} ->
- NewHeaders = lists:map(fun({Header, HVal}) ->
- {binary_to_list(Header), binary_to_list(HVal)}
- end, Headers),
- Args#extern_resp_args{headers=NewHeaders};
- _ -> % unknown key
- Msg = lists:flatten(io_lib:format("Invalid data from external server: ~p", [{Key, Value}])),
- throw({external_response_error, Msg})
- end
- end, #extern_resp_args{}, Response).
-
-default_or_content_type(DefaultContentType, Headers) ->
- {ContentType, OtherHeaders} = lists:partition(
- fun({HeaderName, _}) ->
- HeaderName == "Content-Type"
- end, Headers),
-
- % XXX: What happens if we were passed multiple content types? We add another?
- case ContentType of
- [{"Content-Type", SetContentType}] ->
- TrueContentType = SetContentType;
- _Else ->
- TrueContentType = DefaultContentType
- end,
-
- HeadersWithContentType = lists:append(OtherHeaders, [{"Content-Type", TrueContentType}]),
- HeadersWithContentType.
diff --git a/apps/chttpd/src/chttpd_misc.erl b/apps/chttpd/src/chttpd_misc.erl
deleted file mode 100644
index 640e7d0d..00000000
--- a/apps/chttpd/src/chttpd_misc.erl
+++ /dev/null
@@ -1,222 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_misc).
-
--export([handle_welcome_req/2,handle_favicon_req/2,handle_utils_dir_req/2,
- handle_all_dbs_req/1,handle_replicate_req/1,handle_restart_req/1,
- handle_uuids_req/1,handle_config_req/1,handle_log_req/1,
- handle_task_status_req/1,handle_sleep_req/1,handle_welcome_req/1,
- handle_utils_dir_req/1, handle_favicon_req/1, handle_system_req/1]).
-
-
--include_lib("couch/include/couch_db.hrl").
-
--import(chttpd,
- [send_json/2,send_json/3,send_json/4,send_method_not_allowed/2,
- start_json_response/2,send_chunk/2,end_json_response/1,
- start_chunked_response/3, send_error/4]).
-
-% httpd global handlers
-
-handle_welcome_req(Req) ->
- handle_welcome_req(Req, <<"Welcome">>).
-
-handle_welcome_req(#httpd{method='GET'}=Req, WelcomeMessage) ->
- send_json(Req, {[
- {couchdb, WelcomeMessage},
- {version, list_to_binary(couch:version())},
- {bigcouch, get_version()}
- ]});
-handle_welcome_req(Req, _) ->
- send_method_not_allowed(Req, "GET,HEAD").
-
-get_version() ->
- Releases = release_handler:which_releases(),
- Version = case [V || {"bigcouch", V, _, current} <- Releases] of
- [] ->
- case [V || {"bigcouch", V, _, permanent} <- Releases] of
- [] ->
- "dev";
- [Permanent] ->
- Permanent
- end;
- [Current] ->
- Current
- end,
- list_to_binary(Version).
-
-handle_favicon_req(Req) ->
- handle_favicon_req(Req, couch_config:get("chttpd", "docroot")).
-
-handle_favicon_req(#httpd{method='GET'}=Req, DocumentRoot) ->
- chttpd:serve_file(Req, "favicon.ico", DocumentRoot);
-handle_favicon_req(Req, _) ->
- send_method_not_allowed(Req, "GET,HEAD").
-
-handle_utils_dir_req(Req) ->
- handle_utils_dir_req(Req, couch_config:get("chttpd", "docroot")).
-
-handle_utils_dir_req(#httpd{method='GET'}=Req, DocumentRoot) ->
- "/" ++ UrlPath = chttpd:path(Req),
- case chttpd:partition(UrlPath) of
- {_ActionKey, "/", RelativePath} ->
- % GET /_utils/path or GET /_utils/
- chttpd:serve_file(Req, RelativePath, DocumentRoot);
- {_ActionKey, "", _RelativePath} ->
- % GET /_utils
- RedirectPath = chttpd:path(Req) ++ "/",
- chttpd:send_redirect(Req, RedirectPath)
- end;
-handle_utils_dir_req(Req, _) ->
- send_method_not_allowed(Req, "GET,HEAD").
-
-handle_sleep_req(#httpd{method='GET'}=Req) ->
- Time = list_to_integer(chttpd:qs_value(Req, "time")),
- receive snicklefart -> ok after Time -> ok end,
- send_json(Req, {[{ok, true}]});
-handle_sleep_req(Req) ->
- send_method_not_allowed(Req, "GET,HEAD").
-
-handle_all_dbs_req(#httpd{method='GET'}=Req) ->
- {ok, DbNames} = fabric:all_dbs(),
- send_json(Req, DbNames);
-handle_all_dbs_req(Req) ->
- send_method_not_allowed(Req, "GET,HEAD").
-
-
-handle_task_status_req(#httpd{method='GET'}=Req) ->
- {Replies, _BadNodes} = gen_server:multi_call(couch_task_status, all),
- Response = lists:flatmap(fun({Node, Tasks}) ->
- [{[{node,Node} | Task]} || Task <- Tasks]
- end, Replies),
- send_json(Req, lists:sort(Response));
-handle_task_status_req(Req) ->
- send_method_not_allowed(Req, "GET,HEAD").
-
-handle_replicate_req(#httpd{method='POST', user_ctx=Ctx} = Req) ->
- PostBody = chttpd:json_body_obj(Req),
- try couch_rep:replicate(PostBody, Ctx) of
- {ok, {continuous, RepId}} ->
- send_json(Req, 202, {[{ok, true}, {<<"_local_id">>, RepId}]});
- {ok, {cancelled, RepId}} ->
- send_json(Req, 200, {[{ok, true}, {<<"_local_id">>, RepId}]});
- {ok, {JsonResults}} ->
- send_json(Req, {[{ok, true} | JsonResults]});
- {error, {Type, Details}} ->
- send_json(Req, 500, {[{error, Type}, {reason, Details}]});
- {error, not_found} ->
- send_json(Req, 404, {[{error, not_found}]});
- {error, Reason} ->
- send_json(Req, 500, {[{error, Reason}]})
- catch
- throw:{db_not_found, Msg} ->
- send_json(Req, 404, {[{error, db_not_found}, {reason, Msg}]});
- throw:{node_not_connected, Msg} ->
- send_json(Req, 404, {[{error, node_not_connected}, {reason, Msg}]})
- end;
-handle_replicate_req(Req) ->
- send_method_not_allowed(Req, "POST").
-
-
-handle_restart_req(#httpd{method='POST'}=Req) ->
- couch_server_sup:restart_core_server(),
- send_json(Req, 200, {[{ok, true}]});
-handle_restart_req(Req) ->
- send_method_not_allowed(Req, "POST").
-
-
-handle_uuids_req(Req) ->
- couch_httpd_misc_handlers:handle_uuids_req(Req).
-
-
-% Config request handler
-
-
-% GET /_config/
-% GET /_config
-handle_config_req(#httpd{method='GET', path_parts=[_]}=Req) ->
- Grouped = lists:foldl(fun({{Section, Key}, Value}, Acc) ->
- case dict:is_key(Section, Acc) of
- true ->
- dict:append(Section, {list_to_binary(Key), list_to_binary(Value)}, Acc);
- false ->
- dict:store(Section, [{list_to_binary(Key), list_to_binary(Value)}], Acc)
- end
- end, dict:new(), couch_config:all()),
- KVs = dict:fold(fun(Section, Values, Acc) ->
- [{list_to_binary(Section), {Values}} | Acc]
- end, [], Grouped),
- send_json(Req, 200, {KVs});
-% GET /_config/Section
-handle_config_req(#httpd{method='GET', path_parts=[_,Section]}=Req) ->
- KVs = [{list_to_binary(Key), list_to_binary(Value)}
- || {Key, Value} <- couch_config:get(Section)],
- send_json(Req, 200, {KVs});
-% PUT /_config/Section/Key
-% "value"
-handle_config_req(#httpd{method='PUT', path_parts=[_, Section, Key]}=Req) ->
- Value = chttpd:json_body(Req),
- Persist = chttpd:header_value(Req, "X-Couch-Persist") /= "false",
- OldValue = couch_config:get(Section, Key, ""),
- ok = couch_config:set(Section, Key, ?b2l(Value), Persist),
- send_json(Req, 200, list_to_binary(OldValue));
-% GET /_config/Section/Key
-handle_config_req(#httpd{method='GET', path_parts=[_, Section, Key]}=Req) ->
- case couch_config:get(Section, Key, null) of
- null ->
- throw({not_found, unknown_config_value});
- Value ->
- send_json(Req, 200, list_to_binary(Value))
- end;
-% DELETE /_config/Section/Key
-handle_config_req(#httpd{method='DELETE',path_parts=[_,Section,Key]}=Req) ->
- Persist = chttpd:header_value(Req, "X-Couch-Persist") /= "false",
- case couch_config:get(Section, Key, null) of
- null ->
- throw({not_found, unknown_config_value});
- OldValue ->
- couch_config:delete(Section, Key, Persist),
- send_json(Req, 200, list_to_binary(OldValue))
- end;
-handle_config_req(Req) ->
- send_method_not_allowed(Req, "GET,PUT,DELETE").
-
-% httpd log handlers
-
-handle_log_req(#httpd{method='GET'}=Req) ->
- Bytes = list_to_integer(chttpd:qs_value(Req, "bytes", "1000")),
- Offset = list_to_integer(chttpd:qs_value(Req, "offset", "0")),
- Chunk = couch_log:read(Bytes, Offset),
- {ok, Resp} = start_chunked_response(Req, 200, [
- % send a plaintext response
- {"Content-Type", "text/plain; charset=utf-8"},
- {"Content-Length", integer_to_list(length(Chunk))}
- ]),
- send_chunk(Resp, Chunk),
- send_chunk(Resp, "");
-handle_log_req(Req) ->
- send_method_not_allowed(Req, "GET").
-
-% Note: this resource is exposed on the backdoor interface, but it's in chttpd
-% because it's not couch trunk
-handle_system_req(Req) ->
- Other = erlang:memory(system) - lists:sum([X || {_,X} <-
- erlang:memory([atom, code, binary, ets])]),
- Memory = [{other, Other} | erlang:memory([atom, atom_used, processes,
- processes_used, binary, code, ets])],
- send_json(Req, {[
- {memory, {Memory}},
- {run_queue, statistics(run_queue)},
- {process_count, erlang:system_info(process_count)},
- {process_limit, erlang:system_info(process_limit)}
- ]}).
diff --git a/apps/chttpd/src/chttpd_oauth.erl b/apps/chttpd/src/chttpd_oauth.erl
deleted file mode 100644
index 84506efe..00000000
--- a/apps/chttpd/src/chttpd_oauth.erl
+++ /dev/null
@@ -1,168 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_oauth).
--include_lib("couch/include/couch_db.hrl").
-
--export([oauth_authentication_handler/1, handle_oauth_req/1, consumer_lookup/2]).
-
-% OAuth auth handler using per-node user db
-oauth_authentication_handler(#httpd{mochi_req=MochiReq}=Req) ->
- serve_oauth(Req, fun(URL, Params, Consumer, Signature) ->
- AccessToken = couch_util:get_value("oauth_token", Params),
- TokenSecret = couch_config:get("oauth_token_secrets", AccessToken),
- case oauth:verify(Signature, atom_to_list(MochiReq:get(method)), URL, Params, Consumer, TokenSecret) of
- true ->
- set_user_ctx(Req, AccessToken);
- false ->
- Req
- end
- end, true).
-
-% Look up the consumer key and get the roles to give the consumer
-set_user_ctx(Req, AccessToken) ->
- DbName = couch_config:get("couch_httpd_auth", "authentication_db", "users"),
- ok = chttpd_auth:ensure_users_db_exists(?l2b(DbName)),
- Name = ?l2b(couch_config:get("oauth_token_users", AccessToken)),
- case chttpd_auth:get_user(Name) of
- nil -> Req;
- User ->
- Roles = couch_util:get_value(<<"roles">>, User, []),
- Req#httpd{user_ctx=#user_ctx{name=Name, roles=Roles}}
- end.
-
-% OAuth request_token
-handle_oauth_req(#httpd{path_parts=[_OAuth, <<"request_token">>], method=Method}=Req) ->
- serve_oauth(Req, fun(URL, Params, Consumer, Signature) ->
- AccessToken = couch_util:get_value("oauth_token", Params),
- TokenSecret = couch_config:get("oauth_token_secrets", AccessToken),
- case oauth:verify(Signature, atom_to_list(Method), URL, Params, Consumer, TokenSecret) of
- true ->
- ok(Req, <<"oauth_token=requestkey&oauth_token_secret=requestsecret">>);
- false ->
- invalid_signature(Req)
- end
- end, false);
-handle_oauth_req(#httpd{path_parts=[_OAuth, <<"authorize">>]}=Req) ->
- {ok, serve_oauth_authorize(Req)};
-handle_oauth_req(#httpd{path_parts=[_OAuth, <<"access_token">>], method='GET'}=Req) ->
- serve_oauth(Req, fun(URL, Params, Consumer, Signature) ->
- case oauth:token(Params) of
- "requestkey" ->
- case oauth:verify(Signature, "GET", URL, Params, Consumer, "requestsecret") of
- true ->
- ok(Req, <<"oauth_token=accesskey&oauth_token_secret=accesssecret">>);
- false ->
- invalid_signature(Req)
- end;
- _ ->
- chttpd:send_error(Req, 400, <<"invalid_token">>, <<"Invalid OAuth token.">>)
- end
- end, false);
-handle_oauth_req(#httpd{path_parts=[_OAuth, <<"access_token">>]}=Req) ->
- chttpd:send_method_not_allowed(Req, "GET").
-
-invalid_signature(Req) ->
- chttpd:send_error(Req, 400, <<"invalid_signature">>, <<"Invalid signature value.">>).
-
-% This needs to be protected i.e. force user to login using HTTP Basic Auth or form-based login.
-serve_oauth_authorize(#httpd{method=Method}=Req) ->
- case Method of
- 'GET' ->
- % Confirm with the User that they want to authenticate the Consumer
- serve_oauth(Req, fun(URL, Params, Consumer, Signature) ->
- AccessToken = couch_util:get_value("oauth_token", Params),
- TokenSecret = couch_config:get("oauth_token_secrets", AccessToken),
- case oauth:verify(Signature, "GET", URL, Params, Consumer, TokenSecret) of
- true ->
- ok(Req, <<"oauth_token=requestkey&oauth_token_secret=requestsecret">>);
- false ->
- invalid_signature(Req)
- end
- end, false);
- 'POST' ->
- % If the User has confirmed, we direct the User back to the Consumer with a verification code
- serve_oauth(Req, fun(URL, Params, Consumer, Signature) ->
- AccessToken = couch_util:get_value("oauth_token", Params),
- TokenSecret = couch_config:get("oauth_token_secrets", AccessToken),
- case oauth:verify(Signature, "POST", URL, Params, Consumer, TokenSecret) of
- true ->
- %redirect(oauth_callback, oauth_token, oauth_verifier),
- ok(Req, <<"oauth_token=requestkey&oauth_token_secret=requestsecret">>);
- false ->
- invalid_signature(Req)
- end
- end, false);
- _ ->
- chttpd:send_method_not_allowed(Req, "GET,POST")
- end.
-
-serve_oauth(#httpd{mochi_req=MochiReq}=Req, Fun, FailSilently) ->
- % 1. In the HTTP Authorization header as defined in OAuth HTTP Authorization Scheme.
- % 2. As the HTTP POST request body with a content-type of application/x-www-form-urlencoded.
- % 3. Added to the URLs in the query part (as defined by [RFC3986] section 3).
- AuthHeader = case MochiReq:get_header_value("authorization") of
- undefined ->
- "";
- Else ->
- [Head | Tail] = re:split(Else, "\\s", [{parts, 2}, {return, list}]),
- case [string:to_lower(Head) | Tail] of
- ["oauth", Rest] -> Rest;
- _ -> ""
- end
- end,
- HeaderParams = oauth_uri:params_from_header_string(AuthHeader),
- %Realm = couch_util:get_value("realm", HeaderParams),
- Params = proplists:delete("realm", HeaderParams) ++ MochiReq:parse_qs(),
- ?LOG_DEBUG("OAuth Params: ~p", [Params]),
- case couch_util:get_value("oauth_version", Params, "1.0") of
- "1.0" ->
- case couch_util:get_value("oauth_consumer_key", Params, undefined) of
- undefined ->
- case FailSilently of
- true -> Req;
- false -> chttpd:send_error(Req, 400, <<"invalid_consumer">>, <<"Invalid consumer.">>)
- end;
- ConsumerKey ->
- SigMethod = couch_util:get_value("oauth_signature_method", Params),
- case consumer_lookup(ConsumerKey, SigMethod) of
- none ->
- chttpd:send_error(Req, 400, <<"invalid_consumer">>, <<"Invalid consumer (key or signature method).">>);
- Consumer ->
- Signature = couch_util:get_value("oauth_signature", Params),
- URL = chttpd:absolute_uri(Req, MochiReq:get(raw_path)),
- Fun(URL, proplists:delete("oauth_signature", Params),
- Consumer, Signature)
- end
- end;
- _ ->
- chttpd:send_error(Req, 400, <<"invalid_oauth_version">>, <<"Invalid OAuth version.">>)
- end.
-
-consumer_lookup(Key, MethodStr) ->
- SignatureMethod = case MethodStr of
- "PLAINTEXT" -> plaintext;
- "HMAC-SHA1" -> hmac_sha1;
- %"RSA-SHA1" -> rsa_sha1;
- _Else -> undefined
- end,
- case SignatureMethod of
- undefined -> none;
- _SupportedMethod ->
- case couch_config:get("oauth_consumer_secrets", Key, undefined) of
- undefined -> none;
- Secret -> {Key, Secret, SignatureMethod}
- end
- end.
-
-ok(#httpd{mochi_req=MochiReq}, Body) ->
- {ok, MochiReq:respond({200, [], Body})}.
diff --git a/apps/chttpd/src/chttpd_rewrite.erl b/apps/chttpd/src/chttpd_rewrite.erl
deleted file mode 100644
index fbf246ab..00000000
--- a/apps/chttpd/src/chttpd_rewrite.erl
+++ /dev/null
@@ -1,418 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-%
-% bind_path is based on bind method from Webmachine
-
-
-%% @doc Module for URL rewriting by pattern matching.
-
--module(chttpd_rewrite).
--export([handle_rewrite_req/3]).
--include_lib("couch/include/couch_db.hrl").
-
--define(SEPARATOR, $\/).
--define(MATCH_ALL, '*').
-
-
-%% doc The http rewrite handler. All rewriting is done from
-%% /dbname/_design/ddocname/_rewrite by default.
-%%
-%% each rules should be in rewrites member of the design doc.
-%% Ex of a complete rule :
-%%
-%% {
-%% ....
-%% "rewrites": [
-%% {
-%% "from": "",
-%% "to": "index.html",
-%% "method": "GET",
-%% "query": {}
-%% }
-%% ]
-%% }
-%%
-%% from: is the path rule used to bind current uri to the rule. It
-%% use pattern matching for that.
-%%
-%% to: rule to rewrite an url. It can contain variables depending on binding
-%% variables discovered during pattern matching and query args (url args and from
-%% the query member.)
-%%
-%% method: method to bind the request method to the rule. by default "*"
-%% query: query args you want to define they can contain dynamic variable
-%% by binding the key to the bindings
-%%
-%%
-%% to and from are path with patterns. pattern can be string starting with ":" or
-%% "*". ex:
-%% /somepath/:var/*
-%%
-%% This path is converted in erlang list by splitting "/". Each var are
-%% converted in atom. "*" is converted to '*' atom. The pattern matching is done
-%% by splitting "/" in request url in a list of token. A string pattern will
-%% match equal token. The star atom ('*' in single quotes) will match any number
-%% of tokens, but may only be present as the last pathtern in a pathspec. If all
-%% tokens are matched and all pathterms are used, then the pathspec matches. It works
-%% like webmachine. Each identified token will be reused in to rule and in query
-%%
-%% The pattern matching is done by first matching the request method to a rule. by
-%% default all methods match a rule. (method is equal to "*" by default). Then
-%% It will try to match the path to one rule. If no rule match, then a 404 error
-%% is displayed.
-%%
-%% Once a rule is found we rewrite the request url using the "to" and
-%% "query" members. The identified token are matched to the rule and
-%% will replace var. if '*' is found in the rule it will contain the remaining
-%% part if it exists.
-%%
-%% Examples:
-%%
-%% Dispatch rule URL TO Tokens
-%%
-%% {"from": "/a/b", /a/b?k=v /some/b?k=v var =:= b
-%% "to": "/some/"} k = v
-%%
-%% {"from": "/a/b", /a/b /some/b?var=b var =:= b
-%% "to": "/some/:var"}
-%%
-%% {"from": "/a", /a /some
-%% "to": "/some/*"}
-%%
-%% {"from": "/a/*", /a/b/c /some/b/c
-%% "to": "/some/*"}
-%%
-%% {"from": "/a", /a /some
-%% "to": "/some/*"}
-%%
-%% {"from": "/a/:foo/*", /a/b/c /some/b/c?foo=b foo =:= b
-%% "to": "/some/:foo/*"}
-%%
-%% {"from": "/a/:foo", /a/b /some/?k=b&foo=b foo =:= b
-%% "to": "/some",
-%% "query": {
-%% "k": ":foo"
-%% }}
-%%
-%% {"from": "/a", /a?foo=b /some/b foo =:= b
-%% "to": "/some/:foo",
-%% }}
-
-
-
-handle_rewrite_req(#httpd{
- path_parts=[DbName, <<"_design">>, DesignName, _Rewrite|PathParts],
- method=Method,
- mochi_req=MochiReq}=Req, _Db, DDoc) ->
-
- % we are in a design handler
- DesignId = <<"_design/", DesignName/binary>>,
- Prefix = <<"/", DbName/binary, "/", DesignId/binary>>,
- QueryList = couch_httpd:qs(Req),
- QueryList1 = [{to_atom(K), V} || {K, V} <- QueryList],
-
- #doc{body={Props}} = DDoc,
-
- % get rules from ddoc
- case couch_util:get_value(<<"rewrites">>, Props) of
- undefined ->
- couch_httpd:send_error(Req, 404, <<"rewrite_error">>,
- <<"Invalid path.">>);
- Rules ->
- % create dispatch list from rules
- DispatchList = [make_rule(Rule) || {Rule} <- Rules],
-
- %% get raw path by matching url to a rule.
- RawPath = case try_bind_path(DispatchList, Method, PathParts,
- QueryList1) of
- no_dispatch_path ->
- throw(not_found);
- {NewPathParts, Bindings} ->
- Parts = [mochiweb_util:quote_plus(X) || X <- NewPathParts],
-
- % build new path, reencode query args, eventually convert
- % them to json
- Path = lists:append(
- string:join(Parts, [?SEPARATOR]),
- case Bindings of
- [] -> [];
- _ -> [$?, encode_query(Bindings)]
- end),
-
- % if path is relative detect it and rewrite path
- case mochiweb_util:safe_relative_path(Path) of
- undefined ->
- ?b2l(Prefix) ++ "/" ++ Path;
- P1 ->
- ?b2l(Prefix) ++ "/" ++ P1
- end
-
- end,
-
- % normalize final path (fix levels "." and "..")
- RawPath1 = ?b2l(iolist_to_binary(normalize_path(RawPath))),
-
- ?LOG_DEBUG("rewrite to ~p ~n", [RawPath1]),
-
- % build a new mochiweb request
- MochiReq1 = mochiweb_request:new(MochiReq:get(socket),
- MochiReq:get(method),
- RawPath1,
- MochiReq:get(version),
- MochiReq:get(headers)),
-
- % cleanup, It force mochiweb to reparse raw uri.
- MochiReq1:cleanup(),
-
- chttpd:handle_request(MochiReq1)
- end.
-
-
-
-%% @doc Try to find a rule matching current url. If none is found
-%% 404 error not_found is raised
-try_bind_path([], _Method, _PathParts, _QueryList) ->
- no_dispatch_path;
-try_bind_path([Dispatch|Rest], Method, PathParts, QueryList) ->
- [{PathParts1, Method1}, RedirectPath, QueryArgs] = Dispatch,
- case bind_method(Method1, Method) of
- true ->
- case bind_path(PathParts1, PathParts, []) of
- {ok, Remaining, Bindings} ->
- Bindings1 = Bindings ++ QueryList,
-
- % we parse query args from the rule and fill
- % it eventually with bindings vars
- QueryArgs1 = make_query_list(QueryArgs, Bindings1, []),
-
- % remove params in QueryLists1 that are already in
- % QueryArgs1
- Bindings2 = lists:foldl(fun({K, V}, Acc) ->
- K1 = to_atom(K),
- KV = case couch_util:get_value(K1, QueryArgs1) of
- undefined -> [{K1, V}];
- _V1 -> []
- end,
- Acc ++ KV
- end, [], Bindings1),
-
- FinalBindings = Bindings2 ++ QueryArgs1,
- NewPathParts = make_new_path(RedirectPath, FinalBindings,
- Remaining, []),
- {NewPathParts, FinalBindings};
- fail ->
- try_bind_path(Rest, Method, PathParts, QueryList)
- end;
- false ->
- try_bind_path(Rest, Method, PathParts, QueryList)
- end.
-
-%% rewriting dynamically the quey list given as query member in
-%% rewrites. Each value is replaced by one binding or an argument
-%% passed in url.
-make_query_list([], _Bindings, Acc) ->
- Acc;
-make_query_list([{Key, {Value}}|Rest], Bindings, Acc) ->
- Value1 = to_json({Value}),
- make_query_list(Rest, Bindings, [{to_atom(Key), Value1}|Acc]);
-make_query_list([{Key, Value}|Rest], Bindings, Acc) when is_binary(Value) ->
- Value1 = replace_var(Key, Value, Bindings),
- make_query_list(Rest, Bindings, [{to_atom(Key), Value1}|Acc]);
-make_query_list([{Key, Value}|Rest], Bindings, Acc) when is_list(Value) ->
- Value1 = replace_var(Key, Value, Bindings),
- make_query_list(Rest, Bindings, [{to_atom(Key), Value1}|Acc]);
-make_query_list([{Key, Value}|Rest], Bindings, Acc) ->
- make_query_list(Rest, Bindings, [{to_atom(Key), Value}|Acc]).
-
-replace_var(Key, Value, Bindings) ->
- case Value of
- <<":", Var/binary>> ->
- get_var(Var, Bindings, Value);
- _ when is_list(Value) ->
- Value1 = lists:foldr(fun(V, Acc) ->
- V1 = case V of
- <<":", VName/binary>> ->
- case get_var(VName, Bindings, V) of
- V2 when is_list(V2) ->
- iolist_to_binary(V2);
- V2 -> V2
- end;
- _ ->
-
- V
- end,
- [V1|Acc]
- end, [], Value),
- to_json(Value1);
- _ when is_binary(Value) ->
- Value;
- _ ->
- case Key of
- <<"key">> -> to_json(Value);
- <<"startkey">> -> to_json(Value);
- <<"endkey">> -> to_json(Value);
- _ ->
- lists:flatten(?JSON_ENCODE(Value))
- end
- end.
-
-
-get_var(VarName, Props, Default) ->
- VarName1 = list_to_atom(binary_to_list(VarName)),
- couch_util:get_value(VarName1, Props, Default).
-
-%% doc: build new patch from bindings. bindings are query args
-%% (+ dynamic query rewritten if needed) and bindings found in
-%% bind_path step.
-make_new_path([], _Bindings, _Remaining, Acc) ->
- lists:reverse(Acc);
-make_new_path([?MATCH_ALL], _Bindings, Remaining, Acc) ->
- Acc1 = lists:reverse(Acc) ++ Remaining,
- Acc1;
-make_new_path([?MATCH_ALL|_Rest], _Bindings, Remaining, Acc) ->
- Acc1 = lists:reverse(Acc) ++ Remaining,
- Acc1;
-make_new_path([P|Rest], Bindings, Remaining, Acc) when is_atom(P) ->
- P2 = case couch_util:get_value(P, Bindings) of
- undefined -> << "undefined">>;
- P1 -> P1
- end,
- make_new_path(Rest, Bindings, Remaining, [P2|Acc]);
-make_new_path([P|Rest], Bindings, Remaining, Acc) ->
- make_new_path(Rest, Bindings, Remaining, [P|Acc]).
-
-
-%% @doc If method of the query fith the rule method. If the
-%% method rule is '*', which is the default, all
-%% request method will bind. It allows us to make rules
-%% depending on HTTP method.
-bind_method(?MATCH_ALL, _Method) ->
- true;
-bind_method(Method, Method) ->
- true;
-bind_method(_, _) ->
- false.
-
-
-%% @doc bind path. Using the rule from we try to bind variables given
-%% to the current url by pattern matching
-bind_path([], [], Bindings) ->
- {ok, [], Bindings};
-bind_path([?MATCH_ALL], Rest, Bindings) when is_list(Rest) ->
- {ok, Rest, Bindings};
-bind_path(_, [], _) ->
- fail;
-bind_path([Token|RestToken],[Match|RestMatch],Bindings) when is_atom(Token) ->
- bind_path(RestToken, RestMatch, [{Token, Match}|Bindings]);
-bind_path([Token|RestToken], [Token|RestMatch], Bindings) ->
- bind_path(RestToken, RestMatch, Bindings);
-bind_path(_, _, _) ->
- fail.
-
-
-%% normalize path.
-normalize_path(Path) ->
- "/" ++ string:join(normalize_path1(string:tokens(Path,
- "/"), []), [?SEPARATOR]).
-
-
-normalize_path1([], Acc) ->
- lists:reverse(Acc);
-normalize_path1([".."|Rest], Acc) ->
- Acc1 = case Acc of
- [] -> [".."|Acc];
- [T|_] when T =:= ".." -> [".."|Acc];
- [_|R] -> R
- end,
- normalize_path1(Rest, Acc1);
-normalize_path1(["."|Rest], Acc) ->
- normalize_path1(Rest, Acc);
-normalize_path1([Path|Rest], Acc) ->
- normalize_path1(Rest, [Path|Acc]).
-
-
-%% @doc transform json rule in erlang for pattern matching
-make_rule(Rule) ->
- Method = case couch_util:get_value(<<"method">>, Rule) of
- undefined -> '*';
- M -> list_to_atom(?b2l(M))
- end,
- QueryArgs = case couch_util:get_value(<<"query">>, Rule) of
- undefined -> [];
- {Args} -> Args
- end,
- FromParts = case couch_util:get_value(<<"from">>, Rule) of
- undefined -> ['*'];
- From ->
- parse_path(From)
- end,
- ToParts = case couch_util:get_value(<<"to">>, Rule) of
- undefined ->
- throw({error, invalid_rewrite_target});
- To ->
- parse_path(To)
- end,
- [{FromParts, Method}, ToParts, QueryArgs].
-
-parse_path(Path) ->
- {ok, SlashRE} = re:compile(<<"\\/">>),
- path_to_list(re:split(Path, SlashRE), [], 0).
-
-%% @doc convert a path rule (from or to) to an erlang list
-%% * and path variable starting by ":" are converted
-%% in erlang atom.
-path_to_list([], Acc, _DotDotCount) ->
- lists:reverse(Acc);
-path_to_list([<<>>|R], Acc, DotDotCount) ->
- path_to_list(R, Acc, DotDotCount);
-path_to_list([<<"*">>|R], Acc, DotDotCount) ->
- path_to_list(R, [?MATCH_ALL|Acc], DotDotCount);
-path_to_list([<<"..">>|R], Acc, DotDotCount) when DotDotCount == 2 ->
- case couch_config:get("httpd", "secure_rewrites", "true") of
- "false" ->
- path_to_list(R, [<<"..">>|Acc], DotDotCount+1);
- _Else ->
- ?LOG_INFO("insecure_rewrite_rule ~p blocked", [lists:reverse(Acc) ++ [<<"..">>] ++ R]),
- throw({insecure_rewrite_rule, "too many ../.. segments"})
- end;
-path_to_list([<<"..">>|R], Acc, DotDotCount) ->
- path_to_list(R, [<<"..">>|Acc], DotDotCount+1);
-path_to_list([P|R], Acc, DotDotCount) ->
- P1 = case P of
- <<":", Var/binary>> ->
- list_to_atom(binary_to_list(Var));
- _ -> P
- end,
- path_to_list(R, [P1|Acc], DotDotCount).
-
-encode_query(Props) ->
- Props1 = lists:foldl(fun ({K, V}, Acc) ->
- V1 = case is_list(V) of
- true -> V;
- false when is_binary(V) ->
- V;
- false ->
- mochiweb_util:quote_plus(V)
- end,
- [{K, V1} | Acc]
- end, [], Props),
- lists:flatten(mochiweb_util:urlencode(Props1)).
-
-to_atom(V) when is_atom(V) ->
- V;
-to_atom(V) when is_binary(V) ->
- to_atom(?b2l(V));
-to_atom(V) ->
- list_to_atom(V).
-
-to_json(V) ->
- iolist_to_binary(?JSON_ENCODE(V)).
diff --git a/apps/chttpd/src/chttpd_show.erl b/apps/chttpd/src/chttpd_show.erl
deleted file mode 100644
index 0aba2835..00000000
--- a/apps/chttpd/src/chttpd_show.erl
+++ /dev/null
@@ -1,311 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_show).
-
--export([handle_doc_show_req/3, handle_doc_update_req/3, handle_view_list_req/3]).
-
--include_lib("couch/include/couch_db.hrl").
-
--import(chttpd,
- [send_json/2,send_json/3,send_json/4,send_method_not_allowed/2,
- start_json_response/2,send_chunk/2,last_chunk/1,send_chunked_error/2,
- start_chunked_response/3, send_error/4]).
-
--record(lacc, {
- req,
- resp = nil,
- qserver,
- lname,
- db,
- etag
-}).
-
-% /db/_design/foo/_show/bar/docid
-% 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.
-
-maybe_open_doc(Db, DocId) ->
- case fabric:open_doc(Db, DocId, [conflicts]) of
- {ok, Doc} ->
- Doc;
- {not_found, _} ->
- nil
- end.
-
-handle_doc_show_req(#httpd{
- path_parts=[_, _, _, _, ShowName, DocId]
- }=Req, Db, DDoc) ->
-
- % open the doc
- Doc = maybe_open_doc(Db, DocId),
-
- % we don't handle revs here b/c they are an internal api
- % returns 404 if there is no doc with DocId
- handle_doc_show(Req, Db, DDoc, ShowName, Doc, DocId);
-
-handle_doc_show_req(#httpd{
- path_parts=[_, _, _, _, ShowName, DocId|Rest]
- }=Req, Db, DDoc) ->
-
- DocParts = [DocId|Rest],
- DocId1 = ?l2b(string:join([?b2l(P)|| P <- DocParts], "/")),
-
- % open the doc
- Doc = maybe_open_doc(Db, DocId1),
-
- % we don't handle revs here b/c they are an internal api
- % pass 404 docs to the show function
- handle_doc_show(Req, Db, DDoc, ShowName, Doc, DocId1);
-
-handle_doc_show_req(#httpd{
- path_parts=[_, _, _, _, ShowName]
- }=Req, Db, DDoc) ->
- % with no docid the doc is nil
- handle_doc_show(Req, Db, DDoc, ShowName, nil);
-
-handle_doc_show_req(Req, _Db, _DDoc) ->
- send_error(Req, 404, <<"show_error">>, <<"Invalid path.">>).
-
-handle_doc_show(Req, Db, DDoc, ShowName, Doc) ->
- handle_doc_show(Req, Db, DDoc, ShowName, Doc, null).
-
-handle_doc_show(Req, Db, DDoc, ShowName, Doc, DocId) ->
- % get responder for ddoc/showname
- CurrentEtag = show_etag(Req, Doc, DDoc, []),
- chttpd:etag_respond(Req, CurrentEtag, fun() ->
- JsonReq = chttpd_external:json_req_obj(Req, Db, DocId),
- JsonDoc = couch_query_servers:json_doc(Doc),
- [<<"resp">>, ExternalResp] =
- couch_query_servers:ddoc_prompt(DDoc, [<<"shows">>, ShowName],
- [JsonDoc, JsonReq]),
- JsonResp = apply_etag(ExternalResp, CurrentEtag),
- chttpd_external:send_external_response(Req, JsonResp)
- end).
-
-
-show_etag(#httpd{user_ctx=UserCtx}=Req, Doc, DDoc, More) ->
- Accept = chttpd:header_value(Req, "Accept"),
- DocPart = case Doc of
- nil -> nil;
- Doc -> chttpd:doc_etag(Doc)
- end,
- couch_httpd:make_etag({couch_httpd:doc_etag(DDoc), DocPart, Accept,
- UserCtx#user_ctx.roles, More}).
-
-% /db/_design/foo/update/bar/docid
-% updates a doc based on a request
-% handle_doc_update_req(#httpd{method = 'GET'}=Req, _Db, _DDoc) ->
-% % anything but GET
-% send_method_not_allowed(Req, "POST,PUT,DELETE,ETC");
-
-handle_doc_update_req(#httpd{
- path_parts=[_, _, _, _, UpdateName, DocId]
- }=Req, Db, DDoc) ->
- Doc = maybe_open_doc(Db, DocId),
- send_doc_update_response(Req, Db, DDoc, UpdateName, Doc, DocId);
-
-handle_doc_update_req(#httpd{
- path_parts=[_, _, _, _, UpdateName]
- }=Req, Db, DDoc) ->
- send_doc_update_response(Req, Db, DDoc, UpdateName, nil, null);
-
-handle_doc_update_req(Req, _Db, _DDoc) ->
- send_error(Req, 404, <<"update_error">>, <<"Invalid path.">>).
-
-send_doc_update_response(Req, Db, DDoc, UpdateName, Doc, DocId) ->
- JsonReq = chttpd_external:json_req_obj(Req, Db, DocId),
- JsonDoc = couch_query_servers:json_doc(Doc),
- Cmd = [<<"updates">>, UpdateName],
- case couch_query_servers:ddoc_prompt(DDoc, Cmd, [JsonDoc, JsonReq]) of
- [<<"up">>, {NewJsonDoc}, JsonResp] ->
- case chttpd:header_value(Req, "X-Couch-Full-Commit", "false") of
- "true" ->
- Options = [full_commit, {user_ctx, Req#httpd.user_ctx}];
- _ ->
- Options = [{user_ctx, Req#httpd.user_ctx}]
- end,
- NewDoc = couch_doc:from_json_obj({NewJsonDoc}),
- Code = 201,
- {ok, _NewRev} = fabric:update_doc(Db, NewDoc, Options);
- [<<"up">>, _Other, JsonResp] ->
- Code = 200
- end,
- JsonResp2 = json_apply_field({<<"code">>, Code}, JsonResp),
- % todo set location field
- chttpd_external:send_external_response(Req, JsonResp2).
-
-
-% view-list request with view and list from same design doc.
-handle_view_list_req(#httpd{method='GET',
- path_parts=[_, _, DesignName, _, ListName, ViewName]}=Req, Db, DDoc) ->
- handle_view_list(Req, Db, DDoc, ListName, {DesignName, ViewName}, nil);
-
-% view-list request with view and list from different design docs.
-handle_view_list_req(#httpd{method='GET',
- path_parts=[_, _, _, _, ListName, DesignName, ViewName]}=Req, Db, DDoc) ->
- handle_view_list(Req, Db, DDoc, ListName, {DesignName, ViewName}, nil);
-
-handle_view_list_req(#httpd{method='GET'}=Req, _Db, _DDoc) ->
- send_error(Req, 404, <<"list_error">>, <<"Invalid path.">>);
-
-handle_view_list_req(#httpd{method='POST',
- path_parts=[_, _, DesignName, _, ListName, ViewName]}=Req, Db, DDoc) ->
- ReqBody = couch_httpd:body(Req),
- {Props2} = ?JSON_DECODE(ReqBody),
- Keys = proplists:get_value(<<"keys">>, Props2, nil),
- handle_view_list(Req#httpd{req_body=ReqBody}, Db, DDoc, ListName,
- {DesignName, ViewName}, Keys);
-
-handle_view_list_req(#httpd{method='POST',
- path_parts=[_, _, _, _, ListName, DesignName, ViewName]}=Req, Db, DDoc) ->
- ReqBody = couch_httpd:body(Req),
- {Props2} = ?JSON_DECODE(ReqBody),
- Keys = proplists:get_value(<<"keys">>, Props2, nil),
- handle_view_list(Req#httpd{req_body=ReqBody}, Db, DDoc, ListName,
- {DesignName, ViewName}, Keys);
-
-handle_view_list_req(#httpd{method='POST'}=Req, _Db, _DDoc) ->
- send_error(Req, 404, <<"list_error">>, <<"Invalid path.">>);
-
-handle_view_list_req(Req, _Db, _DDoc) ->
- send_method_not_allowed(Req, "GET,POST,HEAD").
-
-handle_view_list(Req, Db, DDoc, LName, {ViewDesignName, ViewName}, Keys) ->
- {ok, VDoc} = fabric:open_doc(Db, <<"_design/", ViewDesignName/binary>>, []),
- Group = couch_view_group:design_doc_to_view_group(VDoc),
- IsReduce = chttpd_view:get_reduce_type(Req),
- ViewType = chttpd_view:extract_view_type(ViewName, Group#group.views,
- IsReduce),
- QueryArgs = chttpd_view:parse_view_params(Req, Keys, ViewType),
- CB = fun list_callback/2,
- Etag = couch_uuids:new(),
- chttpd:etag_respond(Req, Etag, fun() ->
- couch_query_servers:with_ddoc_proc(DDoc, fun(QServer) ->
- Acc0 = #lacc{
- lname = LName,
- req = Req,
- qserver = QServer,
- db = Db,
- etag = Etag
- },
- fabric:query_view(Db, VDoc, ViewName, CB, Acc0, QueryArgs)
- end)
- end).
-
-list_callback({total_and_offset, Total, Offset}, #lacc{resp=nil} = Acc) ->
- start_list_resp({[{<<"total_rows">>, Total}, {<<"offset">>, Offset}]}, Acc);
-list_callback({total_and_offset, _, _}, Acc) ->
- % a sorted=false view where the message came in late. Ignore.
- {ok, Acc};
-list_callback({row, Row}, #lacc{resp=nil} = Acc) ->
- % first row of a reduce view, or a sorted=false view
- {ok, NewAcc} = start_list_resp({[]}, Acc),
- send_list_row(Row, NewAcc);
-list_callback({row, Row}, Acc) ->
- send_list_row(Row, Acc);
-list_callback(complete, Acc) ->
- #lacc{qserver = {Proc, _}, resp = Resp0} = Acc,
- if Resp0 =:= nil ->
- {ok, #lacc{resp = Resp}} = start_list_resp({[]}, Acc);
- true ->
- Resp = Resp0
- end,
- [<<"end">>, Chunk] = couch_query_servers:proc_prompt(Proc, [<<"list_end">>]),
- send_non_empty_chunk(Resp, Chunk),
- couch_httpd:last_chunk(Resp),
- {ok, Resp};
-list_callback({error, Reason}, {_, Resp}) ->
- chttpd:send_chunked_error(Resp, {error, Reason}).
-
-start_list_resp(Head, Acc) ->
- #lacc{
- req = Req,
- db = Db,
- qserver = QServer,
- lname = LName,
- etag = Etag
- } = Acc,
-
- % use a separate process because we're already in a receive loop, and
- % json_req_obj calls fabric:get_db_info()
- spawn_monitor(fun() -> exit(chttpd_external:json_req_obj(Req, Db)) end),
- receive {'DOWN', _, _, _, JsonReq} -> ok end,
-
- [<<"start">>,Chunk,JsonResp] = couch_query_servers:ddoc_proc_prompt(QServer,
- [<<"lists">>, LName], [Head, JsonReq]),
- JsonResp2 = apply_etag(JsonResp, Etag),
- #extern_resp_args{
- code = Code,
- ctype = CType,
- headers = ExtHeaders
- } = couch_httpd_external:parse_external_response(JsonResp2),
- JsonHeaders = couch_httpd_external:default_or_content_type(CType, ExtHeaders),
- {ok, Resp} = start_chunked_response(Req, Code, JsonHeaders),
- send_non_empty_chunk(Resp, Chunk),
- {ok, Acc#lacc{resp=Resp}}.
-
-send_list_row(Row, #lacc{qserver = {Proc, _}, resp = Resp} = Acc) ->
- try couch_query_servers:proc_prompt(Proc, [<<"list_row">>, Row]) of
- [<<"chunks">>, Chunk] ->
- send_non_empty_chunk(Resp, Chunk),
- {ok, Acc};
- [<<"end">>, Chunk] ->
- send_non_empty_chunk(Resp, Chunk),
- couch_httpd:last_chunk(Resp),
- {stop, Resp}
- catch Error ->
- chttpd:send_chunked_error(Resp, Error),
- {stop, Resp}
- end.
-
-send_non_empty_chunk(_, []) ->
- ok;
-send_non_empty_chunk(Resp, Chunk) ->
- send_chunk(Resp, Chunk).
-
-% Maybe this is in the proplists API
-% todo move to couch_util
-json_apply_field(H, {L}) ->
- json_apply_field(H, L, []).
-json_apply_field({Key, NewValue}, [{Key, _OldVal} | Headers], Acc) ->
- % drop matching keys
- json_apply_field({Key, NewValue}, Headers, Acc);
-json_apply_field({Key, NewValue}, [{OtherKey, OtherVal} | Headers], Acc) ->
- % something else is next, leave it alone.
- json_apply_field({Key, NewValue}, Headers, [{OtherKey, OtherVal} | Acc]);
-json_apply_field({Key, NewValue}, [], Acc) ->
- % end of list, add ours
- {[{Key, NewValue}|Acc]}.
-
-apply_etag({ExternalResponse}, CurrentEtag) ->
- % Here we embark on the delicate task of replacing or creating the
- % headers on the JsonResponse object. We need to control the Etag and
- % Vary headers. If the external function controls the Etag, we'd have to
- % run it to check for a match, which sort of defeats the purpose.
- case couch_util:get_value(<<"headers">>, ExternalResponse, nil) of
- nil ->
- % no JSON headers
- % add our Etag and Vary headers to the response
- {[{<<"headers">>, {[{<<"Etag">>, CurrentEtag}, {<<"Vary">>, <<"Accept">>}]}} | ExternalResponse]};
- JsonHeaders ->
- {[case Field of
- {<<"headers">>, JsonHeaders} -> % add our headers
- JsonHeadersEtagged = json_apply_field({<<"Etag">>, CurrentEtag}, JsonHeaders),
- JsonHeadersVaried = json_apply_field({<<"Vary">>, <<"Accept">>}, JsonHeadersEtagged),
- {<<"headers">>, JsonHeadersVaried};
- _ -> % skip non-header fields
- Field
- end || Field <- ExternalResponse]}
- end.
-
diff --git a/apps/chttpd/src/chttpd_sup.erl b/apps/chttpd/src/chttpd_sup.erl
deleted file mode 100644
index bfe6be90..00000000
--- a/apps/chttpd/src/chttpd_sup.erl
+++ /dev/null
@@ -1,25 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_sup).
--behaviour(supervisor).
--export([init/1]).
-
--export([start_link/1]).
-
-start_link(Args) ->
- supervisor:start_link({local,?MODULE}, ?MODULE, Args).
-
-init([]) ->
- Mod = chttpd,
- Spec = {Mod, {Mod,start_link,[]}, permanent, 100, worker, [Mod]},
- {ok, {{one_for_one, 3, 10}, [Spec]}}.
diff --git a/apps/chttpd/src/chttpd_view.erl b/apps/chttpd/src/chttpd_view.erl
deleted file mode 100644
index 89f91cb2..00000000
--- a/apps/chttpd/src/chttpd_view.erl
+++ /dev/null
@@ -1,306 +0,0 @@
-% 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. See the
-% License for the specific language governing permissions and limitations under
-% the License.
-
--module(chttpd_view).
--include_lib("couch/include/couch_db.hrl").
-
--export([handle_view_req/3, handle_temp_view_req/2, get_reduce_type/1,
- parse_view_params/3, view_group_etag/2, view_group_etag/3,
- parse_bool_param/1, extract_view_type/3]).
-
--import(chttpd,
- [send_json/2,send_json/3,send_json/4,send_method_not_allowed/2,send_chunk/2,
- start_json_response/2, start_json_response/3, end_json_response/1,
- send_chunked_error/2]).
-
-design_doc_view(Req, Db, DDoc, ViewName, Keys) ->
- Group = couch_view_group:design_doc_to_view_group(DDoc),
- IsReduce = get_reduce_type(Req),
- ViewType = extract_view_type(ViewName, Group#group.views, IsReduce),
- QueryArgs = parse_view_params(Req, Keys, ViewType),
- % TODO proper calculation of etag
- % Etag = view_group_etag(ViewGroup, Db, Keys),
- Etag = couch_uuids:new(),
- couch_stats_collector:increment({httpd, view_reads}),
- chttpd:etag_respond(Req, Etag, fun() ->
- {ok, Resp} = chttpd:start_json_response(Req, 200, [{"Etag",Etag}]),
- CB = fun view_callback/2,
- fabric:query_view(Db, DDoc, ViewName, CB, {nil, Resp}, QueryArgs)
- end).
-
-view_callback({total_and_offset, Total, Offset}, {nil, Resp}) ->
- Chunk = "{\"total_rows\":~p,\"offset\":~p,\"rows\":[\r\n",
- send_chunk(Resp, io_lib:format(Chunk, [Total, Offset])),
- {ok, {"", Resp}};
-view_callback({total_and_offset, _, _}, Acc) ->
- % a sorted=false view where the message came in late. Ignore.
- {ok, Acc};
-view_callback({row, Row}, {nil, Resp}) ->
- % first row of a reduce view, or a sorted=false view
- send_chunk(Resp, ["{\"rows\":[\r\n", ?JSON_ENCODE(Row)]),
- {ok, {",\r\n", Resp}};
-view_callback({row, Row}, {Prepend, Resp}) ->
- send_chunk(Resp, [Prepend, ?JSON_ENCODE(Row)]),
- {ok, {",\r\n", Resp}};
-view_callback(complete, {nil, Resp}) ->
- send_chunk(Resp, "{\"rows\":[]}"),
- end_json_response(Resp),
- {ok, Resp};
-view_callback(complete, {_, Resp}) ->
- send_chunk(Resp, "\r\n]}"),
- end_json_response(Resp),
- {ok, Resp};
-view_callback({error, Reason}, {_, Resp}) ->
- chttpd:send_chunked_error(Resp, {error, Reason}).
-
-extract_view_type(_ViewName, [], _IsReduce) ->
- throw({not_found, missing_named_view});
-extract_view_type(ViewName, [View|Rest], IsReduce) ->
- case lists:member(ViewName, [Name || {Name, _} <- View#view.reduce_funs]) of
- true ->
- if IsReduce -> reduce; true -> red_map end;
- false ->
- case lists:member(ViewName, View#view.map_names) of
- true -> map;
- false -> extract_view_type(ViewName, Rest, IsReduce)
- end
- end.
-
-handle_view_req(#httpd{method='GET',
- path_parts=[_, _, _, _, ViewName]}=Req, Db, DDoc) ->
- design_doc_view(Req, Db, DDoc, ViewName, nil);
-
-handle_view_req(#httpd{method='POST',
- path_parts=[_, _, _, _, ViewName]}=Req, Db, DDoc) ->
- {Fields} = chttpd:json_body_obj(Req),
- case couch_util:get_value(<<"keys">>, Fields) of
- Keys when is_list(Keys) ->
- design_doc_view(Req, Db, DDoc, ViewName, Keys);
- _ ->
- throw({bad_request, "`keys` body member must be an array."})
- end;
-
-handle_view_req(Req, _Db, _DDoc) ->
- send_method_not_allowed(Req, "GET,POST,HEAD").
-
-handle_temp_view_req(Req, _Db) ->
- Msg = <<"Temporary views are not supported in BigCouch">>,
- chttpd:send_error(Req, 403, forbidden, Msg).
-
-reverse_key_default(?MIN_STR) -> ?MAX_STR;
-reverse_key_default(?MAX_STR) -> ?MIN_STR;
-reverse_key_default(Key) -> Key.
-
-get_reduce_type(Req) ->
- list_to_atom(chttpd:qs_value(Req, "reduce", "true")).
-
-parse_view_params(Req, Keys, ViewType) ->
- QueryList = chttpd:qs(Req),
- QueryParams =
- lists:foldl(fun({K, V}, Acc) ->
- parse_view_param(K, V) ++ Acc
- end, [], QueryList),
- IsMultiGet = case Keys of
- nil -> false;
- _ -> true
- end,
- Args = #view_query_args{
- view_type=ViewType,
- multi_get=IsMultiGet,
- keys=Keys
- },
- QueryArgs = lists:foldl(fun({K, V}, Args2) ->
- validate_view_query(K, V, Args2)
- end, Args, lists:reverse(QueryParams)), % Reverse to match QS order.
-
- GroupLevel = QueryArgs#view_query_args.group_level,
- case {ViewType, GroupLevel, IsMultiGet} of
- {reduce, exact, true} ->
- QueryArgs;
- {reduce, _, false} ->
- QueryArgs;
- {reduce, _, _} ->
- Msg = <<"Multi-key fetchs for reduce "
- "view must include `group=true`">>,
- throw({query_parse_error, Msg});
- _ ->
- QueryArgs
- end,
- QueryArgs.
-
-parse_view_param("", _) ->
- [];
-parse_view_param("key", Value) ->
- JsonKey = ?JSON_DECODE(Value),
- [{start_key, JsonKey}, {end_key, JsonKey}];
-parse_view_param("startkey_docid", Value) ->
- [{start_docid, ?l2b(Value)}];
-parse_view_param("endkey_docid", Value) ->
- [{end_docid, ?l2b(Value)}];
-parse_view_param("startkey", Value) ->
- [{start_key, ?JSON_DECODE(Value)}];
-parse_view_param("endkey", Value) ->
- [{end_key, ?JSON_DECODE(Value)}];
-parse_view_param("limit", Value) ->
- [{limit, parse_positive_int_param(Value)}];
-parse_view_param("count", _Value) ->
- throw({query_parse_error, <<"Query parameter 'count' is now 'limit'.">>});
-parse_view_param("stale", "ok") ->
- [{stale, ok}];
-parse_view_param("stale", _Value) ->
- throw({query_parse_error, <<"stale only available as stale=ok">>});
-parse_view_param("update", _Value) ->
- throw({query_parse_error, <<"update=false is now stale=ok">>});
-parse_view_param("descending", Value) ->
- [{descending, parse_bool_param(Value)}];
-parse_view_param("skip", Value) ->
- [{skip, parse_int_param(Value)}];
-parse_view_param("group", Value) ->
- case parse_bool_param(Value) of
- true -> [{group_level, exact}];
- false -> [{group_level, 0}]
- end;
-parse_view_param("group_level", Value) ->
- [{group_level, parse_positive_int_param(Value)}];
-parse_view_param("inclusive_end", Value) ->
- [{inclusive_end, parse_bool_param(Value)}];
-parse_view_param("reduce", Value) ->
- [{reduce, parse_bool_param(Value)}];
-parse_view_param("include_docs", Value) ->
- [{include_docs, parse_bool_param(Value)}];
-parse_view_param("list", Value) ->
- [{list, ?l2b(Value)}];
-parse_view_param("callback", _) ->
- []; % Verified in the JSON response functions
-parse_view_param("show_total_rows", Value) ->
- [{show_total_rows, parse_bool_param(Value)}];
-parse_view_param("sorted", Value) ->
- [{sorted, parse_bool_param(Value)}];
-parse_view_param(Key, Value) ->
- [{extra, {Key, Value}}].
-
-validate_view_query(start_key, Value, Args) ->
- case Args#view_query_args.multi_get of
- true ->
- Msg = <<"Query parameter `start_key` is "
- "not compatiible with multi-get">>,
- throw({query_parse_error, Msg});
- _ ->
- Args#view_query_args{start_key=Value}
- end;
-validate_view_query(start_docid, Value, Args) ->
- Args#view_query_args{start_docid=Value};
-validate_view_query(end_key, Value, Args) ->
- case Args#view_query_args.multi_get of
- true->
- Msg = <<"Query paramter `end_key` is "
- "not compatibile with multi-get">>,
- throw({query_parse_error, Msg});
- _ ->
- Args#view_query_args{end_key=Value}
- end;
-validate_view_query(end_docid, Value, Args) ->
- Args#view_query_args{end_docid=Value};
-validate_view_query(limit, Value, Args) ->
- Args#view_query_args{limit=Value};
-validate_view_query(list, Value, Args) ->
- Args#view_query_args{list=Value};
-validate_view_query(stale, Value, Args) ->
- Args#view_query_args{stale=Value};
-validate_view_query(descending, true, Args) ->
- case Args#view_query_args.direction of
- rev -> Args; % Already reversed
- fwd ->
- Args#view_query_args{
- direction = rev,
- start_docid =
- reverse_key_default(Args#view_query_args.start_docid),
- end_docid =
- reverse_key_default(Args#view_query_args.end_docid)
- }
- end;
-validate_view_query(descending, false, Args) ->
- Args; % Ignore default condition
-validate_view_query(skip, Value, Args) ->
- Args#view_query_args{skip=Value};
-validate_view_query(group_level, Value, Args) ->
- case Args#view_query_args.view_type of
- reduce ->
- Args#view_query_args{group_level=Value};
- _ ->
- Msg = <<"Invalid URL parameter 'group' or "
- " 'group_level' for non-reduce view.">>,
- throw({query_parse_error, Msg})
- end;
-validate_view_query(inclusive_end, Value, Args) ->
- Args#view_query_args{inclusive_end=Value};
-validate_view_query(reduce, _, Args) ->
- case Args#view_query_args.view_type of
- map ->
- Msg = <<"Invalid URL parameter `reduce` for map view.">>,
- throw({query_parse_error, Msg});
- _ ->
- Args
- end;
-validate_view_query(include_docs, true, Args) ->
- case Args#view_query_args.view_type of
- reduce ->
- Msg = <<"Query paramter `include_docs` "
- "is invalid for reduce views.">>,
- throw({query_parse_error, Msg});
- _ ->
- Args#view_query_args{include_docs=true}
- end;
-validate_view_query(include_docs, _Value, Args) ->
- Args;
-validate_view_query(sorted, false, Args) ->
- Args#view_query_args{sorted=false};
-validate_view_query(sorted, _Value, Args) ->
- Args;
-validate_view_query(extra, _Value, Args) ->
- Args.
-
-view_group_etag(Group, Db) ->
- view_group_etag(Group, Db, nil).
-
-view_group_etag(#group{sig=Sig,current_seq=CurrentSeq}, _Db, Extra) ->
- % ?LOG_ERROR("Group ~p",[Group]),
- % This is not as granular as it could be.
- % If there are updates to the db that do not effect the view index,
- % they will change the Etag. For more granular Etags we'd need to keep
- % track of the last Db seq that caused an index change.
- chttpd:make_etag({Sig, CurrentSeq, Extra}).
-
-parse_bool_param("true") -> true;
-parse_bool_param("false") -> false;
-parse_bool_param(Val) ->
- Msg = io_lib:format("Invalid value for boolean paramter: ~p", [Val]),
- throw({query_parse_error, ?l2b(Msg)}).
-
-parse_int_param(Val) ->
- case (catch list_to_integer(Val)) of
- IntVal when is_integer(IntVal) ->
- IntVal;
- _ ->
- Msg = io_lib:format("Invalid value for integer parameter: ~p", [Val]),
- throw({query_parse_error, ?l2b(Msg)})
- end.
-
-parse_positive_int_param(Val) ->
- case parse_int_param(Val) of
- IntVal when IntVal >= 0 ->
- IntVal;
- _ ->
- Fmt = "Invalid value for positive integer parameter: ~p",
- Msg = io_lib:format(Fmt, [Val]),
- throw({query_parse_error, ?l2b(Msg)})
- end.