summaryrefslogtreecommitdiff
path: root/deps/chttpd/src
diff options
context:
space:
mode:
Diffstat (limited to 'deps/chttpd/src')
-rw-r--r--deps/chttpd/src/chttpd.app.src7
-rw-r--r--deps/chttpd/src/chttpd.erl782
-rw-r--r--deps/chttpd/src/chttpd_app.erl21
-rw-r--r--deps/chttpd/src/chttpd_db.erl1264
-rw-r--r--deps/chttpd/src/chttpd_external.erl174
-rw-r--r--deps/chttpd/src/chttpd_misc.erl283
-rw-r--r--deps/chttpd/src/chttpd_rewrite.erl421
-rw-r--r--deps/chttpd/src/chttpd_show.erl314
-rw-r--r--deps/chttpd/src/chttpd_sup.erl25
-rw-r--r--deps/chttpd/src/chttpd_view.erl393
10 files changed, 3684 insertions, 0 deletions
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.