summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--src/chttpd/src/chttpd.erl27
-rw-r--r--src/chttpd/test/eunit/chttpd_handlers_tests.erl24
-rw-r--r--src/couch_pse_tests/src/cpse_test_purge_replication.erl16
-rw-r--r--src/couch_replicator/src/couch_replicator.erl1
-rw-r--r--src/couch_replicator/src/couch_replicator_api_wrap.erl150
-rw-r--r--src/couch_replicator/src/couch_replicator_doc_processor.erl12
-rw-r--r--src/couch_replicator/src/couch_replicator_doc_processor_worker.erl6
-rw-r--r--src/couch_replicator/src/couch_replicator_docs.erl29
-rw-r--r--src/couch_replicator/src/couch_replicator_filters.erl13
-rw-r--r--src/couch_replicator/src/couch_replicator_ids.erl44
-rw-r--r--src/couch_replicator/src/couch_replicator_scheduler_job.erl31
-rw-r--r--src/couch_replicator/src/couch_replicator_utils.erl56
-rw-r--r--src/couch_replicator/src/couch_replicator_worker.erl143
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl4
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_compact_tests.erl5
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl10
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl5
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl5
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl6
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl5
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl5
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl5
-rw-r--r--src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl5
-rw-r--r--test/elixir/test/replication_test.exs36
-rw-r--r--test/javascript/tests/purge.js4
-rw-r--r--test/javascript/tests/replication.js65
-rw-r--r--test/javascript/tests/rev_stemming.js8
-rw-r--r--test/javascript/tests/users_db.js13
-rw-r--r--test/javascript/tests/view_conflicts.js6
-rw-r--r--test/javascript/tests/view_include_docs.js6
30 files changed, 175 insertions, 570 deletions
diff --git a/src/chttpd/src/chttpd.erl b/src/chttpd/src/chttpd.erl
index 1e1d638be..f3f1e3818 100644
--- a/src/chttpd/src/chttpd.erl
+++ b/src/chttpd/src/chttpd.erl
@@ -410,8 +410,8 @@ maybe_log(_HttpReq, #httpd_resp{should_log = false}) ->
%% the target, it's hacked to make it a full url and treated as a remote.
possibly_hack(#httpd{path_parts=[<<"_replicate">>]}=Req) ->
{Props0} = chttpd:json_body_obj(Req),
- Props1 = fix_uri(Req, Props0, <<"source">>),
- Props2 = fix_uri(Req, Props1, <<"target">>),
+ Props1 = error_if_local_endpoint(Props0, <<"source">>),
+ Props2 = error_if_local_endpoint(Props1, <<"target">>),
put(post_body, {Props2}),
Req;
possibly_hack(Req) ->
@@ -439,7 +439,7 @@ check_url_encoding([$% | _]) ->
check_url_encoding([_ | Rest]) ->
check_url_encoding(Rest).
-fix_uri(Req, Props, Type) ->
+error_if_local_endpoint(Props, Type) ->
case replication_uri(Type, Props) of
undefined ->
Props;
@@ -448,8 +448,8 @@ fix_uri(Req, Props, Type) ->
true ->
Props;
false ->
- Uri = make_uri(Req, quote(Uri0)),
- [{Type,Uri}|proplists:delete(Type,Props)]
+ Msg = "Local endpoints not supported since CouchDB 3.x",
+ throw({bad_request, Msg})
end
end.
@@ -468,23 +468,6 @@ is_http(<<"https://", _/binary>>) ->
is_http(_) ->
false.
-make_uri(Req, Raw) ->
- Port = integer_to_list(mochiweb_socket_server:get(chttpd, port)),
- Url = list_to_binary(["http://", config:get("httpd", "bind_address"),
- ":", Port, "/", Raw]),
- Headers = [
- {<<"authorization">>, ?l2b(header_value(Req,"authorization",""))},
- {<<"cookie">>, ?l2b(extract_cookie(Req))}
- ],
- {[{<<"url">>,Url}, {<<"headers">>,{Headers}}]}.
-
-extract_cookie(#httpd{mochi_req = MochiReq}) ->
- case MochiReq:get_cookie_value("AuthSession") of
- undefined ->
- "";
- AuthSession ->
- "AuthSession=" ++ AuthSession
- end.
%%% end hack
set_auth_handlers() ->
diff --git a/src/chttpd/test/eunit/chttpd_handlers_tests.erl b/src/chttpd/test/eunit/chttpd_handlers_tests.erl
index f3e8f5dcd..56407a0ee 100644
--- a/src/chttpd/test/eunit/chttpd_handlers_tests.erl
+++ b/src/chttpd/test/eunit/chttpd_handlers_tests.erl
@@ -37,35 +37,27 @@ replicate_test_() ->
foreach,
fun setup/0, fun teardown/1,
[
- fun should_escape_dbname_on_replicate/1
+ fun should_error_on_local_replicate_endpoint/1
]
}
}
}.
-should_escape_dbname_on_replicate(Url) ->
+should_error_on_local_replicate_endpoint(Url) ->
?_test(
begin
- UrlBin = ?l2b(Url),
Request = couch_util:json_encode({[
- {<<"source">>, <<UrlBin/binary, "/foo%2Fbar">>},
- {<<"target">>, <<"bar/baz">>},
- {<<"create_target">>, true}
+ {<<"source">>, <<"localdb">>},
+ {<<"target">>, <<"anotherlocaldb">>}
]}),
- {ok, 200, _, Body} = request_replicate(Url ++ "/_replicate", Request),
- JSON = couch_util:json_decode(Body),
-
- Source = json_value(JSON, [<<"source">>]),
- Target = json_value(JSON, [<<"target">>, <<"url">>]),
- ?assertEqual(<<UrlBin/binary, "/foo%2Fbar">>, Source),
- ?assertEqual(<<UrlBin/binary, "/bar%2Fbaz">>, Target)
+ {ok, 400, _, Body} = request_replicate(Url ++ "/_replicate", Request),
+ {JSONProps} = couch_util:json_decode(Body),
+ Expect = <<"Local endpoints not supported since CouchDB 3.x">>,
+ ?assertEqual(Expect, couch_util:get_value(<<"reason">>, JSONProps))
end).
-json_value(JSON, Keys) ->
- couch_util:get_nested_json_value(JSON, Keys).
-
request_replicate(Url, Body) ->
Headers = [{"Content-Type", "application/json"}],
Handler = {chttpd_misc, handle_replicate_req},
diff --git a/src/couch_pse_tests/src/cpse_test_purge_replication.erl b/src/couch_pse_tests/src/cpse_test_purge_replication.erl
index fb09eeba6..4a0822525 100644
--- a/src/couch_pse_tests/src/cpse_test_purge_replication.erl
+++ b/src/couch_pse_tests/src/cpse_test_purge_replication.erl
@@ -21,7 +21,7 @@
setup_all() ->
- cpse_util:setup_all([mem3, fabric, couch_replicator]).
+ cpse_util:setup_all([mem3, fabric, chttpd, couch_replicator]).
setup_each() ->
@@ -48,8 +48,8 @@ cpse_purge_http_replication({Source, Target}) ->
]),
RepObject = {[
- {<<"source">>, Source},
- {<<"target">>, Target}
+ {<<"source">>, db_url(Source)},
+ {<<"target">>, db_url(Target)}
]},
{ok, _} = couch_replicator:replicate(RepObject, ?ADMIN_USER),
@@ -100,8 +100,8 @@ cpse_purge_http_replication({Source, Target}) ->
% Show that replicating from the target
% back to the source reintroduces the doc
RepObject2 = {[
- {<<"source">>, Target},
- {<<"target">>, Source}
+ {<<"source">>, db_url(Target)},
+ {<<"target">>, db_url(Source)}
]},
{ok, _} = couch_replicator:replicate(RepObject2, ?ADMIN_USER),
@@ -200,3 +200,9 @@ make_shard(DbName) ->
dbname = DbName,
range = [0, 16#FFFFFFFF]
}.
+
+
+db_url(DbName) ->
+ Addr = config:get("httpd", "bind_address", "127.0.0.1"),
+ Port = mochiweb_socket_server:get(couch_httpd, port),
+ ?l2b(io_lib:format("http://~s:~b/~s", [Addr, Port, DbName])).
diff --git a/src/couch_replicator/src/couch_replicator.erl b/src/couch_replicator/src/couch_replicator.erl
index 39141c301..e4fa31cee 100644
--- a/src/couch_replicator/src/couch_replicator.erl
+++ b/src/couch_replicator/src/couch_replicator.erl
@@ -358,7 +358,6 @@ strip_url_creds_test_() ->
end,
fun (_) -> meck:unload() end,
[
- t_strip_local_db_creds(),
t_strip_http_basic_creds(),
t_strip_http_props_creds()
]
diff --git a/src/couch_replicator/src/couch_replicator_api_wrap.erl b/src/couch_replicator/src/couch_replicator_api_wrap.erl
index 44c290d33..ab1de7df9 100644
--- a/src/couch_replicator/src/couch_replicator_api_wrap.erl
+++ b/src/couch_replicator/src/couch_replicator_api_wrap.erl
@@ -23,8 +23,8 @@
-include("couch_replicator_api_wrap.hrl").
-export([
- db_open/2,
- db_open/4,
+ db_open/1,
+ db_open/3,
db_close/1,
get_db_info/1,
get_pending_count/2,
@@ -67,10 +67,10 @@ db_uri(Db) ->
db_uri(couch_db:name(Db)).
-db_open(Db, Options) ->
- db_open(Db, Options, false, []).
+db_open(Db) ->
+ db_open(Db, false, []).
-db_open(#httpdb{} = Db1, _Options, Create, CreateParams) ->
+db_open(#httpdb{} = Db1, Create, CreateParams) ->
{ok, Db} = couch_replicator_httpc:setup(Db1),
try
case Create of
@@ -118,51 +118,19 @@ db_open(#httpdb{} = Db1, _Options, Create, CreateParams) ->
exit:Error ->
db_close(Db),
erlang:exit(Error)
- end;
-db_open(DbName, Options, Create, _CreateParams) ->
- try
- case Create of
- false ->
- ok;
- true ->
- ok = couch_httpd:verify_is_server_admin(
- get_value(user_ctx, Options)),
- couch_db:create(DbName, Options)
- end,
- case couch_db:open(DbName, Options) of
- {error, {illegal_database_name, _}} ->
- throw({db_not_found, DbName});
- {not_found, _Reason} ->
- throw({db_not_found, DbName});
- {ok, _Db} = Success ->
- Success
- end
- catch
- throw:{unauthorized, _} ->
- throw({unauthorized, DbName})
end.
db_close(#httpdb{httpc_pool = Pool} = HttpDb) ->
couch_replicator_auth:cleanup(HttpDb),
unlink(Pool),
- ok = couch_replicator_httpc_pool:stop(Pool);
-db_close(DbName) ->
- catch couch_db:close(DbName).
+ ok = couch_replicator_httpc_pool:stop(Pool).
get_db_info(#httpdb{} = Db) ->
send_req(Db, [],
fun(200, _, {Props}) ->
{ok, Props}
- end);
-get_db_info(Db) ->
- DbName = couch_db:name(Db),
- UserCtx = couch_db:get_user_ctx(Db),
- {ok, InfoDb} = couch_db:open(DbName, [{user_ctx, UserCtx}]),
- {ok, Info} = couch_db:get_db_info(InfoDb),
- couch_db:close(InfoDb),
- {ok, [{couch_util:to_binary(K), V} || {K, V} <- Info]}.
-
+ end).
get_pending_count(#httpdb{} = Db, Seq) when is_number(Seq) ->
% Source looks like Apache CouchDB and not Cloudant so we fall
@@ -179,14 +147,7 @@ get_pending_count(#httpdb{} = Db, Seq) ->
Options = [{path, "_changes"}, {qs, [{"since", ?JSON_ENCODE(Seq)}, {"limit", "0"}]}],
send_req(Db, Options, fun(200, _, {Props}) ->
{ok, couch_util:get_value(<<"pending">>, Props, null)}
- end);
-get_pending_count(Db, Seq) when is_number(Seq) ->
- DbName = couch_db:name(Db),
- UserCtx = couch_db:get_user_ctx(Db),
- {ok, CountDb} = couch_db:open(DbName, [{user_ctx, UserCtx}]),
- Pending = couch_db:count_changes_since(CountDb, Seq),
- couch_db:close(CountDb),
- {ok, Pending}.
+ end).
get_view_info(#httpdb{} = Db, DDocId, ViewName) ->
Path = io_lib:format("~s/_view/~s/_info", [DDocId, ViewName]),
@@ -194,11 +155,7 @@ get_view_info(#httpdb{} = Db, DDocId, ViewName) ->
fun(200, _, {Props}) ->
{VInfo} = couch_util:get_value(<<"view_index">>, Props, {[]}),
{ok, VInfo}
- end);
-get_view_info(Db, DDocId, ViewName) ->
- DbName = couch_db:name(Db),
- {ok, VInfo} = couch_mrview:get_view_info(DbName, DDocId, ViewName),
- {ok, [{couch_util:to_binary(K), V} || {K, V} <- VInfo]}.
+ end).
ensure_full_commit(#httpdb{} = Db) ->
@@ -210,9 +167,7 @@ ensure_full_commit(#httpdb{} = Db) ->
{ok, get_value(<<"instance_start_time">>, Props)};
(_, _, {Props}) ->
{error, get_value(<<"error">>, Props)}
- end);
-ensure_full_commit(Db) ->
- couch_db:ensure_full_commit(Db).
+ end).
get_missing_revs(#httpdb{} = Db, IdRevs) ->
@@ -232,10 +187,7 @@ get_missing_revs(#httpdb{} = Db, IdRevs) ->
{Id, MissingRevs, PossibleAncestors}
end,
{ok, lists:map(ConvertToNativeFun, Props)}
- end);
-get_missing_revs(Db, IdRevs) ->
- couch_db:get_missing_revs(Db, IdRevs).
-
+ end).
open_doc_revs(#httpdb{retries = 0} = HttpDb, Id, Revs, Options, _Fun, _Acc) ->
@@ -331,10 +283,8 @@ open_doc_revs(#httpdb{} = HttpDb, Id, Revs, Options, Fun, Acc) ->
wait = Wait
},
open_doc_revs(RetryDb, Id, Revs, Options, Fun, Acc)
- end;
-open_doc_revs(Db, Id, Revs, Options, Fun, Acc) ->
- {ok, Results} = couch_db:open_doc_revs(Db, Id, Revs, Options),
- {ok, lists:foldl(fun(R, A) -> {_, A2} = Fun(R, A), A2 end, Acc, Results)}.
+ end.
+
error_reason({http_request_failed, "GET", _Url, {error, timeout}}) ->
timeout;
@@ -353,14 +303,7 @@ open_doc(#httpdb{} = Db, Id, Options) ->
{ok, couch_doc:from_json_obj(Body)};
(_, _, {Props}) ->
{error, get_value(<<"error">>, Props)}
- end);
-open_doc(Db, Id, Options) ->
- case couch_db:open_doc(Db, Id, Options) of
- {ok, _} = Ok ->
- Ok;
- {not_found, _Reason} ->
- {error, <<"not_found">>}
- end.
+ end).
update_doc(Db, Doc, Options) ->
@@ -411,9 +354,7 @@ update_doc(#httpdb{} = HttpDb, #doc{id = DocId} = Doc, Options, Type) ->
{_, Error} ->
{error, Error}
end
- end);
-update_doc(Db, Doc, Options, Type) ->
- couch_db:update_doc(Db, Doc, Options, Type).
+ end).
update_docs(Db, DocList, Options) ->
@@ -468,10 +409,7 @@ update_docs(#httpdb{} = HttpDb, DocList, Options, UpdateType) ->
{error, request_body_too_large};
(417, _, Results) when is_list(Results) ->
{ok, bulk_results_to_errors(DocList, Results, remote)}
- end);
-update_docs(Db, DocList, Options, UpdateType) ->
- Result = couch_db:update_docs(Db, DocList, Options, UpdateType),
- {ok, bulk_results_to_errors(DocList, Result, UpdateType)}.
+ end).
changes_since(#httpdb{headers = Headers1, timeout = InactiveTimeout} = HttpDb,
@@ -538,38 +476,7 @@ changes_since(#httpdb{headers = Headers1, timeout = InactiveTimeout} = HttpDb,
throw(retry_no_limit);
exit:{http_request_failed, _, _, _} = Error ->
throw({retry_limit, Error})
- end;
-changes_since(Db, Style, StartSeq, UserFun, Options) ->
- DocIds = get_value(doc_ids, Options),
- Selector = get_value(selector, Options),
- Filter = case {DocIds, Selector} of
- {undefined, undefined} ->
- ?b2l(get_value(filter, Options, <<>>));
- {_, undefined} ->
- "_doc_ids";
- {undefined, _} ->
- "_selector"
- end,
- Args = #changes_args{
- style = Style,
- since = StartSeq,
- filter = Filter,
- feed = case get_value(continuous, Options, false) of
- true ->
- "continuous";
- false ->
- "normal"
- end,
- timeout = infinity
- },
- QueryParams = get_value(query_params, Options, {[]}),
- Req = changes_json_req(Db, Filter, QueryParams, Options),
- ChangesFeedFun = couch_changes:handle_db_changes(Args, {json_req, Req}, Db),
- ChangesFeedFun(fun({change, Change, _}, _) ->
- UserFun(json_to_doc_info(Change));
- (_, _) ->
- ok
- end).
+ end.
% internal functions
@@ -614,29 +521,6 @@ parse_changes_feed(Options, UserFun, DataStreamFun) ->
json_stream_parse:events(DataStreamFun, EventFun)
end.
-changes_json_req(_Db, "", _QueryParams, _Options) ->
- {[]};
-changes_json_req(_Db, "_doc_ids", _QueryParams, Options) ->
- {[{<<"doc_ids">>, get_value(doc_ids, Options)}]};
-changes_json_req(_Db, "_selector", _QueryParams, Options) ->
- {[{<<"selector">>, get_value(selector, Options)}]};
-changes_json_req(Db, FilterName, {QueryParams}, _Options) ->
- {ok, Info} = couch_db:get_db_info(Db),
- % simulate a request to db_name/_changes
- {[
- {<<"info">>, {Info}},
- {<<"id">>, null},
- {<<"method">>, 'GET'},
- {<<"path">>, [couch_db:name(Db), <<"_changes">>]},
- {<<"query">>, {[{<<"filter">>, FilterName} | QueryParams]}},
- {<<"headers">>, []},
- {<<"body">>, []},
- {<<"peer">>, <<"replicator">>},
- {<<"form">>, []},
- {<<"cookie">>, []},
- {<<"userCtx">>, couch_util:json_user_ctx(Db)}
- ]}.
-
options_to_query_args(HttpDb, Path, Options0) ->
case lists:keytake(max_url_len, 1, Options0) of
diff --git a/src/couch_replicator/src/couch_replicator_doc_processor.erl b/src/couch_replicator/src/couch_replicator_doc_processor.erl
index 1b43598da..772037d8d 100644
--- a/src/couch_replicator/src/couch_replicator_doc_processor.erl
+++ b/src/couch_replicator/src/couch_replicator_doc_processor.erl
@@ -886,8 +886,8 @@ change() ->
{<<"id">>, ?DOC1},
{doc, {[
{<<"_id">>, ?DOC1},
- {<<"source">>, <<"src">>},
- {<<"target">>, <<"tgt">>}
+ {<<"source">>, <<"http://srchost.local/src">>},
+ {<<"target">>, <<"http://tgthost.local/tgt">>}
]}}
]}.
@@ -897,8 +897,8 @@ change(State) ->
{<<"id">>, ?DOC1},
{doc, {[
{<<"_id">>, ?DOC1},
- {<<"source">>, <<"src">>},
- {<<"target">>, <<"tgt">>},
+ {<<"source">>, <<"http://srchost.local/src">>},
+ {<<"target">>, <<"http://tgthost.local/tgt">>},
{<<"_replication_state">>, State}
]}}
]}.
@@ -910,8 +910,8 @@ deleted_change() ->
{<<"deleted">>, true},
{doc, {[
{<<"_id">>, ?DOC1},
- {<<"source">>, <<"src">>},
- {<<"target">>, <<"tgt">>}
+ {<<"source">>, <<"http://srchost.local/src">>},
+ {<<"target">>, <<"http://tgthost.local/tgt">>}
]}}
]}.
diff --git a/src/couch_replicator/src/couch_replicator_doc_processor_worker.erl b/src/couch_replicator/src/couch_replicator_doc_processor_worker.erl
index aa048bfab..a4c829323 100644
--- a/src/couch_replicator/src/couch_replicator_doc_processor_worker.erl
+++ b/src/couch_replicator/src/couch_replicator_doc_processor_worker.erl
@@ -137,7 +137,7 @@ maybe_add_job_to_scheduler({DbName, DocId}, Rep, WRef) ->
-define(DB, <<"db">>).
-define(DOC1, <<"doc1">>).
--define(R1, {"0b7831e9a41f9322a8600ccfa02245f2", ""}).
+-define(R1, {"ad08e05057046eabe898a2572bbfb573", ""}).
doc_processor_worker_test_() ->
@@ -277,8 +277,8 @@ did_not_add_job() ->
change() ->
{[
{<<"_id">>, ?DOC1},
- {<<"source">>, <<"src">>},
- {<<"target">>, <<"tgt">>}
+ {<<"source">>, <<"http://srchost.local/src">>},
+ {<<"target">>, <<"http://tgthost.local/tgt">>}
]}.
-endif.
diff --git a/src/couch_replicator/src/couch_replicator_docs.erl b/src/couch_replicator/src/couch_replicator_docs.erl
index bbf9694d7..c07caa1aa 100644
--- a/src/couch_replicator/src/couch_replicator_docs.erl
+++ b/src/couch_replicator/src/couch_replicator_docs.erl
@@ -423,8 +423,8 @@ parse_rep_db(<<"http://", _/binary>> = Url, Proxy, Options) ->
parse_rep_db(<<"https://", _/binary>> = Url, Proxy, Options) ->
parse_rep_db({[{<<"url">>, Url}]}, Proxy, Options);
-parse_rep_db(<<DbName/binary>>, _Proxy, _Options) ->
- DbName;
+parse_rep_db(<<_/binary>>, _Proxy, _Options) ->
+ throw({error, <<"Local endpoints not supported since CouchDB 3.x">>});
parse_rep_db(undefined, _Proxy, _Options) ->
throw({error, <<"Missing replicator database">>}).
@@ -822,4 +822,29 @@ t_vdu_does_not_crash_on_save(DbName) ->
?assertEqual({ok, forbidden}, save_rep_doc(DbName, Doc))
end).
+
+local_replication_endpoint_error_test_() ->
+ {
+ foreach,
+ fun () -> meck:expect(config, get,
+ fun(_, _, Default) -> Default end)
+ end,
+ fun (_) -> meck:unload() end,
+ [
+ t_error_on_local_endpoint()
+ ]
+ }.
+
+
+t_error_on_local_endpoint() ->
+ ?_test(begin
+ RepDoc = {[
+ {<<"_id">>, <<"someid">>},
+ {<<"source">>, <<"localdb">>},
+ {<<"target">>, <<"http://somehost.local/tgt">>}
+ ]},
+ Expect = <<"Local endpoints not supported since CouchDB 3.x">>,
+ ?assertThrow({bad_rep_doc, Expect}, parse_rep_doc_without_id(RepDoc))
+ end).
+
-endif.
diff --git a/src/couch_replicator/src/couch_replicator_filters.erl b/src/couch_replicator/src/couch_replicator_filters.erl
index 5668820d1..c8980001a 100644
--- a/src/couch_replicator/src/couch_replicator_filters.erl
+++ b/src/couch_replicator/src/couch_replicator_filters.erl
@@ -14,7 +14,7 @@
-export([
parse/1,
- fetch/4,
+ fetch/3,
view_type/2,
ejsort/1
]).
@@ -63,11 +63,11 @@ parse(Options) ->
% Fetches body of filter function from source database. Guaranteed to either
% return {ok, Body} or an {error, Reason}. Also assume this function might
% block due to network / socket issues for an undeterminted amount of time.
--spec fetch(binary(), binary(), binary(), #user_ctx{}) ->
+-spec fetch(binary(), binary(), binary()) ->
{ok, {[_]}} | {error, binary()}.
-fetch(DDocName, FilterName, Source, UserCtx) ->
+fetch(DDocName, FilterName, Source) ->
{Pid, Ref} = spawn_monitor(fun() ->
- try fetch_internal(DDocName, FilterName, Source, UserCtx) of
+ try fetch_internal(DDocName, FilterName, Source) of
Resp ->
exit({exit_ok, Resp})
catch
@@ -108,9 +108,8 @@ view_type(Props, Options) ->
% Private functions
-fetch_internal(DDocName, FilterName, Source, UserCtx) ->
- Db = case (catch couch_replicator_api_wrap:db_open(Source,
- [{user_ctx, UserCtx}])) of
+fetch_internal(DDocName, FilterName, Source) ->
+ Db = case (catch couch_replicator_api_wrap:db_open(Source)) of
{ok, Db0} ->
Db0;
DbError ->
diff --git a/src/couch_replicator/src/couch_replicator_ids.erl b/src/couch_replicator/src/couch_replicator_ids.erl
index e10b98082..04e71c3ef 100644
--- a/src/couch_replicator/src/couch_replicator_ids.erl
+++ b/src/couch_replicator/src/couch_replicator_ids.erl
@@ -39,19 +39,19 @@ replication_id(#rep{options = Options} = Rep) ->
% If a change is made to how replications are identified,
% please add a new clause and increase ?REP_ID_VERSION.
-replication_id(#rep{user_ctx = UserCtx} = Rep, 4) ->
+replication_id(#rep{} = Rep, 4) ->
UUID = couch_server:get_uuid(),
- SrcInfo = get_v4_endpoint(UserCtx, Rep#rep.source),
- TgtInfo = get_v4_endpoint(UserCtx, Rep#rep.target),
+ SrcInfo = get_v4_endpoint(Rep#rep.source),
+ TgtInfo = get_v4_endpoint(Rep#rep.target),
maybe_append_filters([UUID, SrcInfo, TgtInfo], Rep);
-replication_id(#rep{user_ctx = UserCtx} = Rep, 3) ->
+replication_id(#rep{} = Rep, 3) ->
UUID = couch_server:get_uuid(),
- Src = get_rep_endpoint(UserCtx, Rep#rep.source),
- Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
+ Src = get_rep_endpoint(Rep#rep.source),
+ Tgt = get_rep_endpoint(Rep#rep.target),
maybe_append_filters([UUID, Src, Tgt], Rep);
-replication_id(#rep{user_ctx = UserCtx} = Rep, 2) ->
+replication_id(#rep{} = Rep, 2) ->
{ok, HostName} = inet:gethostname(),
Port = case (catch mochiweb_socket_server:get(couch_httpd, port)) of
P when is_number(P) ->
@@ -64,14 +64,14 @@ replication_id(#rep{user_ctx = UserCtx} = Rep, 2) ->
% ... mochiweb_socket_server:get(https, port)
list_to_integer(config:get("httpd", "port", "5984"))
end,
- Src = get_rep_endpoint(UserCtx, Rep#rep.source),
- Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
+ Src = get_rep_endpoint(Rep#rep.source),
+ Tgt = get_rep_endpoint(Rep#rep.target),
maybe_append_filters([HostName, Port, Src, Tgt], Rep);
-replication_id(#rep{user_ctx = UserCtx} = Rep, 1) ->
+replication_id(#rep{} = Rep, 1) ->
{ok, HostName} = inet:gethostname(),
- Src = get_rep_endpoint(UserCtx, Rep#rep.source),
- Tgt = get_rep_endpoint(UserCtx, Rep#rep.target),
+ Src = get_rep_endpoint(Rep#rep.source),
+ Tgt = get_rep_endpoint(Rep#rep.target),
maybe_append_filters([HostName, Src, Tgt], Rep).
@@ -91,7 +91,7 @@ convert({BaseId, Ext} = Id) when is_list(BaseId), is_list(Ext) ->
% Private functions
maybe_append_filters(Base,
- #rep{source = Source, user_ctx = UserCtx, options = Options}) ->
+ #rep{source = Source, options = Options}) ->
Base2 = Base ++
case couch_replicator_filters:parse(Options) of
{ok, nil} ->
@@ -99,7 +99,7 @@ maybe_append_filters(Base,
{ok, {view, Filter, QueryParams}} ->
[Filter, QueryParams];
{ok, {user, {Doc, Filter}, QueryParams}} ->
- case couch_replicator_filters:fetch(Doc, Filter, Source, UserCtx) of
+ case couch_replicator_filters:fetch(Doc, Filter, Source) of
{ok, Code} ->
[Code, QueryParams];
{error, Error} ->
@@ -127,23 +127,19 @@ maybe_append_options(Options, RepOptions) ->
end, [], Options).
-get_rep_endpoint(_UserCtx, #httpdb{url=Url, headers=Headers}) ->
+get_rep_endpoint(#httpdb{url=Url, headers=Headers}) ->
DefaultHeaders = (#httpdb{})#httpdb.headers,
- {remote, Url, Headers -- DefaultHeaders};
-get_rep_endpoint(UserCtx, <<DbName/binary>>) ->
- {local, DbName, UserCtx}.
+ {remote, Url, Headers -- DefaultHeaders}.
-get_v4_endpoint(UserCtx, #httpdb{} = HttpDb) ->
- {remote, Url, Headers} = get_rep_endpoint(UserCtx, HttpDb),
+get_v4_endpoint(#httpdb{} = HttpDb) ->
+ {remote, Url, Headers} = get_rep_endpoint(HttpDb),
{{UserFromHeaders, _}, HeadersWithoutBasicAuth} =
couch_replicator_utils:remove_basic_auth_from_headers(Headers),
{UserFromUrl, Host, NonDefaultPort, Path} = get_v4_url_info(Url),
User = pick_defined_value([UserFromUrl, UserFromHeaders]),
OAuth = undefined, % Keep this to ensure checkpoints don't change
- {remote, User, Host, NonDefaultPort, Path, HeadersWithoutBasicAuth, OAuth};
-get_v4_endpoint(UserCtx, <<DbName/binary>>) ->
- {local, DbName, UserCtx}.
+ {remote, User, Host, NonDefaultPort, Path, HeadersWithoutBasicAuth, OAuth}.
pick_defined_value(Values) ->
@@ -201,7 +197,7 @@ replication_id_convert_test_() ->
http_v4_endpoint_test_() ->
[?_assertMatch({remote, User, Host, Port, Path, HeadersNoAuth, undefined},
- get_v4_endpoint(nil, #httpdb{url = Url, headers = Headers})) ||
+ get_v4_endpoint(#httpdb{url = Url, headers = Headers})) ||
{{User, Host, Port, Path, HeadersNoAuth}, {Url, Headers}} <- [
{
{undefined, "host", default, "/", []},
diff --git a/src/couch_replicator/src/couch_replicator_scheduler_job.erl b/src/couch_replicator/src/couch_replicator_scheduler_job.erl
index 412ff7d05..565a2bd97 100644
--- a/src/couch_replicator/src/couch_replicator_scheduler_job.erl
+++ b/src/couch_replicator/src/couch_replicator_scheduler_job.erl
@@ -40,8 +40,6 @@
]).
-import(couch_replicator_utils, [
- start_db_compaction_notifier/2,
- stop_db_compaction_notifier/1,
pp_rep_id/1
]).
@@ -75,8 +73,6 @@
workers,
stats = couch_replicator_stats:new(),
session_id,
- source_db_compaction_notifier = nil,
- target_db_compaction_notifier = nil,
source_monitor = nil,
target_monitor = nil,
source_seq = nil,
@@ -226,21 +222,6 @@ handle_call({report_seq_done, Seq, StatsInc}, From,
update_task(NewState),
{noreply, NewState}.
-handle_cast({db_compacted, DbName}, State) ->
- #rep_state{
- source = Source,
- target = Target
- } = State,
- SourceName = couch_replicator_utils:local_db_name(Source),
- TargetName = couch_replicator_utils:local_db_name(Target),
- case DbName of
- SourceName ->
- {ok, NewSource} = couch_db:reopen(Source),
- {noreply, State#rep_state{source = NewSource}};
- TargetName ->
- {ok, NewTarget} = couch_db:reopen(Target),
- {noreply, State#rep_state{target = NewTarget}}
- end;
handle_cast(checkpoint, State) ->
case do_checkpoint(State) of
@@ -412,8 +393,6 @@ terminate(Reason, State) ->
terminate_cleanup(State) ->
update_task(State),
- stop_db_compaction_notifier(State#rep_state.source_db_compaction_notifier),
- stop_db_compaction_notifier(State#rep_state.target_db_compaction_notifier),
couch_replicator_api_wrap:db_close(State#rep_state.source),
couch_replicator_api_wrap:db_close(State#rep_state.target).
@@ -572,16 +551,16 @@ init_state(Rep) ->
#rep{
id = {BaseId, _Ext},
source = Src0, target = Tgt,
- options = Options, user_ctx = UserCtx,
+ options = Options,
type = Type, view = View,
start_time = StartTime,
stats = Stats
} = Rep,
% Adjust minimum number of http source connections to 2 to avoid deadlock
Src = adjust_maxconn(Src0, BaseId),
- {ok, Source} = couch_replicator_api_wrap:db_open(Src, [{user_ctx, UserCtx}]),
+ {ok, Source} = couch_replicator_api_wrap:db_open(Src),
{CreateTargetParams} = get_value(create_target_params, Options, {[]}),
- {ok, Target} = couch_replicator_api_wrap:db_open(Tgt, [{user_ctx, UserCtx}],
+ {ok, Target} = couch_replicator_api_wrap:db_open(Tgt,
get_value(create_target, Options, false), CreateTargetParams),
{ok, SourceInfo} = couch_replicator_api_wrap:get_db_info(Source),
@@ -613,10 +592,6 @@ init_state(Rep) ->
src_starttime = get_value(<<"instance_start_time">>, SourceInfo),
tgt_starttime = get_value(<<"instance_start_time">>, TargetInfo),
session_id = couch_uuids:random(),
- source_db_compaction_notifier =
- start_db_compaction_notifier(Source, self()),
- target_db_compaction_notifier =
- start_db_compaction_notifier(Target, self()),
source_monitor = db_monitor(Source),
target_monitor = db_monitor(Target),
source_seq = SourceSeq,
diff --git a/src/couch_replicator/src/couch_replicator_utils.erl b/src/couch_replicator/src/couch_replicator_utils.erl
index b0d706953..ccf241324 100644
--- a/src/couch_replicator/src/couch_replicator_utils.erl
+++ b/src/couch_replicator/src/couch_replicator_utils.erl
@@ -14,11 +14,6 @@
-export([
parse_rep_doc/2,
- open_db/1,
- close_db/1,
- local_db_name/1,
- start_db_compaction_notifier/2,
- stop_db_compaction_notifier/1,
replication_id/2,
sum_stats/2,
is_deleted/1,
@@ -32,9 +27,6 @@
normalize_rep/1
]).
--export([
- handle_db_event/3
-]).
-include_lib("couch/include/couch_db.hrl").
-include("couch_replicator.hrl").
@@ -46,50 +38,6 @@
]).
-open_db(#httpdb{} = HttpDb) ->
- HttpDb;
-open_db(Db) ->
- DbName = couch_db:name(Db),
- UserCtx = couch_db:get_user_ctx(Db),
- {ok, NewDb} = couch_db:open(DbName, [{user_ctx, UserCtx}]),
- NewDb.
-
-
-close_db(#httpdb{}) ->
- ok;
-close_db(Db) ->
- couch_db:close(Db).
-
-
-local_db_name(#httpdb{}) ->
- undefined;
-local_db_name(Db) ->
- couch_db:name(Db).
-
-
-start_db_compaction_notifier(#httpdb{}, _) ->
- nil;
-start_db_compaction_notifier(Db, Server) ->
- DbName = couch_db:name(Db),
- {ok, Pid} = couch_event:link_listener(
- ?MODULE, handle_db_event, Server, [{dbname, DbName}]
- ),
- Pid.
-
-
-stop_db_compaction_notifier(nil) ->
- ok;
-stop_db_compaction_notifier(Listener) ->
- couch_event:stop_listener(Listener).
-
-
-handle_db_event(DbName, compacted, Server) ->
- gen_server:cast(Server, {db_compacted, DbName}),
- {ok, Server};
-handle_db_event(_DbName, _Event, Server) ->
- {ok, Server}.
-
-
rep_error_to_binary(Error) ->
couch_util:to_binary(error_reason(Error)).
@@ -289,14 +237,14 @@ normalize_rep_test_() ->
?_test(begin
EJson1 = {[
{<<"source">>, <<"http://host.com/source_db">>},
- {<<"target">>, <<"local">>},
+ {<<"target">>, <<"http://target.local/db">>},
{<<"doc_ids">>, [<<"a">>, <<"c">>, <<"b">>]},
{<<"other_field">>, <<"some_value">>}
]},
Rep1 = couch_replicator_docs:parse_rep_doc_without_id(EJson1),
EJson2 = {[
{<<"other_field">>, <<"unrelated">>},
- {<<"target">>, <<"local">>},
+ {<<"target">>, <<"http://target.local/db">>},
{<<"source">>, <<"http://host.com/source_db">>},
{<<"doc_ids">>, [<<"c">>, <<"a">>, <<"b">>]},
{<<"other_field2">>, <<"unrelated2">>}
diff --git a/src/couch_replicator/src/couch_replicator_worker.erl b/src/couch_replicator/src/couch_replicator_worker.erl
index ec98fa0f3..986c32c0a 100644
--- a/src/couch_replicator/src/couch_replicator_worker.erl
+++ b/src/couch_replicator/src/couch_replicator_worker.erl
@@ -28,18 +28,11 @@
% TODO: maybe make both buffer max sizes configurable
-define(DOC_BUFFER_BYTE_SIZE, 512 * 1024). % for remote targets
--define(DOC_BUFFER_LEN, 10). % for local targets, # of documents
-define(MAX_BULK_ATT_SIZE, 64 * 1024).
-define(MAX_BULK_ATTS_PER_DOC, 8).
-define(STATS_DELAY, 10000000). % 10 seconds (in microseconds)
-define(MISSING_DOC_RETRY_MSEC, 2000).
--import(couch_replicator_utils, [
- open_db/1,
- close_db/1,
- start_db_compaction_notifier/2,
- stop_db_compaction_notifier/1
-]).
-import(couch_util, [
to_binary/1,
get_value/3
@@ -62,8 +55,6 @@
pending_fetch = nil,
flush_waiter = nil,
stats = couch_replicator_stats:new(),
- source_db_compaction_notifier = nil,
- target_db_compaction_notifier = nil,
batch = #batch{}
}).
@@ -71,14 +62,7 @@
start_link(Cp, #httpdb{} = Source, Target, ChangesManager, MaxConns) ->
gen_server:start_link(
- ?MODULE, {Cp, Source, Target, ChangesManager, MaxConns}, []);
-
-start_link(Cp, Source, Target, ChangesManager, _MaxConns) ->
- Pid = spawn_link(fun() ->
- erlang:put(last_stats_report, os:timestamp()),
- queue_fetch_loop(Source, Target, Cp, Cp, ChangesManager)
- end),
- {ok, Pid}.
+ ?MODULE, {Cp, Source, Target, ChangesManager, MaxConns}, []).
init({Cp, Source, Target, ChangesManager, MaxConns}) ->
@@ -92,12 +76,8 @@ init({Cp, Source, Target, ChangesManager, MaxConns}) ->
cp = Cp,
max_parallel_conns = MaxConns,
loop = LoopPid,
- source = open_db(Source),
- target = open_db(Target),
- source_db_compaction_notifier =
- start_db_compaction_notifier(Source, self()),
- target_db_compaction_notifier =
- start_db_compaction_notifier(Target, self())
+ source = Source,
+ target = Target
},
{ok, State}.
@@ -141,24 +121,6 @@ handle_call(flush, {Pid, _} = From,
{noreply, State2#state{flush_waiter = From}}.
-handle_cast({db_compacted, DbName} = Msg, #state{} = State) ->
- #state{
- source = Source,
- target = Target
- } = State,
- SourceName = couch_replicator_utils:local_db_name(Source),
- TargetName = couch_replicator_utils:local_db_name(Target),
- case DbName of
- SourceName ->
- {ok, NewSource} = couch_db:reopen(Source),
- {noreply, State#state{source = NewSource}};
- TargetName ->
- {ok, NewTarget} = couch_db:reopen(Target),
- {noreply, State#state{target = NewTarget}};
- _Else ->
- {stop, {unexpected_async_call, Msg}, State}
- end;
-
handle_cast(Msg, State) ->
{stop, {unexpected_async_call, Msg}, State}.
@@ -213,11 +175,8 @@ handle_info({'EXIT', Pid, Reason}, State) ->
{stop, {process_died, Pid, Reason}, State}.
-terminate(_Reason, State) ->
- close_db(State#state.source),
- close_db(State#state.target),
- stop_db_compaction_notifier(State#state.source_db_compaction_notifier),
- stop_db_compaction_notifier(State#state.target_db_compaction_notifier).
+terminate(_Reason, _State) ->
+ ok.
format_status(_Opt, [_PDict, State]) ->
#state{
@@ -253,20 +212,10 @@ queue_fetch_loop(Source, Target, Parent, Cp, ChangesManager) ->
ok = gen_server:call(Cp, {report_seq_done, ReportSeq, Stats}, infinity),
queue_fetch_loop(Source, Target, Parent, Cp, ChangesManager);
{changes, ChangesManager, Changes, ReportSeq} ->
- Target2 = open_db(Target),
- {IdRevs, Stats0} = find_missing(Changes, Target2),
- case Source of
- #httpdb{} ->
- ok = gen_server:call(Parent, {add_stats, Stats0}, infinity),
- remote_process_batch(IdRevs, Parent),
- {ok, Stats} = gen_server:call(Parent, flush, infinity);
- _Db ->
- Source2 = open_db(Source),
- Stats = local_process_batch(
- IdRevs, Cp, Source2, Target2, #batch{}, Stats0),
- close_db(Source2)
- end,
- close_db(Target2),
+ {IdRevs, Stats0} = find_missing(Changes, Target),
+ ok = gen_server:call(Parent, {add_stats, Stats0}, infinity),
+ remote_process_batch(IdRevs, Parent),
+ {ok, Stats} = gen_server:call(Parent, flush, infinity),
ok = gen_server:call(Cp, {report_seq_done, ReportSeq, Stats}, infinity),
erlang:put(last_stats_report, os:timestamp()),
couch_log:debug("Worker reported completion of seq ~p", [ReportSeq]),
@@ -274,32 +223,6 @@ queue_fetch_loop(Source, Target, Parent, Cp, ChangesManager) ->
end.
-local_process_batch([], _Cp, _Src, _Tgt, #batch{docs = []}, Stats) ->
- Stats;
-
-local_process_batch([], Cp, Source, Target, #batch{docs = Docs, size = Size}, Stats) ->
- case Target of
- #httpdb{} ->
- couch_log:debug("Worker flushing doc batch of size ~p bytes", [Size]);
- _Db ->
- couch_log:debug("Worker flushing doc batch of ~p docs", [Size])
- end,
- Stats2 = flush_docs(Target, Docs),
- Stats3 = couch_replicator_utils:sum_stats(Stats, Stats2),
- local_process_batch([], Cp, Source, Target, #batch{}, Stats3);
-
-local_process_batch([IdRevs | Rest], Cp, Source, Target, Batch, Stats) ->
- {ok, {_, DocList, Stats2, _}} = fetch_doc(
- Source, IdRevs, fun local_doc_handler/2, {Target, [], Stats, Cp}),
- {Batch2, Stats3} = lists:foldl(
- fun(Doc, {Batch0, Stats0}) ->
- {Batch1, S} = maybe_flush_docs(Target, Batch0, Doc),
- {Batch1, couch_replicator_utils:sum_stats(Stats0, S)}
- end,
- {Batch, Stats2}, DocList),
- local_process_batch(Rest, Cp, Source, Target, Batch2, Stats3).
-
-
remote_process_batch([], _Parent) ->
ok;
@@ -319,10 +242,8 @@ remote_process_batch([{Id, Revs, PAs} | Rest], Parent) ->
spawn_doc_reader(Source, Target, FetchParams) ->
Parent = self(),
spawn_link(fun() ->
- Source2 = open_db(Source),
fetch_doc(
- Source2, FetchParams, fun remote_doc_handler/2, {Parent, Target}),
- close_db(Source2)
+ Source, FetchParams, fun remote_doc_handler/2, {Parent, Target})
end).
@@ -350,29 +271,6 @@ fetch_doc(Source, {Id, Revs, PAs}, DocHandler, Acc) ->
end.
-local_doc_handler({ok, Doc}, {Target, DocList, Stats, Cp}) ->
- Stats2 = couch_replicator_stats:increment(docs_read, Stats),
- case batch_doc(Doc) of
- true ->
- {ok, {Target, [Doc | DocList], Stats2, Cp}};
- false ->
- couch_log:debug("Worker flushing doc with attachments", []),
- Target2 = open_db(Target),
- Success = (flush_doc(Target2, Doc) =:= ok),
- close_db(Target2),
- Stats3 = case Success of
- true ->
- couch_replicator_stats:increment(docs_written, Stats2);
- false ->
- couch_replicator_stats:increment(doc_write_failures, Stats2)
- end,
- Stats4 = maybe_report_stats(Cp, Stats3),
- {ok, {Target, DocList, Stats4, Cp}}
- end;
-local_doc_handler(_, Acc) ->
- {ok, Acc}.
-
-
remote_doc_handler({ok, #doc{atts = []} = Doc}, {Parent, _} = Acc) ->
ok = gen_server:call(Parent, {batch_doc, Doc}, infinity),
{ok, Acc};
@@ -383,9 +281,7 @@ remote_doc_handler({ok, Doc}, {Parent, Target} = Acc) ->
% convenient to call it ASAP to avoid ibrowse inactivity timeouts.
Stats = couch_replicator_stats:new([{docs_read, 1}]),
couch_log:debug("Worker flushing doc with attachments", []),
- Target2 = open_db(Target),
- Success = (flush_doc(Target2, Doc) =:= ok),
- close_db(Target2),
+ Success = (flush_doc(Target, Doc) =:= ok),
{Result, Stats2} = case Success of
true ->
{{ok, Acc}, couch_replicator_stats:increment(docs_written, Stats)};
@@ -402,17 +298,13 @@ spawn_writer(Target, #batch{docs = DocList, size = Size}) ->
case {Target, Size > 0} of
{#httpdb{}, true} ->
couch_log:debug("Worker flushing doc batch of size ~p bytes", [Size]);
- {_Db, true} ->
- couch_log:debug("Worker flushing doc batch of ~p docs", [Size]);
_ ->
ok
end,
Parent = self(),
spawn_link(
fun() ->
- Target2 = open_db(Target),
- Stats = flush_docs(Target2, DocList),
- close_db(Target2),
+ Stats = flush_docs(Target, DocList),
ok = gen_server:call(Parent, {add_stats, Stats}, infinity)
end).
@@ -462,17 +354,6 @@ maybe_flush_docs(#httpdb{} = Target, Batch, Doc) ->
Stats = couch_replicator_stats:new(),
{#batch{docs = [JsonDoc | DocAcc], size = SizeAcc2}, Stats}
end
- end;
-
-maybe_flush_docs(Target, #batch{docs = DocAcc, size = SizeAcc}, Doc) ->
- case SizeAcc + 1 of
- SizeAcc2 when SizeAcc2 >= ?DOC_BUFFER_LEN ->
- couch_log:debug("Worker flushing doc batch of ~p docs", [SizeAcc2]),
- Stats = flush_docs(Target, [Doc | DocAcc]),
- {#batch{}, Stats};
- SizeAcc2 ->
- Stats = couch_replicator_stats:new(),
- {#batch{docs = [Doc | DocAcc], size = SizeAcc2}, Stats}
end.
diff --git a/src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl b/src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl
index 7fe84d2d9..ac4bb84f3 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_attachments_too_large.erl
@@ -33,7 +33,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
attachment_too_large_replication_test_() ->
- Pairs = [{local, remote}, {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Attachment size too large replication tests",
{
@@ -96,8 +96,6 @@ delete_db(DbName) ->
ok = couch_server:delete(DbName, [?ADMIN_CTX]).
-db_url(local, DbName) ->
- DbName;
db_url(remote, DbName) ->
Addr = config:get("httpd", "bind_address", "127.0.0.1"),
Port = mochiweb_socket_server:get(couch_httpd, port),
diff --git a/src/couch_replicator/test/eunit/couch_replicator_compact_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_compact_tests.erl
index 7cc530c19..eb3fc82c5 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_compact_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_compact_tests.erl
@@ -33,8 +33,6 @@ setup() ->
ok = couch_db:close(Db),
DbName.
-setup(local) ->
- setup();
setup(remote) ->
{remote, setup()};
setup({A, B}) ->
@@ -56,8 +54,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
ok = test_util:stop_couch(Ctx).
compact_test_() ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Compaction during replication tests",
{
diff --git a/src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl
index d34e9f020..70b25a31b 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_filtered_tests.erl
@@ -60,8 +60,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
ok = test_util:stop_couch(Ctx).
filtered_replication_test_() ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Filtered replication tests",
{
@@ -72,8 +71,7 @@ filtered_replication_test_() ->
}.
query_filtered_replication_test_() ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Filtered with query replication tests",
{
@@ -84,7 +82,7 @@ query_filtered_replication_test_() ->
}.
view_filtered_replication_test_() ->
- Pairs = [{local, local}],
+ Pairs = [{remote, remote}],
{
"Filtered with a view replication tests",
{
@@ -236,8 +234,6 @@ create_docs(DbName) ->
delete_db(DbName) ->
ok = couch_server:delete(DbName, [?ADMIN_CTX]).
-db_url(local, DbName) ->
- DbName;
db_url(remote, DbName) ->
Addr = config:get("httpd", "bind_address", "127.0.0.1"),
Port = mochiweb_socket_server:get(couch_httpd, port),
diff --git a/src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl
index 70eda0566..1447acfa7 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_id_too_long_tests.erl
@@ -33,8 +33,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
id_too_long_replication_test_() ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Doc id too long tests",
{
@@ -86,8 +85,6 @@ delete_db(DbName) ->
ok = couch_server:delete(DbName, [?ADMIN_CTX]).
-db_url(local, DbName) ->
- DbName;
db_url(remote, DbName) ->
Addr = config:get("httpd", "bind_address", "127.0.0.1"),
Port = mochiweb_socket_server:get(couch_httpd, port),
diff --git a/src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl
index b9adf5c4b..27c89a0cd 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_large_atts_tests.erl
@@ -33,8 +33,6 @@ setup() ->
ok = couch_db:close(Db),
DbName.
-setup(local) ->
- setup();
setup(remote) ->
{remote, setup()};
setup({A, B}) ->
@@ -58,8 +56,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
ok = test_util:stop_couch(Ctx).
large_atts_test_() ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Replicate docs with large attachments",
{
diff --git a/src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl
index eee5b1647..be1bfa344 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_many_leaves_tests.erl
@@ -37,8 +37,7 @@ setup() ->
ok = couch_db:close(Db),
DbName.
-setup(local) ->
- setup();
+
setup(remote) ->
{remote, setup()};
setup({A, B}) ->
@@ -60,8 +59,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
ok = test_util:stop_couch(Ctx).
docs_with_many_leaves_test_() ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Replicate documents with many leaves",
{
diff --git a/src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl
index c1681781f..ff08b5ee5 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_missing_stubs_tests.erl
@@ -30,8 +30,6 @@ setup() ->
ok = couch_db:close(Db),
DbName.
-setup(local) ->
- setup();
setup(remote) ->
{remote, setup()};
setup({A, B}) ->
@@ -53,8 +51,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
ok = test_util:stop_couch(Ctx).
missing_stubs_test_() ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Replicate docs with missing stubs (COUCHDB-1365)",
{
diff --git a/src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl
index a7f4c5df3..7d92bdcb1 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_selector_tests.erl
@@ -31,8 +31,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
ok = test_util:stop_couch(Ctx).
selector_replication_test_() ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Selector filtered replication tests",
{
@@ -113,8 +112,6 @@ create_docs(DbName) ->
delete_db(DbName) ->
ok = couch_server:delete(DbName, [?ADMIN_CTX]).
-db_url(local, DbName) ->
- DbName;
db_url(remote, DbName) ->
Addr = config:get("httpd", "bind_address", "127.0.0.1"),
Port = mochiweb_socket_server:get(couch_httpd, port),
diff --git a/src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl b/src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl
index af3a285f5..8aebbe151 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_small_max_request_size_target.erl
@@ -19,9 +19,6 @@ setup() ->
DbName.
-setup(local) ->
- setup();
-
setup(remote) ->
{remote, setup()};
@@ -47,7 +44,7 @@ teardown(_, {Ctx, {Source, Target}}) ->
reduce_max_request_size_test_() ->
- Pairs = [{local, remote}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"Replicate docs when target has a small max_http_request_size",
{
diff --git a/src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl b/src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl
index c2fcf8bf1..8e4a21dbb 100644
--- a/src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl
+++ b/src/couch_replicator/test/eunit/couch_replicator_use_checkpoints_tests.erl
@@ -51,8 +51,6 @@ setup() ->
ok = couch_db:close(Db),
DbName.
-setup(local) ->
- setup();
setup(remote) ->
{remote, setup()};
setup({_, Fun, {A, B}}) ->
@@ -88,8 +86,7 @@ use_checkpoints_test_() ->
}.
use_checkpoints_tests(UseCheckpoints, Fun) ->
- Pairs = [{local, local}, {local, remote},
- {remote, local}, {remote, remote}],
+ Pairs = [{remote, remote}],
{
"use_checkpoints: " ++ atom_to_list(UseCheckpoints),
{
diff --git a/test/elixir/test/replication_test.exs b/test/elixir/test/replication_test.exs
index a6e1303e6..6d4360d88 100644
--- a/test/elixir/test/replication_test.exs
+++ b/test/elixir/test/replication_test.exs
@@ -9,9 +9,6 @@ defmodule ReplicationTest do
# TODO: Parameterize these
@admin_account "adm:pass"
@db_pairs_prefixes [
- {"local-to-local", "", ""},
- {"remote-to-local", "http://127.0.0.1:15984/", ""},
- {"local-to-remote", "", "http://127.0.0.1:15984/"},
{"remote-to-remote", "http://127.0.0.1:15984/", "http://127.0.0.1:15984/"}
]
@@ -21,20 +18,11 @@ defmodule ReplicationTest do
@moduletag :skip_on_jenkins
- test "source database does not exist" do
- name = random_db_name()
- check_not_found(name <> "_src", name <> "_tgt")
- end
-
- test "source database not found with path - COUCHDB-317" do
- name = random_db_name()
- check_not_found(name <> "_src", name <> "_tgt")
- end
-
test "source database not found with host" do
name = random_db_name()
- url = "http://127.0.0.1:15984/" <> name <> "_src"
- check_not_found(url, name <> "_tgt")
+ src_url = "http://127.0.0.1:15984/" <> name <> "_src"
+ tgt_url = "http://127.0.0.1:15984/" <> name <> "_tgt"
+ check_not_found(src_url, tgt_url)
end
def check_not_found(src, tgt) do
@@ -55,7 +43,9 @@ defmodule ReplicationTest do
doc = %{"_id" => "doc1"}
[doc] = save_docs(src_db_name, [doc])
- result = replicate(src_db_name, "http://127.0.0.1:15984/" <> tgt_db_name)
+ repl_src = "http://127.0.0.1:15984/" <> src_db_name
+ repl_tgt = "http://127.0.0.1:15984/" <> tgt_db_name
+ result = replicate(repl_src, repl_tgt)
assert result["ok"]
assert is_list(result["history"])
history = Enum.at(result["history"], 0)
@@ -79,7 +69,9 @@ defmodule ReplicationTest do
[doc] = save_docs(src_db_name, [doc])
- result = replicate(src_db_name, "http://127.0.0.1:15984/" <> tgt_db_name)
+ repl_src = "http://127.0.0.1:15984/" <> src_db_name
+ repl_tgt = "http://127.0.0.1:15984/" <> tgt_db_name
+ result = replicate(repl_src, repl_tgt)
assert result["ok"]
assert is_list(result["history"])
@@ -127,7 +119,8 @@ defmodule ReplicationTest do
repl_body = %{:continuous => true, :create_target => true}
repl_src = "http://127.0.0.1:15984/" <> src_db_name
- result = replicate(repl_src, tgt_db_name, body: repl_body)
+ repl_tgt = "http://127.0.0.1:15984/" <> tgt_db_name
+ result = replicate(repl_src, repl_tgt, body: repl_body)
assert result["ok"]
assert is_binary(result["_local_id"])
@@ -167,8 +160,9 @@ defmodule ReplicationTest do
save_docs(src_db_name, make_docs(1..6))
repl_src = "http://127.0.0.1:15984/" <> src_db_name
+ repl_tgt = "http://127.0.0.1:15984/" <> tgt_db_name
repl_body = %{"continuous" => true}
- result = replicate(repl_src, tgt_db_name, body: repl_body)
+ result = replicate(repl_src, repl_tgt, body: repl_body)
assert result["ok"]
assert is_binary(result["_local_id"])
@@ -282,7 +276,9 @@ defmodule ReplicationTest do
end
end
- result = replicate(src_prefix <> src_db_name, tgt_prefix <> tgt_db_name)
+ repl_src = src_prefix <> src_db_name
+ repl_tgt = tgt_prefix <> tgt_db_name
+ result = replicate(repl_src, repl_tgt)
assert result["ok"]
src_info =
diff --git a/test/javascript/tests/purge.js b/test/javascript/tests/purge.js
index 0c11d9ad8..36ea955d6 100644
--- a/test/javascript/tests/purge.js
+++ b/test/javascript/tests/purge.js
@@ -114,7 +114,9 @@ couchTests.purge = function(debug) {
var docB = {_id:"test", a:2};
dbA.save(docA);
dbB.save(docB);
- CouchDB.replicate(dbA.name, dbB.name);
+ var rep_src = CouchDB.protocol + CouchDB.host + "/" + dbA.name;
+ var rep_tgt = CouchDB.protocol + CouchDB.host + "/" + dbB.name;
+ CouchDB.replicate(rep_src, rep_tgt);
var xhr = CouchDB.request("POST", "/" + dbB.name + "/_purge", {
body: JSON.stringify({"test":[docA._rev]})
});
diff --git a/test/javascript/tests/replication.js b/test/javascript/tests/replication.js
index ba586b409..7b2d379e3 100644
--- a/test/javascript/tests/replication.js
+++ b/test/javascript/tests/replication.js
@@ -22,18 +22,6 @@ couchTests.replication = function(debug) {
var dbPairsPrefixes = [
{
- source: "",
- target: ""
- },
- {
- source: CouchDB.protocol + host + "/",
- target: ""
- },
- {
- source: "",
- target: CouchDB.protocol + host + "/"
- },
- {
source: CouchDB.protocol + host + "/",
target: CouchDB.protocol + host + "/"
}
@@ -585,25 +573,10 @@ couchTests.replication = function(debug) {
//TEquals(sourceInfo.update_seq, repResult.source_last_seq);
}
-
- // test error when source database does not exist
- try {
- CouchDB.replicate("foobar", "test_suite_db");
- T(false, "should have failed with db_not_found error");
- } catch (x) {
- TEquals("db_not_found", x.error);
- }
-
- // validate COUCHDB-317
- try {
- CouchDB.replicate("/foobar", "test_suite_db");
- T(false, "should have failed with db_not_found error");
- } catch (x) {
- TEquals("db_not_found", x.error);
- }
-
try {
- CouchDB.replicate(CouchDB.protocol + host + "/foobar", "test_suite_db");
+ var rep_src = CouchDB.protocol + host + "/foobar";
+ var rep_tgt = CouchDB.protocol + host + "/test_suite_db";
+ CouchDB.replicate(rep_src, rep_tgt);
T(false, "should have failed with db_not_found error");
} catch (x) {
TEquals("db_not_found", x.error);
@@ -1591,18 +1564,6 @@ couchTests.replication = function(debug) {
dbPairsPrefixes = [
{
- source: "",
- target: ""
- },
- {
- source: CouchDB.protocol + host + "/",
- target: ""
- },
- {
- source: "",
- target: CouchDB.protocol + "joe:erly@" + host + "/"
- },
- {
source: CouchDB.protocol + host + "/",
target: CouchDB.protocol + "joe:erly@" + host + "/"
}
@@ -1660,18 +1621,6 @@ couchTests.replication = function(debug) {
// case 2) user triggering the replication is not a reader (nor admin) of the source DB
dbPairsPrefixes = [
{
- source: "",
- target: ""
- },
- {
- source: CouchDB.protocol + "joe:erly@" + host + "/",
- target: ""
- },
- {
- source: "",
- target: CouchDB.protocol + host + "/"
- },
- {
source: CouchDB.protocol + "joe:erly@" + host + "/",
target: CouchDB.protocol + host + "/"
}
@@ -1745,7 +1694,7 @@ couchTests.replication = function(debug) {
TEquals(true, sourceDb.save(doc).ok);
repResult = CouchDB.replicate(
- sourceDb.name,
+ CouchDB.protocol + host + "/" + sourceDb.name,
CouchDB.protocol + host + "/" + targetDb.name
);
TEquals(true, repResult.ok);
@@ -1768,7 +1717,7 @@ couchTests.replication = function(debug) {
TEquals(true, sourceDb.save(doc).ok);
repResult = CouchDB.replicate(
- sourceDb.name,
+ CouchDB.protocol + host + "/" + sourceDb.name,
CouchDB.protocol + host + "/" + targetDb.name
);
TEquals(true, repResult.ok);
@@ -1819,7 +1768,7 @@ couchTests.replication = function(debug) {
repResult = CouchDB.replicate(
CouchDB.protocol + host + "/" + sourceDb.name,
- targetDb.name,
+ CouchDB.protocol + host + "/" + sourceDb.name,
{
body: {
continuous: true,
@@ -1880,7 +1829,7 @@ couchTests.replication = function(debug) {
repResult = CouchDB.replicate(
CouchDB.protocol + host + "/" + sourceDb.name,
- targetDb.name,
+ CouchDB.protocol + host + "/" + targetDb.name,
{
body: {
continuous: true
diff --git a/test/javascript/tests/rev_stemming.js b/test/javascript/tests/rev_stemming.js
index 238868f60..13fbdc407 100644
--- a/test/javascript/tests/rev_stemming.js
+++ b/test/javascript/tests/rev_stemming.js
@@ -69,7 +69,9 @@ couchTests.rev_stemming = function(debug) {
// If you replicate after you make more edits than the limit, you'll
// cause a spurious edit conflict.
- CouchDB.replicate(db.name, dbB.name);
+ var rep_src = CouchDB.protocol + CouchDB.host + "/" + db.name;
+ var rep_tgt = CouchDB.protocol + CouchDB.host + "/" + dbB.name;
+ CouchDB.replicate(rep_src, rep_tgt);
var docB1 = dbB.open("foo",{conflicts:true})
T(docB1._conflicts == null);
@@ -79,7 +81,7 @@ couchTests.rev_stemming = function(debug) {
}
// one less edit than limit, no conflict
- CouchDB.replicate(db.name, dbB.name);
+ CouchDB.replicate(rep_src, rep_tgt);
var docB1 = dbB.open("foo",{conflicts:true})
T(docB1._conflicts == null);
@@ -89,7 +91,7 @@ couchTests.rev_stemming = function(debug) {
T(db.save(doc).ok);
}
- CouchDB.replicate(db.name, dbB.name);
+ CouchDB.replicate(rep_src, rep_tgt);
var docB2 = dbB.open("foo",{conflicts:true});
diff --git a/test/javascript/tests/users_db.js b/test/javascript/tests/users_db.js
index 20be325ca..7d9682185 100644
--- a/test/javascript/tests/users_db.js
+++ b/test/javascript/tests/users_db.js
@@ -68,14 +68,13 @@ couchTests.users_db = function(debug) {
CouchDB.logout();
// ok, now create a conflicting edit on the jchris doc, and make sure there's no login.
- // (use replication to create the conflict) - need 2 be admin
CouchDB.login("jan", "apple");
- CouchDB.replicate(usersDb.name, usersDbAlt.name);
- // save in one DB
+ // save using new_edits=false
var jchrisUser2 = JSON.parse(JSON.stringify(jchrisUserDoc));
jchrisUser2.foo = "bar";
+ jchrisUser2._rev = "1-7a28b8e96ee17f723ebc1e9f89640783";
- T(usersDb.save(jchrisUser2).ok);
+ T(usersDb.save(jchrisUser2, {new_edits: false}).ok);
try {
usersDb.save(jchrisUserDoc);
T(false && "should be an update conflict");
@@ -83,12 +82,6 @@ couchTests.users_db = function(debug) {
T(true);
}
- // then in the other
- var jchrisUser3 = JSON.parse(JSON.stringify(jchrisUserDoc));
- jchrisUser3.foo = "barrrr";
- T(usersDbAlt.save(jchrisUser3).ok);
- CouchDB.replicate(usersDbAlt.name, usersDb.name); // now we should have a conflict
-
var jchrisWithConflict = usersDb.open(jchrisUserDoc._id, {conflicts : true});
T(jchrisWithConflict._conflicts.length == 1);
CouchDB.logout();
diff --git a/test/javascript/tests/view_conflicts.js b/test/javascript/tests/view_conflicts.js
index b1c938c61..3b5a9509e 100644
--- a/test/javascript/tests/view_conflicts.js
+++ b/test/javascript/tests/view_conflicts.js
@@ -24,14 +24,16 @@ couchTests.view_conflicts = function(debug) {
var docA = {_id: "foo", bar: 42};
T(dbA.save(docA).ok);
- CouchDB.replicate(dbA.name, dbB.name);
+ var rep_src = CouchDB.protocol + CouchDB.host + "/" + dbA.name;
+ var rep_tgt = CouchDB.protocol + CouchDB.host + "/" + dbB.name;
+ CouchDB.replicate(rep_src, rep_tgt);
var docB = dbB.open("foo");
docB.bar = 43;
dbB.save(docB);
docA.bar = 41;
dbA.save(docA);
- CouchDB.replicate(dbA.name, dbB.name);
+ CouchDB.replicate(rep_src, rep_tgt);
var doc = dbB.open("foo", {conflicts: true});
T(doc._conflicts.length == 1);
diff --git a/test/javascript/tests/view_include_docs.js b/test/javascript/tests/view_include_docs.js
index cefc2cf90..270defbb5 100644
--- a/test/javascript/tests/view_include_docs.js
+++ b/test/javascript/tests/view_include_docs.js
@@ -169,8 +169,10 @@ couchTests.view_include_docs = function(debug) {
var doc2 = {_id: "bar", value: 2, str: "2"};
TEquals(true, dbA.save(doc2).ok);
-
- TEquals(true, CouchDB.replicate(dbA.name, dbB.name).ok);
+
+ var rep_src = CouchDB.protocol + CouchDB.host + "/" + dbA.name;
+ var rep_tgt = CouchDB.protocol + CouchDB.host + "/" + dbB.name;
+ TEquals(true, CouchDB.replicate(rep_src, rep_tgt).ok);
doc1b = dbB.open("foo", {conflicts: true});
TEquals(true, doc1b._conflicts instanceof Array);