Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 5853D200C40 for ; Thu, 23 Mar 2017 17:33:34 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 56790160B83; Thu, 23 Mar 2017 16:33:34 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 4E359160B68 for ; Thu, 23 Mar 2017 17:33:33 +0100 (CET) Received: (qmail 37663 invoked by uid 500); 23 Mar 2017 16:33:32 -0000 Mailing-List: contact commits-help@couchdb.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@couchdb.apache.org Delivered-To: mailing list commits@couchdb.apache.org Received: (qmail 37654 invoked by uid 99); 23 Mar 2017 16:33:32 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 23 Mar 2017 16:33:32 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 57CC1DFB31; Thu, 23 Mar 2017 16:33:32 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: davisp@apache.org To: commits@couchdb.apache.org Message-Id: <3bc64cfbca2b4028a8ed229b2aaceeae@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: fabric commit: updated refs/heads/COUCHDB-3288-remove-public-db-record to 455ade2 [Forced Update!] Date: Thu, 23 Mar 2017 16:33:32 +0000 (UTC) archived-at: Thu, 23 Mar 2017 16:33:34 -0000 Repository: couchdb-fabric Updated Branches: refs/heads/COUCHDB-3288-remove-public-db-record 03a54cda6 -> 455ade2ff (forced update) Remove public db record COUCHDB-3288 Project: http://git-wip-us.apache.org/repos/asf/couchdb-fabric/repo Commit: http://git-wip-us.apache.org/repos/asf/couchdb-fabric/commit/455ade2f Tree: http://git-wip-us.apache.org/repos/asf/couchdb-fabric/tree/455ade2f Diff: http://git-wip-us.apache.org/repos/asf/couchdb-fabric/diff/455ade2f Branch: refs/heads/COUCHDB-3288-remove-public-db-record Commit: 455ade2ffbcbcaf73acb4064a8c270e369a165bd Parents: dcfeda2 Author: Paul J. Davis Authored: Wed Feb 1 16:36:38 2017 -0600 Committer: Paul J. Davis Committed: Thu Mar 23 11:32:44 2017 -0500 ---------------------------------------------------------------------- include/couch_db_tmp.hrl | 296 ------------------------------------------ src/fabric.erl | 12 +- src/fabric_rpc.erl | 113 +++------------- src/fabric_util.erl | 3 +- 4 files changed, 24 insertions(+), 400 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/couchdb-fabric/blob/455ade2f/include/couch_db_tmp.hrl ---------------------------------------------------------------------- diff --git a/include/couch_db_tmp.hrl b/include/couch_db_tmp.hrl deleted file mode 100644 index cd3a047..0000000 --- a/include/couch_db_tmp.hrl +++ /dev/null @@ -1,296 +0,0 @@ -% Licensed under the Apache License, Version 2.0 (the "License"); you may not -% use this file except in compliance with the License. You may obtain a copy of -% the License at -% -% http://www.apache.org/licenses/LICENSE-2.0 -% -% Unless required by applicable law or agreed to in writing, software -% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT -% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the -% License for the specific language governing permissions and limitations under -% the License. - --define(LOCAL_DOC_PREFIX, "_local/"). --define(DESIGN_DOC_PREFIX0, "_design"). --define(DESIGN_DOC_PREFIX, "_design/"). - --define(MIN_STR, <<"">>). --define(MAX_STR, <<255>>). % illegal utf string - --define(JSON_ENCODE(V), couch_util:json_encode(V)). --define(JSON_DECODE(V), couch_util:json_decode(V)). - --define(b2l(V), binary_to_list(V)). --define(l2b(V), list_to_binary(V)). - --define(DEFAULT_ATTACHMENT_CONTENT_TYPE, <<"application/octet-stream">>). - --define(LOG_DEBUG(Format, Args), couch_log:debug(Format, Args)). --define(LOG_INFO(Format, Args), couch_log:notice(Format, Args)). --define(LOG_ERROR(Format, Args), couch_log:error(Format, Args)). - --record(rev_info, - { - rev, - seq = 0, - deleted = false, - body_sp = nil % stream pointer - }). - --record(doc_info, - { - id = <<"">>, - high_seq = 0, - revs = [] % rev_info - }). - --record(full_doc_info, - {id = <<"">>, - update_seq = 0, - deleted = false, - data_size = 0, - rev_tree = [] - }). - --record(httpd, - {mochi_req, - peer, - method, - path_parts, - db_url_handlers, - user_ctx, - req_body = undefined, - design_url_handlers, - auth, - default_fun, - url_handlers - }). - - --record(doc, - { - id = <<"">>, - revs = {0, []}, - - % the json body object. - body = {[]}, - - atts = [], % attachments - - deleted = false, - - % key/value tuple of meta information, provided when using special options: - % couch_db:open_doc(Db, Id, Options). - meta = [] - }). - - --record(att, - { - name, - type, - att_len, - disk_len, % length of the attachment in its identity form - % (that is, without a content encoding applied to it) - % differs from att_len when encoding /= identity - md5= <<>>, - revpos=0, - data, - encoding=identity % currently supported values are: - % identity, gzip - % additional values to support in the future: - % deflate, compress - }). - - --record(user_ctx, - { - name=null, - roles=[], - handler - }). - -% This should be updated anytime a header change happens that requires more -% than filling in new defaults. -% -% As long the changes are limited to new header fields (with inline -% defaults) added to the end of the record, then there is no need to increment -% the disk revision number. -% -% if the disk revision is incremented, then new upgrade logic will need to be -% added to couch_db_updater:init_db. - --define(LATEST_DISK_VERSION, 5). - --record(db_header, - {disk_version = ?LATEST_DISK_VERSION, - update_seq = 0, - unused = 0, - id_tree_state = nil, - seq_tree_state = nil, - local_tree_state = nil, - purge_seq = 0, - purged_docs = nil, - security_ptr = nil, - revs_limit = 1000 - }). - --record(db, - {main_pid = nil, - update_pid = nil, - compactor_pid = nil, - instance_start_time, % number of microsecs since jan 1 1970 as a binary string - fd, - fd_monitor, - header = #db_header{}, - committed_update_seq, - id_tree, - seq_tree, - local_tree, - update_seq, - name, - filepath, - validate_doc_funs = undefined, - security = [], - security_ptr = nil, - user_ctx = #user_ctx{}, - waiting_delayed_commit = nil, - revs_limit = 1000, - fsync_options = [], - is_sys_db = false - }). - - --record(view_query_args, { - start_key, - end_key, - start_docid = ?MIN_STR, - end_docid = ?MAX_STR, - - direction = fwd, - inclusive_end=true, % aka a closed-interval - - limit = 10000000000, % Huge number to simplify logic - skip = 0, - - group_level = 0, - - view_type = nil, - include_docs = false, - stale = false, - multi_get = false, - callback = nil, - list = nil, - keys = nil, - sorted = true, - extra = [] -}). - --record(view_fold_helper_funs, { - reduce_count, - passed_end, - start_response, - send_row -}). - --record(reduce_fold_helper_funs, { - start_response, - send_row -}). - --record(extern_resp_args, { - code = 200, - stop = false, - data = <<>>, - ctype = "application/json", - headers = [], - json = nil -}). - --record(group, { - sig=nil, - dbname, - fd=nil, - name, - def_lang, - design_options=[], - views, - id_btree=nil, - current_seq=0, - purge_seq=0, - query_server=nil, - waiting_delayed_commit=nil, - atts=[] - }). - --record(view, - {id_num, - map_names=[], - def, - btree=nil, - reduce_funs=[], - dbcopies=[], - options=[] - }). - --record(index_header, - {seq=0, - purge_seq=0, - id_btree_state=nil, - view_states=nil - }). - --record(http_db, { - url, - auth = [], - resource = "", - headers = [ - {"User-Agent", "CouchDB/"++couch:version()}, - {"Accept", "application/json"}, - {"Accept-Encoding", "gzip"} - ], - qs = [], - method = get, - body = nil, - options = [ - {response_format,binary}, - {inactivity_timeout, 30000} - ], - retries = 10, - pause = 500, - conn = nil -}). - -% small value used in revision trees to indicate the revision isn't stored --define(REV_MISSING, []). - --record(changes_args, { - feed = "normal", - dir = fwd, - since = "0", - limit = 1000000000000000, - style = main_only, - heartbeat, - timeout, - filter, - include_docs = false -}). - --record(proc, { - pid, - lang, - client = nil, - ddoc_keys = [], - prompt_fun, - set_timeout_fun, - stop_fun, - data_fun -}). - --record(leaf, { - deleted, - ptr, - seq, - size = 0, - atts = [] -}). http://git-wip-us.apache.org/repos/asf/couchdb-fabric/blob/455ade2f/src/fabric.erl ---------------------------------------------------------------------- diff --git a/src/fabric.erl b/src/fabric.erl index 05eed7e..c0f95df 100644 --- a/src/fabric.erl +++ b/src/fabric.erl @@ -38,7 +38,7 @@ -include_lib("fabric/include/fabric.hrl"). --type dbname() :: (iodata() | #db{}). +-type dbname() :: (iodata() | tuple()). -type docid() :: iodata(). -type revision() :: {integer(), binary()}. -type callback() :: fun((any(), any()) -> {ok | stop, any()}). @@ -476,10 +476,12 @@ dbname(DbName) when is_list(DbName) -> list_to_binary(DbName); dbname(DbName) when is_binary(DbName) -> DbName; -dbname(#db{name=Name}) -> - Name; -dbname(DbName) -> - erlang:error({illegal_database_name, DbName}). +dbname(Db) -> + try + couch_db:name(Db) + catch error:badarg -> + erlang:error({illegal_database_name, Db}) + end. name(Thing) -> couch_util:to_binary(Thing). http://git-wip-us.apache.org/repos/asf/couchdb-fabric/blob/455ade2f/src/fabric_rpc.erl ---------------------------------------------------------------------- diff --git a/src/fabric_rpc.erl b/src/fabric_rpc.erl index d2ef8a9..0ae1d7b 100644 --- a/src/fabric_rpc.erl +++ b/src/fabric_rpc.erl @@ -38,7 +38,8 @@ }). %% rpc endpoints -%% call to with_db will supply your M:F with a #db{} and then remaining args +%% call to with_db will supply your M:F with a Db instance +%% and then remaining args %% @equiv changes(DbName, Args, StartSeq, []) changes(DbName, Args, StartSeq) -> @@ -76,13 +77,13 @@ changes(DbName, Options, StartVector, DbOptions) -> args = Args, options = Options, pending = couch_db:count_changes_since(Db, StartSeq), - epochs = get_epochs(Db) + epochs = couch_db:get_epochs(Db) }, try {ok, #cacc{seq=LastSeq, pending=Pending, epochs=Epochs}} = couch_db:changes_since(Db, StartSeq, Enum, Opts, Acc0), rexi:stream_last({complete, [ - {seq, {LastSeq, uuid(Db), owner_of(LastSeq, Epochs)}}, + {seq, {LastSeq, uuid(Db), couch_db:owner_of(Epochs, LastSeq)}}, {pending, Pending} ]}) after @@ -225,7 +226,7 @@ get_missing_revs(DbName, IdRevsList, Options) -> not_found -> {Id, Revs, []} end - end, IdRevsList, couch_btree:lookup(Db#db.id_tree, Ids))}; + end, IdRevsList, couch_db:get_full_doc_infos(Db, Ids))}; Error -> Error end). @@ -249,8 +250,9 @@ group_info(DbName, DDocId, DbOptions) -> reset_validation_funs(DbName) -> case get_or_create_db(DbName, []) of - {ok, #db{main_pid = Pid}} -> - gen_server:cast(Pid, {load_validation_funs, undefined}); + {ok, Db} -> + DbPid = couch_db:get_pid(Db), + gen_server:cast(DbPid, {load_validation_funs, undefined}); _ -> ok end. @@ -356,7 +358,7 @@ changes_enumerator(DocInfo, Acc) -> Opts = if Conflicts -> [conflicts | DocOptions]; true -> DocOptions end, ChangesRow = {change, [ {pending, Pending-1}, - {seq, {Seq, uuid(Db), owner_of(Seq, Epochs)}}, + {seq, {Seq, uuid(Db), couch_db:owner_of(Epochs, Seq)}}, {id, Id}, {changes, Results}, {deleted, Del} | @@ -453,78 +455,17 @@ set_io_priority(DbName, Options) -> ok end. -calculate_start_seq(_Db, _Node, Seq) when is_integer(Seq) -> - Seq; -calculate_start_seq(Db, Node, {Seq, Uuid}) -> - % Treat the current node as the epoch node - calculate_start_seq(Db, Node, {Seq, Uuid, Node}); -calculate_start_seq(Db, _Node, {Seq, Uuid, EpochNode}) -> - case is_prefix(Uuid, couch_db:get_uuid(Db)) of - true -> - case is_owner(EpochNode, Seq, couch_db:get_epochs(Db)) of - true -> Seq; - false -> 0 - end; - false -> - %% The file was rebuilt, most likely in a different - %% order, so rewind. - 0 - end; -calculate_start_seq(Db, _Node, {replace, OriginalNode, Uuid, Seq}) -> - case is_prefix(Uuid, couch_db:get_uuid(Db)) of - true -> - start_seq(get_epochs(Db), OriginalNode, Seq); - false -> +calculate_start_seq(Db, Node, Seq) -> + case couch_db:calculate_start_seq(Db, Node, Seq) of + N when is_integer(N) -> + N; + {replace, OriginalNode, Uuid, OriginalSeq} -> %% Scan history looking for an entry with %% * target_node == TargetNode %% * target_uuid == TargetUUID %% * target_seq =< TargetSeq %% If such an entry is found, stream from associated source_seq - mem3_rep:find_source_seq(Db, OriginalNode, Uuid, Seq) - end. - -is_prefix(Pattern, Subject) -> - binary:longest_common_prefix([Pattern, Subject]) == size(Pattern). - -is_owner(Node, Seq, Epochs) -> - validate_epochs(Epochs), - Node =:= owner_of(Seq, Epochs). - -owner_of(_Seq, []) -> - undefined; -owner_of(Seq, [{EpochNode, EpochSeq} | _Rest]) when Seq > EpochSeq -> - EpochNode; -owner_of(Seq, [_ | Rest]) -> - owner_of(Seq, Rest). - -get_epochs(Db) -> - Epochs = couch_db:get_epochs(Db), - validate_epochs(Epochs), - Epochs. - -start_seq([{OrigNode, EpochSeq} | _], OrigNode, Seq) when Seq > EpochSeq -> - %% OrigNode is the owner of the Seq so we can safely stream from there - Seq; -start_seq([{_, NewSeq}, {OrigNode, _} | _], OrigNode, Seq) when Seq > NewSeq -> - %% We transferred this file before Seq was written on OrigNode, so we need - %% to stream from the beginning of the next epoch. Note that it is _not_ - %% necessary for the current node to own the epoch beginning at NewSeq - NewSeq; -start_seq([_ | Rest], OrigNode, Seq) -> - start_seq(Rest, OrigNode, Seq); -start_seq([], OrigNode, Seq) -> - erlang:error({epoch_mismatch, OrigNode, Seq}). - -validate_epochs(Epochs) -> - %% Assert uniqueness. - case length(Epochs) == length(lists:ukeysort(2, Epochs)) of - true -> ok; - false -> erlang:error(duplicate_epoch) - end, - %% Assert order. - case Epochs == lists:sort(fun({_, A}, {_, B}) -> B =< A end, Epochs) of - true -> ok; - false -> erlang:error(epoch_order) + mem3_rep:find_source_seq(Db, OriginalNode, Uuid, OriginalSeq) end. uuid(Db) -> @@ -537,30 +478,6 @@ uuid_prefix_len() -> -ifdef(TEST). -include_lib("eunit/include/eunit.hrl"). -calculate_start_seq_test() -> - %% uuid mismatch is always a rewind. - Hdr1 = couch_db_header:new(), - Hdr2 = couch_db_header:set(Hdr1, [{epochs, [{node1, 1}]}, {uuid, <<"uuid1">>}]), - ?assertEqual(0, calculate_start_seq(#db{header=Hdr2}, node1, {1, <<"uuid2">>})), - %% uuid matches and seq is owned by node. - Hdr3 = couch_db_header:set(Hdr2, [{epochs, [{node1, 1}]}]), - ?assertEqual(2, calculate_start_seq(#db{header=Hdr3}, node1, {2, <<"uuid1">>})), - %% uuids match but seq is not owned by node. - Hdr4 = couch_db_header:set(Hdr2, [{epochs, [{node2, 2}, {node1, 1}]}]), - ?assertEqual(0, calculate_start_seq(#db{header=Hdr4}, node1, {3, <<"uuid1">>})), - %% return integer if we didn't get a vector. - ?assertEqual(4, calculate_start_seq(#db{}, foo, 4)). - -is_owner_test() -> - ?assertNot(is_owner(foo, 1, [])), - ?assertNot(is_owner(foo, 1, [{foo, 1}])), - ?assert(is_owner(foo, 2, [{foo, 1}])), - ?assert(is_owner(foo, 50, [{bar, 100}, {foo, 1}])), - ?assert(is_owner(foo, 50, [{baz, 200}, {bar, 100}, {foo, 1}])), - ?assert(is_owner(bar, 150, [{baz, 200}, {bar, 100}, {foo, 1}])), - ?assertError(duplicate_epoch, is_owner(foo, 1, [{foo, 1}, {bar, 1}])), - ?assertError(epoch_order, is_owner(foo, 1, [{foo, 100}, {bar, 200}])). - maybe_filtered_json_doc_no_filter_test() -> Body = {[{<<"a">>, 1}]}, Doc = #doc{id = <<"1">>, revs = {1, [<<"r1">>]}, body = Body}, http://git-wip-us.apache.org/repos/asf/couchdb-fabric/blob/455ade2f/src/fabric_util.erl ---------------------------------------------------------------------- diff --git a/src/fabric_util.erl b/src/fabric_util.erl index 7e3f23e..c5aef03 100644 --- a/src/fabric_util.erl +++ b/src/fabric_util.erl @@ -302,7 +302,8 @@ path_ends_with(Path, Suffix) -> fake_db(DbName, Opts) -> {SecProps} = fabric:get_security(DbName), % as admin UserCtx = couch_util:get_value(user_ctx, Opts, #user_ctx{}), - #db{name = DbName, security = SecProps, user_ctx = UserCtx}. + {ok, Db} = couch_db:clustered_db(DbName, UserCtx, SecProps), + Db. %% test function kv(Item, Count) ->