diff options
Diffstat (limited to 'deps/chttpd')
-rw-r--r-- | deps/chttpd/README.md | 16 | ||||
-rwxr-xr-x | deps/chttpd/rebar | bin | 0 -> 100732 bytes | |||
-rw-r--r-- | deps/chttpd/src/chttpd.app.src | 7 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd.erl | 782 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd_app.erl | 21 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd_db.erl | 1264 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd_external.erl | 174 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd_misc.erl | 283 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd_rewrite.erl | 421 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd_show.erl | 314 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd_sup.erl | 25 | ||||
-rw-r--r-- | deps/chttpd/src/chttpd_view.erl | 393 | ||||
-rw-r--r-- | deps/chttpd/test/chttpd_delayed_response_test.erl | 43 | ||||
-rw-r--r-- | deps/chttpd/test/mock_request.erl | 39 |
14 files changed, 3782 insertions, 0 deletions
diff --git a/deps/chttpd/README.md b/deps/chttpd/README.md new file mode 100644 index 00000000..784bedd7 --- /dev/null +++ b/deps/chttpd/README.md @@ -0,0 +1,16 @@ +## chttpd + +chttpd is a cluster-aware http layer for [CouchDB][1]. It is used in [BigCouch][2] as the http front-end. + +### License +[Apache 2.0][3] + +### Contact + * [http://cloudant.com][4] + * [info@cloudant.com][5] + +[1]: http://couchdb.apache.org +[2]: http://github.com/cloudant/bigcouch +[3]: http://www.apache.org/licenses/LICENSE-2.0.html +[4]: http://cloudant.com +[5]: mailto:info@cloudant.com diff --git a/deps/chttpd/rebar b/deps/chttpd/rebar Binary files differnew file mode 100755 index 00000000..30c43ba5 --- /dev/null +++ b/deps/chttpd/rebar diff --git a/deps/chttpd/src/chttpd.app.src b/deps/chttpd/src/chttpd.app.src new file mode 100644 index 00000000..6897076c --- /dev/null +++ b/deps/chttpd/src/chttpd.app.src @@ -0,0 +1,7 @@ +{application, chttpd, [ + {description, "HTTP interface for CouchDB cluster"}, + {vsn, git}, + {registered, [chttpd_sup, chttpd]}, + {applications, [kernel, stdlib, couch, fabric]}, + {mod, {chttpd_app,[]}} +]}.
\ No newline at end of file diff --git a/deps/chttpd/src/chttpd.erl b/deps/chttpd/src/chttpd.erl new file mode 100644 index 00000000..a4f053aa --- /dev/null +++ b/deps/chttpd/src/chttpd.erl @@ -0,0 +1,782 @@ +% 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, start_link/1, start_link/2, + 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]). + +-export([start_delayed_json_response/2, start_delayed_json_response/3, + start_delayed_json_response/4, + start_delayed_chunked_response/3, start_delayed_chunked_response/4, + send_delayed_chunk/2, send_delayed_last_chunk/1, + send_delayed_error/2, end_delayed_json_response/1, + get_delayed_req/1]). + +-record(delayed_resp, { + start_fun, + req, + code, + headers, + first_chunk +}). + +start_link() -> + start_link(http). +start_link(http) -> + Port = couch_config:get("chttpd", "port", "5984"), + start_link(?MODULE, [{port, Port}]); + +start_link(https) -> + Port = couch_config:get("chttps", "port", "6984"), + CertFile = couch_config:get("chttps", "cert_file", nil), + KeyFile = couch_config:get("chttps", "key_file", nil), + Options = case CertFile /= nil andalso KeyFile /= nil of + true -> + SslOpts = [{certfile, CertFile}, {keyfile, KeyFile}], + + %% set password if one is needed for the cert + SslOpts1 = case couch_config:get("chttps", "password", nil) of + nil -> SslOpts; + Password -> + SslOpts ++ [{password, Password}] + end, + % do we verify certificates ? + FinalSslOpts = case couch_config:get("chttps", + "verify_ssl_certificates", "false") of + "false" -> SslOpts1; + "true" -> + case couch_config:get("chttps", + "cacert_file", nil) of + nil -> + io:format("Verify SSL certificate " + ++"enabled but file containing " + ++"PEM encoded CA certificates is " + ++"missing", []), + throw({error, missing_cacerts}); + CaCertFile -> + Depth = list_to_integer(couch_config:get("chttps", + "ssl_certificate_max_depth", + "1")), + FinalOpts = [ + {cacertfile, CaCertFile}, + {depth, Depth}, + {verify, verify_peer}], + % allows custom verify fun. + case couch_config:get("chttps", + "verify_fun", nil) of + nil -> FinalOpts; + SpecStr -> + FinalOpts + ++ [{verify_fun, couch_httpd:make_arity_3_fun(SpecStr)}] + end + end + end, + + [{port, Port}, + {ssl, true}, + {ssl_opts, FinalSslOpts}]; + false -> + io:format("SSL enabled but PEM certificates are missing.", []), + throw({error, missing_certs}) + end, + start_link(https, Options). + +start_link(Name, Options) -> + Options1 = Options ++ [ + {loop, fun ?MODULE:handle_request/1}, + {name, Name}, + {ip, couch_config:get("chttpd", "bind_address", any)} + ], + ServerOptsCfg = couch_config:get("chttpd", "server_options", "[]"), + {ok, ServerOpts} = couch_util:parse_term(ServerOptsCfg), + Options2 = lists:keymerge(1, lists:sort(Options1), lists:sort(ServerOpts)), + case mochiweb_http:start(Options2) 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(); +config_change("chttpd", "server_options") -> + ?MODULE:stop(). + +stop() -> + catch mochiweb_http:stop(https), + mochiweb_http:stop(?MODULE). + +handle_request(MochiReq) -> + Begin = now(), + + case couch_config:get("chttpd", "socket_options") of + undefined -> + ok; + SocketOptsCfg -> + {ok, SocketOpts} = couch_util:parse_term(SocketOptsCfg), + ok = mochiweb_socket:setopts(MochiReq:get(socket), SocketOpts) + end, + + AuthenticationFuns = [ + fun couch_httpd_auth:cookie_authentication_handler/1, + fun couch_httpd_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, "/"), + + Peer = MochiReq:get(peer), + LogForClosedSocket = io_lib:format("mochiweb_recv_error for ~s - ~p ~s", [ + 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() + }, + + % put small token on heap to keep requests synced to backend calls + erlang:put(nonce, couch_util:to_hex(crypto:rand_bytes(4))), + + Result = + try + case authenticate_request(HttpReq, AuthenticationFuns) of + #httpd{} = Req -> + HandlerFun = url_handler(HandlerKey), + HandlerFun(possibly_hack(Req)); + Response -> + Response + end + catch + throw:{http_head_abort, Resp0} -> + {ok, Resp0}; + throw:{http_abort, Resp0, Reason0} -> + {aborted, Resp0, Reason0}; + 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); + Tag:Error -> + Stack = erlang:get_stacktrace(), + ?LOG_ERROR("Uncaught error in HTTP request: ~p",[{Tag, Error}]), + ?LOG_INFO("Stacktrace: ~p",[Stack]), + send_error(HttpReq, {Error, nil, Stack}) + end, + + RequestTime = timer:now_diff(now(), Begin)/1000, + {Status, Code} = case Result of + {ok, Resp} -> + {ok, Resp:get(code)}; + {aborted, Resp, _} -> + {aborted, Resp:get(code)} + end, + Host = MochiReq:get_header_value("Host"), + ?LOG_INFO("~s ~s ~s ~s ~B ~p ~B", [Peer, Host, + atom_to_list(Method1), RawUri, Code, Status, round(RequestTime)]), + couch_stats_collector:record({couchdb, request_time}, RequestTime), + case Result of + {ok, _} -> + couch_stats_collector:increment({httpd, requests}), + {ok, Resp}; + {aborted, _, Reason} -> + couch_stats_collector:increment({httpd, aborted_requests}), + ?LOG_ERROR("Response abnormally terminated: ~p", [Reason]), + exit(normal) + end. + +%% HACK: replication currently handles two forms of input, #db{} style +%% and #http_db style. We need a third that makes use of fabric. #db{} +%% works fine for replicating the dbs and nodes database because they +%% aren't sharded. So for now when a local db is specified as the source or +%% the target, it's hacked to make it a full url and treated as a remote. +possibly_hack(#httpd{path_parts=[<<"_replicate">>]}=Req) -> + {Props0} = couch_httpd:json_body_obj(Req), + Props1 = fix_uri(Req, Props0, <<"source">>), + Props2 = fix_uri(Req, Props1, <<"target">>), + put(post_body, {Props2}), + Req; +possibly_hack(Req) -> + Req. + +fix_uri(Req, Props, Type) -> + case is_http(replication_uri(Type, Props)) of + true -> + Props; + false -> + Uri = make_uri(Req,replication_uri(Type, Props)), + [{Type,Uri}|proplists:delete(Type,Props)] + end. + +replication_uri(Type, PostProps) -> + case couch_util:get_value(Type, PostProps) of + {Props} -> + couch_util:get_value(<<"url">>, Props); + Else -> + Else + end. + +is_http(<<"http://", _/binary>>) -> + true; +is_http(<<"https://", _/binary>>) -> + true; +is_http(_) -> + false. + +make_uri(Req, Raw) -> + Url = list_to_binary(["http://", couch_config:get("httpd", "bind_address"), + ":", couch_config:get("chttpd", "port"), "/", Raw]), + Headers = [ + {<<"authorization">>, ?l2b(header_value(Req,"authorization",""))}, + {<<"cookie">>, ?l2b(header_value(Req,"cookie",""))} + ], + {[{<<"url">>,Url}, {<<"headers">>,{Headers}}]}. +%%% end hack + + +% 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 couch_httpd_auth:handle_session_req/1; +url_handler("_oauth") -> fun couch_httpd_oauth:handle_oauth_req/1; +%% showroom_http module missing in bigcouch +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() ++ couch_httpd_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"; + _ -> + case MochiReq:get(scheme) of + https -> + "https"; + http -> + "http" + end + 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(Req) -> + couch_httpd:body(Req). + +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() ++ + couch_httpd_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() ++ + couch_httpd_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() ++ + couch_httpd_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) -> + couch_httpd:send_json(Req, Code, [reqid() | Headers], Value). + +start_json_response(Req, Code) -> + start_json_response(Req, Code, []). + +start_json_response(Req, Code, Headers) -> + couch_httpd:start_json_response(Req, Code, [reqid() | Headers]). + +end_json_response(Resp) -> + couch_httpd:end_json_response(Resp). + +start_delayed_json_response(Req, Code) -> + start_delayed_json_response(Req, Code, []). + +start_delayed_json_response(Req, Code, Headers) -> + start_delayed_json_response(Req, Code, Headers, ""). + +start_delayed_json_response(Req, Code, Headers, FirstChunk) -> + {ok, #delayed_resp{ + start_fun = fun start_json_response/3, + req = Req, + code = Code, + headers = Headers, + first_chunk = FirstChunk}}. + +start_delayed_chunked_response(Req, Code, Headers) -> + start_delayed_chunked_response(Req, Code, Headers, ""). + +start_delayed_chunked_response(Req, Code, Headers, FirstChunk) -> + {ok, #delayed_resp{ + start_fun = fun start_chunked_response/3, + req = Req, + code = Code, + headers = Headers, + first_chunk = FirstChunk}}. + +send_delayed_chunk(Resp, Chunk) -> + {ok, Resp1} = start_delayed_response(Resp), + send_chunk(Resp1, Chunk). + +send_delayed_last_chunk(Req) -> + send_delayed_chunk(Req, []). + +send_delayed_error(#httpd{}=Req, Reason) -> + {Code, ErrorStr, ReasonStr} = error_info(Reason), + send_error(Req, Code, ErrorStr, ReasonStr); +send_delayed_error(#delayed_resp{req=Req}, Reason) -> + {Code, ErrorStr, ReasonStr} = error_info(Reason), + send_error(Req, Code, ErrorStr, ReasonStr); +send_delayed_error(Resp, Reason) -> + throw({http_abort, Resp, Reason}). + +end_delayed_json_response(Resp) -> + {ok, Resp1} = start_delayed_response(Resp), + end_json_response(Resp1). + +get_delayed_req(#delayed_resp{req=#httpd{mochi_req=MochiReq}}) -> + MochiReq; +get_delayed_req(Resp) -> + Resp:get(request). + +start_delayed_response(#delayed_resp{start_fun=StartFun, req=Req, code=Code, + headers=Headers, first_chunk=FirstChunk}) -> + {ok, Resp} = StartFun(Req, Code, Headers), + case FirstChunk of + "" -> {ok, Resp}; + _ -> send_chunk(Resp, FirstChunk) + end; +start_delayed_response(Resp) -> + {ok, Resp}. + +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(requested_range_not_satisfiable) -> + {416, <<"requested_range_not_satisfiable">>, <<"Requested range not satisfiable">>}; +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, null}) -> + {500, couch_util:to_binary(Error), null}; +error_info({Error, Reason}) -> + {500, couch_util:to_binary(Error), couch_util:to_binary(Reason)}; +error_info({Error, nil, _Stack}) -> + error_info(Error); +error_info({Error, Reason, _Stack}) -> + error_info({Error, Reason}); +error_info(Error) -> + {500, couch_util:to_binary(Error), null}. + +error_headers(#httpd{mochi_req=MochiReq}=Req, 401=Code, ErrorStr, ReasonStr) -> + % this is where the basic auth popup is triggered + case MochiReq:get_header_value("X-CouchDB-WWW-Authenticate") of + undefined -> + case couch_config:get("httpd", "WWW-Authenticate", nil) of + nil -> + % If the client is a browser and the basic auth popup isn't turned on + % redirect to the session page. + case ErrorStr of + <<"unauthorized">> -> + case couch_config:get("couch_httpd_auth", "authentication_redirect", nil) of + nil -> {Code, []}; + AuthRedirect -> + case couch_config:get("couch_httpd_auth", "require_valid_user", "false") of + "true" -> + % send the browser popup header no matter what if we are require_valid_user + {Code, [{"WWW-Authenticate", "Basic realm=\"server\""}]}; + _False -> + % if the accept header matches html, then do the redirect. else proceed as usual. + Accepts = case MochiReq:get_header_value("Accept") of + undefined -> + % According to the HTTP 1.1 spec, if the Accept + % header is missing, it means the client accepts + % all media types. + "html"; + Else -> + Else + end, + case re:run(Accepts, "\\bhtml\\b", + [{capture, none}, caseless]) of + nomatch -> + {Code, []}; + match -> + AuthRedirectBin = ?l2b(AuthRedirect), + % Redirect to the path the user requested, not + % the one that is used internally. + UrlReturnRaw = case MochiReq:get_header_value("x-couchdb-vhost-path") of + undefined -> MochiReq:get(path); + VHostPath -> VHostPath + end, + UrlReturn = ?l2b(couch_util:url_encode(UrlReturnRaw)), + UrlReason = ?l2b(couch_util:url_encode(ReasonStr)), + {302, [{"Location", couch_httpd:absolute_uri(Req, <<AuthRedirectBin/binary,"?return=",UrlReturn/binary,"&reason=",UrlReason/binary>>)}]} + end + end + end; + _Else -> + {Code, []} + end; + Type -> + {Code, [{"WWW-Authenticate", Type}]} + end; + Type -> + {Code, [{"WWW-Authenticate", Type}]} + end; +error_headers(_, Code, _, _) -> + {Code, []}. + +send_error(_Req, {already_sent, Resp, _Error}) -> + {ok, Resp}; + +send_error(Req, Error) -> + {Code, ErrorStr, ReasonStr} = error_info(Error), + {Code1, Headers} = error_headers(Req, Code, ErrorStr, ReasonStr), + send_error(Req, Code1, Headers, ErrorStr, ReasonStr, json_stack(Error)). + +send_error(Req, Code, ErrorStr, ReasonStr) -> + send_error(Req, Code, [], ErrorStr, ReasonStr, []). + +send_error(Req, Code, Headers, ErrorStr, ReasonStr) -> + send_error(Req, Code, Headers, ErrorStr, ReasonStr, []). + +send_error(Req, Code, Headers, ErrorStr, ReasonStr, Stack) -> + send_json(Req, Code, Headers, + {[{<<"error">>, ErrorStr}, + {<<"reason">>, ReasonStr} | + case Stack of [] -> []; _ -> [{stack, Stack}] end + ]}). + +% 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} | + case json_stack(Error) of [] -> []; Stack -> [{stack, Stack}] end + ]}, + 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, <<>>). + +server_header() -> + couch_httpd:server_header(). + +reqid() -> + {"X-Couch-Request-ID", get(nonce)}. + +json_stack({_Error, _Reason, Stack}) -> + lists:map(fun({M,F,A0}) -> + A = if is_integer(A0) -> A0; is_list(A0) -> length(A0); true -> 0 end, + list_to_binary(io_lib:format("~s:~s/~B", [M,F,A])); + (_) -> + <<"bad entry in stacktrace">> + end, Stack); +json_stack(_) -> + []. diff --git a/deps/chttpd/src/chttpd_app.erl b/deps/chttpd/src/chttpd_app.erl new file mode 100644 index 00000000..d7a5aef8 --- /dev/null +++ b/deps/chttpd/src/chttpd_app.erl @@ -0,0 +1,21 @@ +% 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/deps/chttpd/src/chttpd_db.erl b/deps/chttpd/src/chttpd_db.erl new file mode 100644 index 00000000..463aaa10 --- /dev/null +++ b/deps/chttpd/src/chttpd_db.erl @@ -0,0 +1,1264 @@ +% 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', []} -> + % if we get ?rev=... the user is using a faulty script where the + % document id is empty by accident. Let them recover safely. + case couch_httpd:qs_value(Req, "rev", false) of + false -> delete_db_req(Req, DbName); + _Rev -> throw({bad_request, + "You tried to DELETE a database with a ?=rev parameter. " + ++ "Did you mean to DELETE a document instead?"}) + end; + {_, []} -> + 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() -> + fabric:changes(Db, fun changes_callback/2, {"normal", {"Etag",Etag}, Req}, + ChangesArgs) + end); + Feed -> + % "longpoll" or "continuous" + fabric:changes(Db, fun changes_callback/2, {Feed, Req}, 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", Req}) -> + {ok, Resp} = chttpd:start_delayed_json_response(Req, 200), + {ok, {"continuous", Resp}}; +changes_callback({change, Change}, {"continuous", Resp}) -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, [?JSON_ENCODE(Change) | "\n"]), + {ok, {"continuous", Resp1}}; +changes_callback({stop, EndSeq0}, {"continuous", Resp}) -> + EndSeq = case is_old_couch(Resp) of true -> 0; false -> EndSeq0 end, + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, + [?JSON_ENCODE({[{<<"last_seq">>, EndSeq}]}) | "\n"]), + chttpd:end_delayed_json_response(Resp1); + +% callbacks for longpoll and normal (single JSON Object) +changes_callback(start, {"normal", {"Etag", Etag}, Req}) -> + FirstChunk = "{\"results\":[\n", + {ok, Resp} = chttpd:start_delayed_json_response(Req, 200, + [{"Etag",Etag}], FirstChunk), + {ok, {"", Resp}}; +changes_callback(start, {_, Req}) -> + FirstChunk = "{\"results\":[\n", + {ok, Resp} = chttpd:start_delayed_json_response(Req, 200, [], FirstChunk), + {ok, {"", Resp}}; +changes_callback({change, Change}, {Prepend, Resp}) -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, [Prepend, ?JSON_ENCODE(Change)]), + {ok, {",\r\n", Resp1}}; +changes_callback({stop, EndSeq}, {_, Resp}) -> + {ok, Resp1} = case is_old_couch(Resp) of + true -> + chttpd:send_delayed_chunk(Resp, "\n],\n\"last_seq\":0}\n"); + false -> + chttpd:send_delayed_chunk(Resp, + ["\n],\n\"last_seq\":", ?JSON_ENCODE(EndSeq), "}\n"]) + end, + chttpd:end_delayed_json_response(Resp1); + +changes_callback(timeout, {Prepend, Resp}) -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, "\n"), + {ok, {Prepend, Resp1}}; +changes_callback({error, Reason}, {_, Resp}) -> + chttpd:send_delayed_error(Resp, Reason). + +is_old_couch(Resp) -> + MochiReq = chttpd:get_delayed_req(Resp), + case MochiReq:get_header_value("user-agent") of + undefined -> + false; + "CouchDB/1.0.0" -> + true; + 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} -> + Handler = couch_util:get_value(Action, DesignUrlHandlers, + fun bad_action_req/3), + Handler(Req, Db, DDoc); + Error -> + throw(Error) + end; + +handle_design_req(Req, Db) -> + db_req(Req, Db). + +bad_action_req(#httpd{path_parts=[_, _, Name|FileNameParts]}=Req, Db, _DDoc) -> + db_attachment_req(Req, Db, <<"_design/",Name/binary>>, FileNameParts). + +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{}=Req, DbName) -> + couch_httpd:verify_is_server_admin(Req), + 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")), + DocUrl = absolute_uri(Req, "/" ++ couch_util:url_encode(DbName)), + case fabric:create_db(DbName, [{n,N}, {q,Q}]) of + ok -> + send_json(Req, 201, [{"Location", DocUrl}], {[{ok, true}]}); + accepted -> + send_json(Req, 202, [{"Location", DocUrl}], {[{ok, true}]}); + {error, file_exists} -> + chttpd:send_error(Req, file_exists); + Error -> + throw(Error) + end. + +delete_db_req(#httpd{}=Req, DbName) -> + couch_httpd:verify_is_server_admin(Req), + case fabric:delete_db(DbName, []) of + ok -> + send_json(Req, 200, {[{ok, true}]}); + accepted -> + send_json(Req, 202, {[{ok, true}]}); + Error -> + throw(Error) + end. + +do_db_req(#httpd{path_parts=[DbName|_], user_ctx=Ctx}=Req, Fun) -> + fabric:get_security(DbName, [{user_ctx,Ctx}]), % calls check_is_reader + Fun(Req, #db{name=DbName, user_ctx=Ctx}). + +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"), + + W = couch_httpd:qs_value(Req, "w", integer_to_list(mem3:quorum(Db))), + Options = [{user_ctx,Ctx}, {w,W}], + + 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, Options)) of + {ok, _} -> ok; + {accepted, _} -> ok; + Error -> + ?LOG_INFO("Batch doc error (~s): ~p",[DocId, Error]) + end + end), + + send_json(Req, 202, [], {[ + {ok, true}, + {id, DocId} + ]}); + _Normal -> + % normal + DocUrl = absolute_uri(Req, [$/, DbName, $/, DocId]), + case fabric:update_doc(Db, Doc2, Options) of + {ok, NewRev} -> + HttpCode = 201; + {accepted, NewRev} -> + HttpCode = 202 + end, + send_json(Req, HttpCode, [{"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), + W = couch_httpd:qs_value(Req, "w", integer_to_list(mem3:quorum(Db))), + case chttpd:header_value(Req, "X-Couch-Full-Commit") of + "true" -> + Options = [full_commit, {user_ctx,Ctx}, {w,W}]; + "false" -> + Options = [delay_commit, {user_ctx,Ctx}, {w,W}]; + _ -> + Options = [{user_ctx,Ctx}, {w,W}] + 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); + {accepted, Results} -> + % output the results + DocResults = lists:zipwith(fun update_doc_result_to_json/2, + Docs, Results), + send_json(Req, 202, 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], + case fabric:update_docs(Db, Docs, [replicated_changes|Options]) of + {ok, Errors} -> + ErrorsJson = lists:map(fun update_doc_result_to_json/1, Errors), + send_json(Req, 201, ErrorsJson); + {accepted, Errors} -> + ErrorsJson = lists:map(fun update_doc_result_to_json/1, Errors), + send_json(Req, 202, ErrorsJson) + end + 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">>],user_ctx=Ctx}=Req, Db) -> + send_json(Req, fabric:get_security(Db, [{user_ctx,Ctx}])); + +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_delayed_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", + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, io_lib:format(Chunk, [Total, Offset])), + {ok, {"", Resp1}}; +all_docs_callback({row, Row}, {Prepend, Resp}) -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, [Prepend, ?JSON_ENCODE(Row)]), + {ok, {",\r\n", Resp1}}; +all_docs_callback(complete, {_, Resp}) -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, "\r\n]}"), + chttpd:end_delayed_json_response(Resp1); +all_docs_callback({error, Reason}, {_, Resp}) -> + chttpd:send_delayed_error(Resp, 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"), + + W = couch_httpd:qs_value(Req, "w", integer_to_list(mem3:quorum(Db))), + Options = [{user_ctx,Ctx}, {w,W}], + + 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 + }, + case fabric:update_doc(Db, NewDoc, Options) of + {ok, NewRev} -> + HttpCode = 201; + {accepted, NewRev} -> + HttpCode = 202 + end, + send_json(Req, HttpCode, [{"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", integer_to_list(mem3:quorum(Db))), + 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, WaitFun, Parser} = couch_doc:doc_from_multi_part_stream(ContentType, + fun() -> receive_request_data(Req) end), + Doc = couch_doc_from_req(Req, DocId, Doc0), + try + Result = update_doc(Req, Db, DocId, Doc, RespHeaders, UpdateType), + WaitFun(), + Result + catch throw:Err -> + % Document rejected by a validate_doc_update function. + couch_doc:abort_multi_part_stream(Parser), + throw(Err) + end; + _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; + {accepted, _} -> 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 + case fabric:update_doc(Db, + Doc#doc{id=TargetDocId, revs=TargetRevs}, [{user_ctx,Ctx}]) of + {ok, NewTargetRev} -> + HttpCode = 201; + {accepted, NewTargetRev} -> + HttpCode = 202 + end, + % respond + {PartRes} = update_doc_result_to_json(TargetDocId, {ok, NewTargetRev}), + send_json(Req, HttpCode, + [{"Etag", "\"" ++ ?b2l(couch_doc:rev_to_str(NewTargetRev)) ++ "\""}], + {[{ok, true}] ++ PartRes}); + +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 + chttpd: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, true), + 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, true) + end; + false -> + send_json(Req, 200, Headers, couch_doc:to_json_obj(Doc, Options)) + end. + +send_docs_multipart(Req, Results, Options1) -> + OuterBoundary = couch_uuids:random(), + InnerBoundary = couch_uuids:random(), + Options = [attachments, follows, att_encoding_info | Options1], + 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, Options)), + {ContentType, _Len} = couch_doc:len_doc_to_multi_part_stream( + InnerBoundary, JsonBytes, Atts, true), + 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, true), + 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) -> + receive_request_data(Req, chttpd:body_length(Req)). + +receive_request_data(Req, LenLeft) when LenLeft > 0 -> + Len = erlang:min(4096, LenLeft), + Data = chttpd:recv(Req, Len), + {Data, fun() -> receive_request_data(Req, LenLeft - iolist_size(Data)) end}; +receive_request_data(_Req, _) -> + throw(<<"expected more data">>). + +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, {accepted, NewRev}) -> + {[{id, DocId}, {rev, couch_doc:rev_to_str(NewRev)}, {accepted, true}]}; +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", integer_to_list(mem3:quorum(Db))), + Options = + case couch_httpd:header_value(Req, "X-Couch-Full-Commit") of + "true" -> + [full_commit, UpdateType, {user_ctx,Ctx}, {w,W}]; + "false" -> + [delay_commit, UpdateType, {user_ctx,Ctx}, {w,W}]; + _ -> + [UpdateType, {user_ctx,Ctx}, {w,W}] + end, + {_, Ref} = spawn_monitor(fun() -> exit(fabric:update_doc(Db, Doc, Options)) end), + Result = receive {'DOWN', Ref, _, _, Res} -> Res end, + case Result of + {{nocatch, Exception}, _Reason} -> + % Exceptions from spawned processes are swallowed and returned, rethrow + throw(Exception); + _ -> + ok + end, + + case Result of + {ok, NewRev} -> + Accepted = false; + {accepted, NewRev} -> + Accepted = true + end, + NewRevStr = couch_doc:rev_to_str(NewRev), + ResponseHeaders = [{"Etag", <<"\"", NewRevStr/binary, "\"">>} | Headers], + case {Accepted, Deleted} of + {true, _} -> + HttpCode = 202; + {false, true} -> + HttpCode = 200; + {false, false} -> + HttpCode = 201 + end, + send_json(Req, HttpCode, 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',mochi_req=MochiReq}=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 when Enc =/= identity -> + % RFC 2616 says that the 'identify' encoding should not be used in + % the Content-Encoding header + [{"Content-Encoding", atom_to_list(Enc)}]; + _ -> + [] + end ++ case Enc of + identity -> + [{"Accept-Ranges", "bytes"}]; + _ -> + [{"Accept-Ranges", "none"}] + 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, + chttpd: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, Resp}), + couch_httpd:last_chunk(Resp); + _ -> + Ranges = parse_ranges(MochiReq:get(range), Len), + case {Enc, Ranges} of + {identity, [{From, To}]} -> + Headers1 = [{<<"Content-Range">>, make_content_range(From, To, Len)}] + ++ Headers, + {ok, Resp} = start_response_length(Req, 206, Headers1, To - From + 1), + couch_doc:range_att_foldl(Att, From, To + 1, + fun(Seg, _) -> send(Resp, Seg) end, {ok, Resp}); + {identity, Ranges} when is_list(Ranges) -> + send_ranges_multipart(Req, Type, Len, Att, Ranges); + _ -> + Headers1 = Headers ++ + if Enc =:= identity orelse ReqAcceptsAttEnc =:= true -> + [{"Content-MD5", base64:encode(Att#att.md5)}]; + true -> + [] + end, + {ok, Resp} = start_response_length(Req, 200, Headers1, Len), + AttFun(Att, fun(Seg, _) -> send(Resp, Seg) end, {ok, Resp}) + end + 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, revs = {Pos, Revs}} = Doc, + DocEdited = Doc#doc{ + % prune revision list as a workaround for key tree bug (COUCHDB-902) + revs = {Pos, case Revs of [] -> []; [Hd|_] -> [Hd] end}, + atts = NewAtt ++ [A || A <- Atts, A#att.name /= FileName] + }, + case fabric:update_doc(Db, DocEdited, [{user_ctx,Ctx}]) of + {ok, UpdatedRev} -> + HttpCode = 201; + {accepted, UpdatedRev} -> + HttpCode = 202 + end, + erlang:put(mochiweb_request_recv, true), + #db{name=DbName} = Db, + + {Status, Headers} = case Method of + 'DELETE' -> + {200, []}; + _ -> + {HttpCode, [{"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"). + +send_ranges_multipart(Req, ContentType, Len, Att, Ranges) -> + Boundary = couch_uuids:random(), + CType = {"Content-Type", + "multipart/byteranges; boundary=\"" ++ ?b2l(Boundary) ++ "\""}, + {ok, Resp} = start_chunked_response(Req, 206, [CType]), + couch_httpd:send_chunk(Resp, <<"--", Boundary/binary>>), + lists:foreach(fun({From, To}) -> + ContentRange = make_content_range(From, To, Len), + couch_httpd:send_chunk(Resp, + <<"\r\nContent-Type: ", ContentType/binary, "\r\n", + "Content-Range: ", ContentRange/binary, "\r\n", + "\r\n">>), + couch_doc:range_att_foldl(Att, From, To + 1, + fun(Seg, _) -> send_chunk(Resp, Seg) end, {ok, Resp}), + couch_httpd:send_chunk(Resp, <<"\r\n--", Boundary/binary>>) + end, Ranges), + couch_httpd:send_chunk(Resp, <<"--">>), + couch_httpd:last_chunk(Resp), + {ok, Resp}. + +parse_ranges(undefined, _Len) -> + undefined; +parse_ranges(fail, _Len) -> + undefined; +parse_ranges(Ranges, Len) -> + parse_ranges(Ranges, Len, []). + +parse_ranges([], _Len, Acc) -> + lists:reverse(Acc); +parse_ranges([{From, To}|_], _Len, _Acc) + when is_integer(From) andalso is_integer(To) andalso To < From -> + throw(requested_range_not_satisfiable); +parse_ranges([{From, To}|Rest], Len, Acc) + when is_integer(To) andalso To >= Len -> + parse_ranges([{From, Len-1}] ++ Rest, Len, Acc); +parse_ranges([{none, To}|Rest], Len, Acc) -> + parse_ranges([{Len - To, Len - 1}] ++ Rest, Len, Acc); +parse_ranges([{From, none}|Rest], Len, Acc) -> + parse_ranges([{From, Len - 1}] ++ Rest, Len, Acc); +parse_ranges([{From,To}|Rest], Len, Acc) -> + parse_ranges(Rest, Len, [{From, To}] ++ Acc). + +make_content_range(From, To, Len) -> + ?l2b(io_lib:format("bytes ~B-~B/~B", [From, To, Len])). + +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}; + {"conflicts", "true"} -> + Args#changes_args{conflicts=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/deps/chttpd/src/chttpd_external.erl b/deps/chttpd/src/chttpd_external.erl new file mode 100644 index 00000000..df27a299 --- /dev/null +++ b/deps/chttpd/src/chttpd_external.erl @@ -0,0 +1,174 @@ +% 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" ++ _ when Method =:= 'POST' -> + 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}}, + {<<"uuid">>, couch_uuids:new()}, + {<<"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/deps/chttpd/src/chttpd_misc.erl b/deps/chttpd/src/chttpd_misc.erl new file mode 100644 index 00000000..1d0556d5 --- /dev/null +++ b/deps/chttpd/src/chttpd_misc.erl @@ -0,0 +1,283 @@ +% 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_method_not_allowed/2, + send_chunk/2,start_chunked_response/3]). + +% 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) -> + ShardDbName = couch_config:get("mem3", "shard_db", "dbs"), + %% shard_db is not sharded but mem3:shards treats it as an edge case + %% so it can be pushed thru fabric + {ok, Info} = fabric:get_db_info(ShardDbName), + Etag = couch_httpd:make_etag({Info}), + chttpd:etag_respond(Req, Etag, fun() -> + {ok, Resp} = chttpd:start_delayed_json_response(Req, 200, [{"Etag",Etag}]), + fabric:all_docs(ShardDbName, fun all_dbs_callback/2, + {nil, Resp}, #view_query_args{}) + end); +handle_all_dbs_req(Req) -> + send_method_not_allowed(Req, "GET,HEAD"). + +all_dbs_callback({total_and_offset, _Total, _Offset}, {_, Resp}) -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, "["), + {ok, {"", Resp1}}; +all_dbs_callback({row, {Row}}, {Prepend, Resp}) -> + case couch_util:get_value(id, Row) of <<"_design", _/binary>> -> + {ok, {Prepend, Resp}}; + DbName -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, [Prepend, ?JSON_ENCODE(DbName)]), + {ok, {",", Resp1}} + end; +all_dbs_callback(complete, {_, Resp}) -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, "]"), + chttpd:end_delayed_json_response(Resp1); +all_dbs_callback({error, Reason}, {_, Resp}) -> + chttpd:send_delayed_error(Resp, Reason). + +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) -> + couch_httpd:validate_ctype(Req, "application/json"), + %% see HACK in chttpd.erl about replication + PostBody = get(post_body), + try replicate(PostBody, Ctx, mem3_rep_manager) 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} -> + try + send_json(Req, 500, {[{error, Reason}]}) + catch + exit:{json_encode, _} -> + send_json(Req, 500, {[{error, couch_util:to_binary(Reason)}]}) + end + catch + throw:{db_not_found, Msg} -> + send_json(Req, 404, {[{error, db_not_found}, {reason, Msg}]}); + throw:{unauthorized, Msg} -> + send_json(Req, 404, {[{error, unauthorized}, {reason, Msg}]}) + end; +handle_replicate_req(Req) -> + send_method_not_allowed(Req, "POST"). + +replicate({Props} = PostBody, Ctx, Module) -> + Node = choose_node([ + couch_util:get_value(<<"source">>, Props), + couch_util:get_value(<<"target">>, Props) + ]), + case rpc:call(Node, couch_rep, replicate, [PostBody, Ctx, Module]) of + {badrpc, Reason} -> + erlang:error(Reason); + Res -> + Res + end. + +choose_node(Key) when is_binary(Key) -> + Checksum = erlang:crc32(Key), + Nodes = lists:sort([node()|erlang:nodes()]), + lists:nth(1 + Checksum rem length(Nodes), Nodes); +choose_node(Key) -> + choose_node(term_to_binary(Key)). + +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])], + {NumberOfGCs, WordsReclaimed, _} = statistics(garbage_collection), + {{input, Input}, {output, Output}} = statistics(io), + {message_queue_len, MessageQueueLen} = process_info(whereis(couch_server), + message_queue_len), + send_json(Req, {[ + {uptime, element(1,statistics(wall_clock)) div 1000}, + {memory, {Memory}}, + {run_queue, statistics(run_queue)}, + {ets_table_count, length(ets:all())}, + {context_switches, element(1, statistics(context_switches))}, + {reductions, element(1, statistics(reductions))}, + {garbage_collection_count, NumberOfGCs}, + {words_reclaimed, WordsReclaimed}, + {io_input, Input}, + {io_output, Output}, + {os_proc_count, couch_proc_manager:get_proc_count()}, + {process_count, erlang:system_info(process_count)}, + {process_limit, erlang:system_info(process_limit)}, + {message_queue_len, MessageQueueLen} + ]}). diff --git a/deps/chttpd/src/chttpd_rewrite.erl b/deps/chttpd/src/chttpd_rewrite.erl new file mode 100644 index 00000000..f512ba5f --- /dev/null +++ b/deps/chttpd/src/chttpd_rewrite.erl @@ -0,0 +1,421 @@ +% 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, {bind, <<"*">>}). + + +%% 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_binding(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.">>); + Bin when is_binary(Bin) -> + couch_httpd:send_error(Req, 400, <<"rewrite_error">>, + <<"Rewrite rules are a String. They must be a JSON Array.">>); + 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, couch_util:to_binary(Method), PathParts, + QueryList1) of + no_dispatch_path -> + throw(not_found); + {NewPathParts, Bindings} -> + Parts = [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. + +quote_plus({bind, X}) -> + mochiweb_util:quote_plus(X); +quote_plus(X) -> + mochiweb_util:quote_plus(X). + +%% @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_binding(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_binding(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_binding(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_binding(Key), Value1}|Acc]); +make_query_list([{Key, Value}|Rest], Bindings, Acc) -> + make_query_list(Rest, Bindings, [{to_binding(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 = to_binding(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([{bind, P}|Rest], Bindings, Remaining, Acc) -> + P2 = case couch_util:get_value({bind, 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({bind, 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([{bind, Token}|RestToken],[Match|RestMatch],Bindings) -> + bind_path(RestToken, RestMatch, [{{bind, 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 -> ?MATCH_ALL; + M -> to_binding(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 -> [?MATCH_ALL]; + 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>> -> + to_binding(Var); + _ -> P + end, + path_to_list(R, [P1|Acc], DotDotCount). + +encode_query(Props) -> + Props1 = lists:foldl(fun ({{bind, K}, V}, Acc) -> + V1 = case is_list(V) orelse is_binary(V) of + true -> V; + false -> + % probably it's a number + quote_plus(V) + end, + [{K, V1} | Acc] + end, [], Props), + lists:flatten(mochiweb_util:urlencode(Props1)). + +to_binding({bind, V}) -> + {bind, V}; +to_binding(V) when is_list(V) -> + to_binding(?l2b(V)); +to_binding(V) -> + {bind, V}. + +to_json(V) -> + iolist_to_binary(?JSON_ENCODE(V)). diff --git a/deps/chttpd/src/chttpd_show.erl b/deps/chttpd/src/chttpd_show.erl new file mode 100644 index 00000000..dadf9c7a --- /dev/null +++ b/deps/chttpd/src/chttpd_show.erl @@ -0,0 +1,314 @@ +% 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"). + +-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) -> + chttpd: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) -> + chttpd: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}), + case fabric:update_doc(Db, NewDoc, Options) of + {ok, _} -> + Code = 201; + {accepted, _} -> + Code = 202 + end; + [<<"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) -> + chttpd: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) -> + chttpd:send_error(Req, 404, <<"list_error">>, <<"Invalid path.">>); + +handle_view_list_req(Req, _Db, _DDoc) -> + chttpd: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, + try couch_query_servers:proc_prompt(Proc, [<<"list_end">>]) of + [<<"end">>, Chunk] -> + {ok, Resp1} = send_non_empty_chunk(Resp, Chunk), + chttpd:send_delayed_last_chunk(Resp1) + catch Error -> + {ok, Resp1} = chttpd:send_delayed_error(Resp, Error), + {stop, Resp1} + end; +list_callback({error, Reason}, #lacc{resp=Resp}) -> + chttpd:send_delayed_error(Resp, 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} = chttpd:start_delayed_chunked_response(Req, Code, + JsonHeaders, 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] -> + {ok, Resp1} = send_non_empty_chunk(Resp, Chunk), + {ok, Acc#lacc{resp=Resp1}}; + [<<"end">>, Chunk] -> + {ok, Resp1} = send_non_empty_chunk(Resp, Chunk), + {ok, Resp2} = chttpd:send_delayed_last_chunk(Resp1), + {stop, Resp2} + catch Error -> + {ok, Resp1} = chttpd:send_delayed_error(Resp, Error), + {stop, Resp1} + end. + +send_non_empty_chunk(Resp, []) -> + {ok, Resp}; +send_non_empty_chunk(Resp, Chunk) -> + chttpd:send_delayed_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/deps/chttpd/src/chttpd_sup.erl b/deps/chttpd/src/chttpd_sup.erl new file mode 100644 index 00000000..bfe6be90 --- /dev/null +++ b/deps/chttpd/src/chttpd_sup.erl @@ -0,0 +1,25 @@ +% 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/deps/chttpd/src/chttpd_view.erl b/deps/chttpd/src/chttpd_view.erl new file mode 100644 index 00000000..7e13c356 --- /dev/null +++ b/deps/chttpd/src/chttpd_view.erl @@ -0,0 +1,393 @@ +% 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]). + + +multi_query_view(Req, Db, DDoc, ViewName, Queries) -> + Group = couch_view_group:design_doc_to_view_group(DDoc), + IsReduce = get_reduce_type(Req), + ViewType = extract_view_type(ViewName, Group#group.views, IsReduce), + % TODO proper calculation of etag + % Etag = view_group_etag(ViewGroup, Db, Queries), + Etag = couch_uuids:new(), + DefaultParams = lists:flatmap(fun({K,V}) -> parse_view_param(K,V) end, + chttpd:qs(Req)), + [couch_stats_collector:increment({httpd, view_reads}) || _I <- Queries], + chttpd:etag_respond(Req, Etag, fun() -> + FirstChunk = "{\"results\":[", + {ok, Resp} = chttpd:start_delayed_json_response(Req, 200, [{"Etag",Etag}], FirstChunk), + {_, Resp1} = lists:foldl(fun({QueryProps}, {Chunk, RespAcc}) -> + if Chunk =/= nil -> chttpd:send_delayed_chunk(Resp, Chunk); true -> ok end, + ThisQuery = lists:flatmap(fun parse_json_view_param/1, QueryProps), + FullParams = lists:ukeymerge(1, ThisQuery, DefaultParams), + {ok, RespAcc1} = fabric:query_view( + Db, + DDoc, + ViewName, + fun view_callback/2, + {nil, RespAcc}, + parse_view_params(FullParams, nil, ViewType) + ), + {",\n", RespAcc1} + end, {nil,Resp}, Queries), + chttpd:send_delayed_chunk(Resp1, "]}"), + chttpd:end_delayed_json_response(Resp1) + end). + +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_delayed_json_response(Req, 200, [{"Etag",Etag}]), + CB = fun view_callback/2, + {ok, Resp1} = fabric:query_view(Db, DDoc, ViewName, CB, {nil, Resp}, QueryArgs), + chttpd:end_delayed_json_response(Resp1) + end). + +view_callback({total_and_offset, Total, Offset}, {nil, Resp}) -> + Chunk = "{\"total_rows\":~p,\"offset\":~p,\"rows\":[\r\n", + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, io_lib:format(Chunk, [Total, Offset])), + {ok, {"", Resp1}}; +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 + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, ["{\"rows\":[\r\n", ?JSON_ENCODE(Row)]), + {ok, {",\r\n", Resp1}}; +view_callback({row, Row}, {Prepend, Resp}) -> + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, [Prepend, ?JSON_ENCODE(Row)]), + {ok, {",\r\n", Resp1}}; +view_callback(complete, {nil, Resp}) -> + chttpd:send_delayed_chunk(Resp, "{\"rows\":[]}"); +view_callback(complete, {_, Resp}) -> + chttpd:send_delayed_chunk(Resp, "\r\n]}"); +view_callback({error, Reason}, {_, Resp}) -> + chttpd:send_delayed_error(Resp, 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), + Queries = couch_util:get_value(<<"queries">>, Fields), + Keys = couch_util:get_value(<<"keys">>, Fields), + case {Queries, Keys} of + {Queries, undefined} when is_list(Queries) -> + multi_query_view(Req, Db, DDoc, ViewName, Queries); + {undefined, Keys} when is_list(Keys) -> + design_doc_view(Req, Db, DDoc, ViewName, Keys); + {undefined, undefined} -> + throw({bad_request, "POST body must contain `keys` or `queries` field"}); + {undefined, _} -> + throw({bad_request, "`keys` body member must be an array"}); + {_, undefined} -> + throw({bad_request, "`queries` body member must be an array"}); + {_, _} -> + throw({bad_request, "`keys` and `queries` are mutually exclusive"}) + end; + +handle_view_req(Req, _Db, _DDoc) -> + chttpd: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_existing_atom(chttpd:qs_value(Req, "reduce", "true")). + +parse_view_params(Req, Keys, ViewType) when not is_list(Req) -> + QueryParams = lists:flatmap(fun({K,V}) -> parse_view_param(K,V) end, + chttpd:qs(Req)), + parse_view_params(QueryParams, Keys, ViewType); +parse_view_params(QueryParams, Keys, ViewType) -> + IsMultiGet = (Keys =/= nil), + 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, QueryParams), + + 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_json_view_param({<<"key">>, V}) -> + [{start_key, V}, {end_key, V}]; +parse_json_view_param({<<"startkey_docid">>, V}) -> + [{start_docid, V}]; +parse_json_view_param({<<"endkey_docid">>, V}) -> + [{end_docid, V}]; +parse_json_view_param({<<"startkey">>, V}) -> + [{start_key, V}]; +parse_json_view_param({<<"endkey">>, V}) -> + [{end_key, V}]; +parse_json_view_param({<<"limit">>, V}) when is_integer(V), V > 0 -> + [{limit, V}]; +parse_json_view_param({<<"stale">>, <<"ok">>}) -> + [{stale, ok}]; +parse_json_view_param({<<"stale">>, <<"update_after">>}) -> + [{stale, update_after}]; +parse_json_view_param({<<"descending">>, V}) when is_boolean(V) -> + [{descending, V}]; +parse_json_view_param({<<"skip">>, V}) when is_integer(V) -> + [{skip, V}]; +parse_json_view_param({<<"group">>, true}) -> + [{group_level, exact}]; +parse_json_view_param({<<"group">>, false}) -> + [{group_level, 0}]; +parse_json_view_param({<<"group_level">>, V}) when is_integer(V), V > 0 -> + [{group_level, V}]; +parse_json_view_param({<<"inclusive_end">>, V}) when is_boolean(V) -> + [{inclusive_end, V}]; +parse_json_view_param({<<"reduce">>, V}) when is_boolean(V) -> + [{reduce, V}]; +parse_json_view_param({<<"include_docs">>, V}) when is_boolean(V) -> + [{include_docs, V}]; +parse_json_view_param({<<"conflicts">>, V}) when is_boolean(V) -> + [{conflicts, V}]; +parse_json_view_param({<<"list">>, V}) -> + [{list, couch_util:to_binary(V)}]; +parse_json_view_param({<<"sorted">>, V}) when is_boolean(V) -> + [{sorted, V}]; +parse_json_view_param({K, V}) -> + [{extra, {K, V}}]. + +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", "update_after") -> + [{stale, update_after}]; +parse_view_param("stale", _Value) -> + throw({query_parse_error, + <<"stale only available as stale=ok or as stale=update_after">>}); +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("conflicts", Value) -> + [{conflicts, 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("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, false, Args) -> + Args; +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(conflicts, true, Args) -> + case Args#view_query_args.view_type of + reduce -> + Msg = <<"Query parameter `conflicts` " + "is invalid for reduce views.">>, + throw({query_parse_error, Msg}); + _ -> + Args#view_query_args{conflicts = true} + end; +validate_view_query(conflicts, _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. diff --git a/deps/chttpd/test/chttpd_delayed_response_test.erl b/deps/chttpd/test/chttpd_delayed_response_test.erl new file mode 100644 index 00000000..f6509dd2 --- /dev/null +++ b/deps/chttpd/test/chttpd_delayed_response_test.erl @@ -0,0 +1,43 @@ +%% Copyright 2011 Cloudant +%% +%% 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_delayed_response_test). +-include_lib("eunit/include/eunit.hrl"). +-include_lib("couch/include/couch_db.hrl"). + +all_test_() -> + {foreach, + fun() -> application:load(couch) end, + fun(_) -> application:unload(couch) end, + [ + fun delayed_chunked_response/1, + fun delayed_chunked_response_after_error/1 + ]}. + +delayed_chunked_response(_) -> + {"sending an error first should be ok", + fun() -> + Req = #httpd{mochi_req=mock_request:new(nil, get, "/", {1, 1}, [])}, + {ok, Resp} = chttpd:start_delayed_chunked_response(Req, 200, []), + ?assertMatch({ok, _}, chttpd:send_delayed_error(Resp, bad_request)) + end}. + +delayed_chunked_response_after_error(_) -> + {"sending an error midstream should throw http_abort", + fun() -> + Req = #httpd{mochi_req=mock_request:new(nil, get, "/", {1, 1}, [])}, + {ok, Resp} = chttpd:start_delayed_chunked_response(Req, 200, []), + {ok, Resp1} = chttpd:send_delayed_chunk(Resp, <<>>), + ?assertThrow({http_abort, _, _}, chttpd:send_delayed_error(Resp1, bad_request)) + end}. diff --git a/deps/chttpd/test/mock_request.erl b/deps/chttpd/test/mock_request.erl new file mode 100644 index 00000000..e1f8b0ad --- /dev/null +++ b/deps/chttpd/test/mock_request.erl @@ -0,0 +1,39 @@ +%% Copyright 2011 Cloudant +%% +%% 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(mock_request, [Socket, Method, RawPath, Version, Headers]). + +-compile(export_all). + +get_header_value(_) -> + undefined. + +parse_qs() -> + []. + +get(method) -> + Method; +get(raw_path) -> + RawPath; +get(version) -> + Version. + +should_close() -> + false. + +respond({Code, ResponseHeaders, _}) -> + mochiweb:new_response({THIS, Code, ResponseHeaders}). + +send(_) -> + ok. |