You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@couchdb.apache.org by da...@apache.org on 2019/12/05 14:52:59 UTC

[couchdb] branch prototype/fdb-layer-get-dbs-info updated (b31b20a -> fec0dcb)

This is an automated email from the ASF dual-hosted git repository.

davisp pushed a change to branch prototype/fdb-layer-get-dbs-info
in repository https://gitbox.apache.org/repos/asf/couchdb.git.


 discard b31b20a  Track the size of data store in a database
 discard 5bfad05  Support `GET /_dbs_info` endpoint
 discard 8bab9cd  Implement `fabric2_db:list_dbs_info/1,2,3`
 discard f2b10ad  Implement async API for `fabric2_fdb:get_info/1`
 discard df6b79a  Track a database level view size rollup
 discard b0a2527  Normalize fabric2 test suite
     add 788538d  Normalize fabric2 test suite
     new 1a7e373  Track a database level view size rollup
     new 288d7b6  Implement async API for `fabric2_fdb:get_info/1`
     new 60fa333  Implement `fabric2_db:list_dbs_info/1,2,3`
     new f9ff19a  Support `GET /_dbs_info` endpoint
     new fec0dcb  Track the size of data store in a database

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (b31b20a)
            \
             N -- N -- N   refs/heads/prototype/fdb-layer-get-dbs-info (fec0dcb)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/fabric/test/fabric2_changes_fold_tests.erl     |  8 +-----
 src/fabric/test/fabric2_db_crud_tests.erl          | 10 +-------
 src/fabric/test/fabric2_db_misc_tests.erl          | 10 +-------
 src/fabric/test/fabric2_db_security_tests.erl      |  9 +------
 src/fabric/test/fabric2_dir_prefix_tests.erl       | 15 +++++------
 src/fabric/test/fabric2_doc_count_tests.erl        |  8 +-----
 src/fabric/test/fabric2_doc_crud_tests.erl         | 10 +-------
 src/fabric/test/fabric2_doc_fold_tests.erl         | 29 ++++++++--------------
 src/fabric/test/fabric2_fdb_tx_retry_tests.erl     | 10 +-------
 src/fabric/test/fabric2_local_doc_fold_tests.erl   | 29 ++++++++--------------
 src/fabric/test/fabric2_node_types_tests.erl       | 10 +-------
 src/fabric/test/fabric2_rev_stemming.erl           | 10 +-------
 .../test/fabric2_test.hrl}                         | 24 ++++++++----------
 src/fabric/test/fabric2_trace_db_create_tests.erl  | 10 +-------
 src/fabric/test/fabric2_trace_db_delete_tests.erl  | 10 +-------
 src/fabric/test/fabric2_trace_db_open_tests.erl    | 10 +-------
 src/fabric/test/fabric2_trace_doc_create_tests.erl |  9 +------
 17 files changed, 50 insertions(+), 171 deletions(-)
 copy src/{chttpd/src/chttpd_test_util.erl => fabric/test/fabric2_test.hrl} (57%)


[couchdb] 05/05: Track the size of data store in a database

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

davisp pushed a commit to branch prototype/fdb-layer-get-dbs-info
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit fec0dcbd794efc49fdb97e6f3d2ee9636f9ae6df
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Wed Dec 4 11:38:48 2019 -0600

    Track the size of data store in a database
    
    This tracks the number of bytes that would be required to store the
    contents of a database as flat files on disk. Currently the following
    items are tracked:
    
        * Doc ids
        * Revisions
        * Doc body as JSON
        * Attachment names
        * Attachment type
        * Attachment length
        * Attachment md5s
        * Attachment headers
        * Local doc id
        * Local doc revision
        * Local doc bodies
---
 src/couch/src/couch_att.erl                |  19 ++++
 src/fabric/include/fabric2.hrl             |   8 +-
 src/fabric/src/fabric2_db.erl              |   6 +-
 src/fabric/src/fabric2_fdb.erl             | 157 ++++++++++++++++++++++++-----
 src/fabric/test/fabric2_doc_crud_tests.erl |   5 +-
 5 files changed, 163 insertions(+), 32 deletions(-)

diff --git a/src/couch/src/couch_att.erl b/src/couch/src/couch_att.erl
index 2c33362..90d498c 100644
--- a/src/couch/src/couch_att.erl
+++ b/src/couch/src/couch_att.erl
@@ -27,6 +27,7 @@
 ]).
 
 -export([
+    external_size/1,
     size_info/1,
     to_disk_term/1,
     from_disk_term/3
@@ -179,6 +180,24 @@ merge_stubs([], _, Merged) ->
     {ok, lists:reverse(Merged)}.
 
 
+external_size(Att) ->
+    NameSize = size(fetch(name, Att)),
+    TypeSize = case fetch(type, Att) of
+        undefined -> 0;
+        Type -> size(Type)
+    end,
+    AttSize = fetch(att_len, Att),
+    Md5Size = case fetch(md5, Att) of
+        undefined -> 0;
+        Md5 -> size(Md5)
+    end,
+    HeadersSize = case fetch(headers, Att) of
+        undefined -> 0;
+        Headers -> couch_ejson_size:encoded_size(Headers)
+    end,
+    NameSize + TypeSize + AttSize + Md5Size + HeadersSize.
+
+
 size_info([]) ->
     {ok, []};
 size_info(Atts) ->
diff --git a/src/fabric/include/fabric2.hrl b/src/fabric/include/fabric2.hrl
index 189995d..fc07f33 100644
--- a/src/fabric/include/fabric2.hrl
+++ b/src/fabric/include/fabric2.hrl
@@ -41,8 +41,14 @@
 
 % Versions
 
--define(CURR_REV_FORMAT, 0).
+% 0 - Initial implementation
+% 1 - Added size information
 
+-define(CURR_REV_FORMAT, 1).
+
+% 0 - Adding local doc versions
+
+-define(CURR_LDOC_FORMAT, 0).
 
 % Misc constants
 
diff --git a/src/fabric/src/fabric2_db.erl b/src/fabric/src/fabric2_db.erl
index 688f794..1bc53ca 100644
--- a/src/fabric/src/fabric2_db.erl
+++ b/src/fabric/src/fabric2_db.erl
@@ -1420,7 +1420,8 @@ update_doc_interactive(Db, Doc0, Future, _Options) ->
         rev_id => {NewRevPos, NewRev},
         rev_path => NewRevPath,
         sequence => undefined,
-        branch_count => undefined
+        branch_count => undefined,
+        rev_size => null
     },
 
     % Gather the list of possible winnig revisions
@@ -1475,7 +1476,8 @@ update_doc_replicated(Db, Doc0, _Options) ->
         rev_id => {RevPos, Rev},
         rev_path => RevPath,
         sequence => undefined,
-        branch_count => undefined
+        branch_count => undefined,
+        rev_size => null
     },
 
     AllRevInfos = fabric2_fdb:get_all_revs(Db, DocId),
diff --git a/src/fabric/src/fabric2_fdb.erl b/src/fabric/src/fabric2_fdb.erl
index c13ef7c..a91f82a 100644
--- a/src/fabric/src/fabric2_fdb.erl
+++ b/src/fabric/src/fabric2_fdb.erl
@@ -36,6 +36,7 @@
 
     get_stat/2,
     incr_stat/3,
+    incr_stat/4,
 
     get_all_revs/2,
     get_winning_revs/3,
@@ -451,6 +452,19 @@ incr_stat(#{} = Db, StatKey, Increment) when is_integer(Increment) ->
     erlfdb:add(Tx, Key, Increment).
 
 
+incr_stat(_Db, _Section, _Key, 0) ->
+    ok;
+
+incr_stat(#{} = Db, Section, Key, Increment) when is_integer(Increment) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = ensure_current(Db),
+
+    BinKey = erlfdb_tuple:pack({?DB_STATS, Section, Key}, DbPrefix),
+    erlfdb:add(Tx, BinKey, Increment).
+
+
 get_all_revs(#{} = Db, DocId) ->
     #{
         tx := Tx,
@@ -570,6 +584,15 @@ get_local_doc(#{} = Db0, <<?LOCAL_DOC_PREFIX, _/binary>> = DocId) ->
 
 get_local_doc_rev(_Db0, <<?LOCAL_DOC_PREFIX, _/binary>> = DocId, Val) ->
     case Val of
+        <<255, RevBin/binary>> ->
+            % Versioned local docs
+            try
+                case erlfdb_tuple:unpack(RevBin) of
+                    {?CURR_LDOC_FORMAT, Rev, _Size} -> Rev
+                end
+            catch _:_ ->
+                erlang:error({invalid_local_doc_rev, DocId, Val})
+            end;
         <<131, _/binary>> ->
             % Compatibility clause for an older encoding format
             try binary_to_term(Val, [safe]) of
@@ -603,9 +626,16 @@ write_doc(#{} = Db0, Doc, NewWinner0, OldWinner, ToUpdate, ToRemove) ->
         deleted = Deleted
     } = Doc,
 
+    % Doc body
+
+    {ok, RevSize} = write_doc_body(Db, Doc),
+
     % Revision tree
 
-    NewWinner = NewWinner0#{winner := true},
+    NewWinner = NewWinner0#{
+        winner := true,
+        rev_size := RevSize
+    },
     NewRevId = maps:get(rev_id, NewWinner),
 
     {WKey, WVal, WinnerVS} = revinfo_to_fdb(Tx, DbPrefix, DocId, NewWinner),
@@ -667,9 +697,7 @@ write_doc(#{} = Db0, Doc, NewWinner0, OldWinner, ToUpdate, ToRemove) ->
     NewSeqVal = erlfdb_tuple:pack({DocId, Deleted, NewRevId}),
     erlfdb:set_versionstamped_key(Tx, NewSeqKey, NewSeqVal),
 
-    % And all the rest...
-
-    ok = write_doc_body(Db, Doc),
+    % Bump db version on design doc changes
 
     IsDDoc = case Doc#doc.id of
         <<?DESIGN_DOC_PREFIX, _/binary>> -> true;
@@ -680,6 +708,8 @@ write_doc(#{} = Db0, Doc, NewWinner0, OldWinner, ToUpdate, ToRemove) ->
         bump_db_version(Db)
     end,
 
+    % Update our document counts
+
     case UpdateStatus of
         created ->
             if not IsDDoc -> ok; true ->
@@ -706,6 +736,15 @@ write_doc(#{} = Db0, Doc, NewWinner0, OldWinner, ToUpdate, ToRemove) ->
             ok
     end,
 
+    % Update database size
+    SizeIncr = RevSize - lists:foldl(fun(RI, Acc) ->
+        Acc + case maps:get(rev_size, RI, null) of
+            null -> 0;
+            Size -> Size
+        end
+    end, 0, ToRemove),
+    incr_stat(Db, <<"sizes">>, <<"external">>, SizeIncr),
+
     ok.
 
 
@@ -717,11 +756,18 @@ write_local_doc(#{} = Db0, Doc) ->
 
     Id = Doc#doc.id,
 
-    {LDocKey, LDocVal, Rows} = local_doc_to_fdb(Db, Doc),
+    {LDocKey, LDocVal, NewSize, Rows} = local_doc_to_fdb(Db, Doc),
 
-    WasDeleted = case erlfdb:wait(erlfdb:get(Tx, LDocKey)) of
-        <<_/binary>> -> false;
-        not_found -> true
+    {WasDeleted, PrevSize} = case erlfdb:wait(erlfdb:get(Tx, LDocKey)) of
+        <<255, RevBin/binary>> ->
+            case erlfdb_tuple:unpack(RevBin) of
+                {?CURR_LDOC_FORMAT, _Rev, Size} ->
+                    {false, Size}
+            end;
+        <<_/binary>> ->
+            {false, 0};
+        not_found ->
+            {true, 0}
     end,
 
     BPrefix = erlfdb_tuple:pack({?DB_LOCAL_DOC_BODIES, Id}, DbPrefix),
@@ -747,6 +793,8 @@ write_local_doc(#{} = Db0, Doc) ->
             ok
     end,
 
+    incr_stat(Db, <<"sizes">>, <<"external">>, NewSize - PrevSize),
+
     ok.
 
 
@@ -1010,9 +1058,11 @@ write_doc_body(#{} = Db0, #doc{} = Doc) ->
         tx := Tx
     } = Db = ensure_current(Db0),
 
+    {Rows, RevSize} = doc_to_fdb(Db, Doc),
     lists:foreach(fun({Key, Value}) ->
         ok = erlfdb:set(Tx, Key, Value)
-    end, doc_to_fdb(Db, Doc)).
+    end, Rows),
+    {ok, RevSize}.
 
 
 clear_doc_body(_Db, _DocId, not_found) ->
@@ -1039,11 +1089,12 @@ revinfo_to_fdb(Tx, DbPrefix, DocId, #{winner := true} = RevId) ->
         deleted := Deleted,
         rev_id := {RevPos, Rev},
         rev_path := RevPath,
-        branch_count := BranchCount
+        branch_count := BranchCount,
+        rev_size := RevSize
     } = RevId,
     VS = new_versionstamp(Tx),
     Key = {?DB_REVS, DocId, not Deleted, RevPos, Rev},
-    Val = {?CURR_REV_FORMAT, VS, BranchCount, list_to_tuple(RevPath)},
+    Val = {?CURR_REV_FORMAT, VS, BranchCount, list_to_tuple(RevPath), RevSize},
     KBin = erlfdb_tuple:pack(Key, DbPrefix),
     VBin = erlfdb_tuple:pack_vs(Val),
     {KBin, VBin, VS};
@@ -1052,38 +1103,49 @@ revinfo_to_fdb(_Tx, DbPrefix, DocId, #{} = RevId) ->
     #{
         deleted := Deleted,
         rev_id := {RevPos, Rev},
-        rev_path := RevPath
+        rev_path := RevPath,
+        rev_size := RevSize
     } = RevId,
     Key = {?DB_REVS, DocId, not Deleted, RevPos, Rev},
-    Val = {?CURR_REV_FORMAT, list_to_tuple(RevPath)},
+    Val = {?CURR_REV_FORMAT, list_to_tuple(RevPath), RevSize},
     KBin = erlfdb_tuple:pack(Key, DbPrefix),
     VBin = erlfdb_tuple:pack(Val),
     {KBin, VBin, undefined}.
 
 
-fdb_to_revinfo(Key, {?CURR_REV_FORMAT, _, _, _} = Val) ->
+fdb_to_revinfo(Key, {?CURR_REV_FORMAT, _, _, _, _} = Val) ->
     {?DB_REVS, _DocId, NotDeleted, RevPos, Rev} = Key,
-    {_RevFormat, Sequence, BranchCount, RevPath} = Val,
+    {_RevFormat, Sequence, BranchCount, RevPath, RevSize} = Val,
     #{
         winner => true,
         deleted => not NotDeleted,
         rev_id => {RevPos, Rev},
         rev_path => tuple_to_list(RevPath),
         sequence => Sequence,
-        branch_count => BranchCount
+        branch_count => BranchCount,
+        rev_size => RevSize
     };
 
-fdb_to_revinfo(Key, {?CURR_REV_FORMAT, _} = Val)  ->
+fdb_to_revinfo(Key, {?CURR_REV_FORMAT, _, _} = Val)  ->
     {?DB_REVS, _DocId, NotDeleted, RevPos, Rev} = Key,
-    {_RevFormat, RevPath} = Val,
+    {_RevFormat, RevPath, RevSize} = Val,
     #{
         winner => false,
         deleted => not NotDeleted,
         rev_id => {RevPos, Rev},
         rev_path => tuple_to_list(RevPath),
         sequence => undefined,
-        branch_count => undefined
-    }.
+        branch_count => undefined,
+        rev_size => RevSize
+    };
+
+fdb_to_revinfo(Key, {0, S, B, R}) ->
+    Val = {?CURR_REV_FORMAT, S, B, R, null},
+    fdb_to_revinfo(Key, Val);
+
+fdb_to_revinfo(Key, {0, R}) ->
+    Val = {?CURR_REV_FORMAT, R, null},
+    fdb_to_revinfo(Key, Val).
 
 
 doc_to_fdb(Db, #doc{} = Doc) ->
@@ -1102,12 +1164,26 @@ doc_to_fdb(Db, #doc{} = Doc) ->
     DiskAtts = lists:map(fun couch_att:to_disk_term/1, Atts),
 
     Value = term_to_binary({Body, DiskAtts, Deleted}, [{minor_version, 1}]),
+    Chunks = chunkify_binary(Value),
 
     {Rows, _} = lists:mapfoldl(fun(Chunk, ChunkId) ->
         Key = erlfdb_tuple:pack({?DB_DOCS, Id, Start, Rev, ChunkId}, DbPrefix),
         {{Key, Chunk}, ChunkId + 1}
-    end, 0, chunkify_binary(Value)),
-    Rows.
+    end, 0, Chunks),
+
+    % Calculate the size of this revision
+    TotalSize = lists:sum([
+        size(Id),
+        size(erlfdb_tuple:pack({Start})),
+        size(Rev),
+        1, % FDB tuple encoding of booleans for deleted flag is 1 byte
+        couch_ejson_size:encoded_size(Body),
+        lists:foldl(fun(Att, Acc) ->
+            couch_att:external_size(Att) + Acc
+        end, 0, Atts)
+    ]),
+
+    {Rows, TotalSize}.
 
 
 fdb_to_doc(_Db, _DocId, _Pos, _Path, []) ->
@@ -1157,8 +1233,28 @@ local_doc_to_fdb(Db, #doc{} = Doc) ->
         {{K, Chunk}, ChunkId + 1}
     end, 0, chunkify_binary(BVal)),
 
-    {Key, StoreRev, Rows}.
+    % Calculate size
+    TotalSize = case Doc#doc.deleted of
+        true ->
+            0;
+        false ->
+            lists:sum([
+                size(Id),
+                size(StoreRev),
+                couch_ejson_size:encoded_size(Body)
+            ])
+    end,
+
+    RawValue = erlfdb_tuple:pack({?CURR_LDOC_FORMAT, StoreRev, TotalSize}),
 
+    % Prefix our tuple encoding to make upgrades easier
+    Value = <<255, RawValue/binary>>,
+
+    {Key, Value, TotalSize, Rows}.
+
+
+fdb_to_local_doc(_Db, _DocId, not_found, []) ->
+    {not_found, missing};
 
 fdb_to_local_doc(_Db, DocId, <<131, _/binary>> = Val, []) ->
     % This is an upgrade clause for the old encoding. We allow reading the old
@@ -1171,18 +1267,25 @@ fdb_to_local_doc(_Db, DocId, <<131, _/binary>> = Val, []) ->
         body = Body
     };
 
-fdb_to_local_doc(_Db, _DocId, not_found, []) ->
-    {not_found, missing};
+fdb_to_local_doc(_Db, DocId, <<255, RevBin/binary>>, Rows) when is_list(Rows) ->
+    Rev = case erlfdb_tuple:unpack(RevBin) of
+        {?CURR_LDOC_FORMAT, Rev0, _Size} -> Rev0
+    end,
 
-fdb_to_local_doc(_Db, DocId, Rev, Rows) when is_list(Rows), is_binary(Rev) ->
     BodyBin = iolist_to_binary(Rows),
     Body = binary_to_term(BodyBin, [safe]),
+
     #doc{
         id = DocId,
         revs = {0, [Rev]},
         deleted = false,
         body = Body
-    }.
+    };
+
+fdb_to_local_doc(Db, DocId, RawRev, Rows) ->
+    BaseRev = erlfdb_tuple:pack({?CURR_LDOC_FORMAT, RawRev, 0}),
+    Rev = <<255, BaseRev/binary>>,
+    fdb_to_local_doc(Db, DocId, Rev, Rows).
 
 
 chunkify_binary(Data) ->
diff --git a/src/fabric/test/fabric2_doc_crud_tests.erl b/src/fabric/test/fabric2_doc_crud_tests.erl
index 184eb4a..46cd4fc 100644
--- a/src/fabric/test/fabric2_doc_crud_tests.erl
+++ b/src/fabric/test/fabric2_doc_crud_tests.erl
@@ -884,11 +884,12 @@ local_doc_with_previous_encoding({Db, _}) ->
     ?assertEqual(NewBody, Doc3#doc.body),
 
     % Old doc now has only the rev number in it
-    OldDocBin = fabric2_fdb:transactional(Db, fun(TxDb) ->
+    <<255, OldDocBin/binary>> = fabric2_fdb:transactional(Db, fun(TxDb) ->
         #{tx := Tx} = TxDb,
         erlfdb:wait(erlfdb:get(Tx, Key))
     end),
-    ?assertEqual(<<"2">> , OldDocBin).
+    Unpacked = erlfdb_tuple:unpack(OldDocBin),
+    ?assertMatch({?CURR_LDOC_FORMAT, <<"2">>, _}, Unpacked).
 
 
 before_doc_update_skips_local_docs({Db0, _}) ->


[couchdb] 03/05: Implement `fabric2_db:list_dbs_info/1,2,3`

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

davisp pushed a commit to branch prototype/fdb-layer-get-dbs-info
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 60fa333307af297e589b1631f54c335268306d79
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Dec 3 12:45:36 2019 -0600

    Implement `fabric2_db:list_dbs_info/1,2,3`
    
    This API allows for listing all database info blobs in a single request.
    It accepts the same parameters as `_all_dbs` for controlling pagination
    of results and so on.
---
 src/fabric/src/fabric2_db.erl             | 100 +++++++++++++++++++++++++-----
 src/fabric/src/fabric2_fdb.erl            |  11 ++++
 src/fabric/test/fabric2_db_crud_tests.erl |  31 ++++++++-
 3 files changed, 126 insertions(+), 16 deletions(-)

diff --git a/src/fabric/src/fabric2_db.erl b/src/fabric/src/fabric2_db.erl
index 88840e7..688f794 100644
--- a/src/fabric/src/fabric2_db.erl
+++ b/src/fabric/src/fabric2_db.erl
@@ -22,6 +22,10 @@
     list_dbs/1,
     list_dbs/3,
 
+    list_dbs_info/0,
+    list_dbs_info/1,
+    list_dbs_info/3,
+
     check_is_admin/1,
     check_is_member/1,
 
@@ -238,6 +242,46 @@ list_dbs(UserFun, UserAcc0, Options) ->
     end).
 
 
+list_dbs_info() ->
+    list_dbs_info([]).
+
+
+list_dbs_info(Options) ->
+    Callback = fun(Value, Acc) ->
+        NewAcc = case Value of
+            {meta, _} -> Acc;
+            {row, DbInfo} -> [DbInfo | Acc];
+            complete -> Acc
+        end,
+        {ok, NewAcc}
+    end,
+    {ok, DbInfos} = list_dbs_info(Callback, [], Options),
+    {ok, lists:reverse(DbInfos)}.
+
+
+list_dbs_info(UserFun, UserAcc0, Options) ->
+    FoldFun = fun(DbName, InfoFuture, {FutureQ, Count, Acc}) ->
+        NewFutureQ = queue:in({DbName, InfoFuture}, FutureQ),
+        drain_info_futures(NewFutureQ, Count + 1, UserFun, Acc)
+    end,
+    fabric2_fdb:transactional(fun(Tx) ->
+        try
+            UserAcc1 = maybe_stop(UserFun({meta, []}, UserAcc0)),
+            InitAcc = {queue:new(), 0, UserAcc1},
+            {FinalFutureQ, _, UserAcc2} = fabric2_fdb:list_dbs_info(
+                    Tx,
+                    FoldFun,
+                    InitAcc,
+                    Options
+                ),
+            UserAcc3 = drain_all_info_futures(FinalFutureQ, UserFun, UserAcc2),
+            {ok, maybe_stop(UserFun(complete, UserAcc3))}
+        catch throw:{stop, FinalUserAcc} ->
+            {ok, FinalUserAcc}
+        end
+    end).
+
+
 is_admin(Db, {SecProps}) when is_list(SecProps) ->
     case fabric2_db_plugin:check_is_admin(Db) of
         true ->
@@ -313,21 +357,7 @@ get_db_info(#{} = Db) ->
     DbProps = fabric2_fdb:transactional(Db, fun(TxDb) ->
         fabric2_fdb:get_info(TxDb)
     end),
-
-    BaseProps = [
-        {cluster, {[{n, 0}, {q, 0}, {r, 0}, {w, 0}]}},
-        {compact_running, false},
-        {data_size, 0},
-        {db_name, name(Db)},
-        {disk_format_version, 0},
-        {disk_size, 0},
-        {instance_start_time, <<"0">>},
-        {purge_seq, 0}
-    ],
-
-    {ok, lists:foldl(fun({Key, Val}, Acc) ->
-        lists:keystore(Key, 1, Acc, {Key, Val})
-    end, BaseProps, DbProps)}.
+    {ok, make_db_info(name(Db), DbProps)}.
 
 
 get_del_doc_count(#{} = Db) ->
@@ -944,6 +974,46 @@ maybe_add_sys_db_callbacks(Db) ->
     }.
 
 
+make_db_info(DbName, Props) ->
+    BaseProps = [
+        {cluster, {[{n, 0}, {q, 0}, {r, 0}, {w, 0}]}},
+        {compact_running, false},
+        {data_size, 0},
+        {db_name, DbName},
+        {disk_format_version, 0},
+        {disk_size, 0},
+        {instance_start_time, <<"0">>},
+        {purge_seq, 0}
+    ],
+
+    lists:foldl(fun({Key, Val}, Acc) ->
+        lists:keystore(Key, 1, Acc, {Key, Val})
+    end, BaseProps, Props).
+
+
+drain_info_futures(FutureQ, Count, _UserFun, Acc) when Count < 100 ->
+    {FutureQ, Count, Acc};
+
+drain_info_futures(FutureQ, Count, UserFun, Acc) when Count >= 100 ->
+    {{value, {DbName, Future}}, RestQ} = queue:out(FutureQ),
+    InfoProps = fabric2_fdb:get_info_wait(Future),
+    DbInfo = make_db_info(DbName, InfoProps),
+    NewAcc = maybe_stop(UserFun({row, DbInfo}, Acc)),
+    {RestQ, Count - 1, NewAcc}.
+
+
+drain_all_info_futures(FutureQ, UserFun, Acc) ->
+    case queue:out(FutureQ) of
+        {{value, {DbName, Future}}, RestQ} ->
+            InfoProps = fabric2_fdb:get_info_wait(Future),
+            DbInfo = make_db_info(DbName, InfoProps),
+            NewAcc = maybe_stop(UserFun({row, DbInfo}, Acc)),
+            drain_all_info_futures(RestQ, UserFun, NewAcc);
+        {empty, _} ->
+            Acc
+    end.
+
+
 new_revid(Db, Doc) ->
     #doc{
         id = DocId,
diff --git a/src/fabric/src/fabric2_fdb.erl b/src/fabric/src/fabric2_fdb.erl
index 17d3c14..c13ef7c 100644
--- a/src/fabric/src/fabric2_fdb.erl
+++ b/src/fabric/src/fabric2_fdb.erl
@@ -27,6 +27,7 @@
     get_dir/1,
 
     list_dbs/4,
+    list_dbs_info/4,
 
     get_info/1,
     get_info_future/2,
@@ -327,6 +328,16 @@ list_dbs(Tx, Callback, AccIn, Options) ->
     end, AccIn, Options).
 
 
+list_dbs_info(Tx, Callback, AccIn, Options) ->
+    LayerPrefix = get_dir(Tx),
+    Prefix = erlfdb_tuple:pack({?ALL_DBS}, LayerPrefix),
+    fold_range({tx, Tx}, Prefix, fun({DbNameKey, DbPrefix}, Acc) ->
+        {DbName} = erlfdb_tuple:unpack(DbNameKey, Prefix),
+        InfoFuture = get_info_future(Tx, DbPrefix),
+        Callback(DbName, InfoFuture, Acc)
+    end, AccIn, Options).
+
+
 get_info(#{} = Db) ->
     #{
         tx := Tx,
diff --git a/src/fabric/test/fabric2_db_crud_tests.erl b/src/fabric/test/fabric2_db_crud_tests.erl
index cc44f7d..8052551 100644
--- a/src/fabric/test/fabric2_db_crud_tests.erl
+++ b/src/fabric/test/fabric2_db_crud_tests.erl
@@ -29,7 +29,8 @@ crud_test_() ->
                 ?TDEF(create_db),
                 ?TDEF(open_db),
                 ?TDEF(delete_db),
-                ?TDEF(list_dbs)
+                ?TDEF(list_dbs),
+                ?TDEF(list_dbs_info)
             ])
         }
     }.
@@ -84,3 +85,31 @@ list_dbs(_) ->
     ?assertEqual(ok, fabric2_db:delete(DbName, [])),
     AllDbs3 = fabric2_db:list_dbs(),
     ?assert(not lists:member(DbName, AllDbs3)).
+
+
+list_dbs_info(_) ->
+    DbName = ?tempdb(),
+    {ok, AllDbInfos1} = fabric2_db:list_dbs_info(),
+
+    ?assert(is_list(AllDbInfos1)),
+    ?assert(not is_db_info_member(DbName, AllDbInfos1)),
+
+    ?assertMatch({ok, _}, fabric2_db:create(DbName, [])),
+    {ok, AllDbInfos2} = fabric2_db:list_dbs_info(),
+    ?assert(is_db_info_member(DbName, AllDbInfos2)),
+
+    ?assertEqual(ok, fabric2_db:delete(DbName, [])),
+    {ok, AllDbInfos3} = fabric2_db:list_dbs_info(),
+    ?assert(not is_db_info_member(DbName, AllDbInfos3)).
+
+
+is_db_info_member(_, []) ->
+    false;
+
+is_db_info_member(DbName, [DbInfo | RestInfos]) ->
+    case lists:keyfind(db_name, 1, DbInfo) of
+        {db_name, DbName} ->
+            true;
+        _E ->
+            is_db_info_member(DbName, RestInfos)
+    end.


[couchdb] 04/05: Support `GET /_dbs_info` endpoint

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

davisp pushed a commit to branch prototype/fdb-layer-get-dbs-info
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit f9ff19a5bd29f7729b7e598262ce8a243cf958fa
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Dec 3 13:44:35 2019 -0600

    Support `GET /_dbs_info` endpoint
    
    Previously only `POST` with a list of keys was supported. The new `GET`
    support just dumps all database info blobs in a single ordered response.
---
 src/chttpd/src/chttpd_misc.erl | 52 +++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 51 insertions(+), 1 deletion(-)

diff --git a/src/chttpd/src/chttpd_misc.erl b/src/chttpd/src/chttpd_misc.erl
index 186ec9f..21a4fef 100644
--- a/src/chttpd/src/chttpd_misc.erl
+++ b/src/chttpd/src/chttpd_misc.erl
@@ -157,6 +157,40 @@ all_dbs_callback({error, Reason}, #vacc{resp=Resp0}=Acc) ->
     {ok, Resp1} = chttpd:send_delayed_error(Resp0, Reason),
     {ok, Acc#vacc{resp=Resp1}}.
 
+handle_dbs_info_req(#httpd{method = 'GET'} = Req) ->
+    ok = chttpd:verify_is_server_admin(Req),
+
+    #mrargs{
+        start_key = StartKey,
+        end_key = EndKey,
+        direction = Dir,
+        limit = Limit,
+        skip = Skip
+    } = couch_mrview_http:parse_params(Req, undefined),
+
+    Options = [
+        {start_key, StartKey},
+        {end_key, EndKey},
+        {dir, Dir},
+        {limit, Limit},
+        {skip, Skip}
+    ],
+
+    % Eventually the Etag for this request will be derived
+    % from the \xFFmetadataVersion key in fdb
+    Etag = <<"foo">>,
+
+    {ok, Resp} = chttpd:etag_respond(Req, Etag, fun() ->
+        Headers = [{"ETag", Etag}],
+        {ok, Resp} = chttpd:start_delayed_json_response(Req, 200, Headers),
+        Callback = fun dbs_info_callback/2,
+        Acc = #vacc{req = Req, resp = Resp},
+        fabric2_db:list_dbs_info(Callback, Acc, Options)
+    end),
+    case is_record(Resp, vacc) of
+        true -> {ok, Resp#vacc.resp};
+        _ -> {ok, Resp}
+    end;
 handle_dbs_info_req(#httpd{method='POST', user_ctx=UserCtx}=Req) ->
     chttpd:validate_ctype(Req, "application/json"),
     Props = chttpd:json_body_obj(Req),
@@ -188,7 +222,23 @@ handle_dbs_info_req(#httpd{method='POST', user_ctx=UserCtx}=Req) ->
     send_chunk(Resp, "]"),
     chttpd:end_json_response(Resp);
 handle_dbs_info_req(Req) ->
-    send_method_not_allowed(Req, "POST").
+    send_method_not_allowed(Req, "GET,HEAD,POST").
+
+dbs_info_callback({meta, _Meta}, #vacc{resp = Resp0} = Acc) ->
+    {ok, Resp1} = chttpd:send_delayed_chunk(Resp0, "["),
+    {ok, Acc#vacc{resp = Resp1}};
+dbs_info_callback({row, Props}, #vacc{resp = Resp0} = Acc) ->
+    Prepend = couch_mrview_http:prepend_val(Acc),
+    Chunk = [Prepend, ?JSON_ENCODE({Props})],
+    {ok, Resp1} = chttpd:send_delayed_chunk(Resp0, Chunk),
+    {ok, Acc#vacc{prepend = ",", resp = Resp1}};
+dbs_info_callback(complete, #vacc{resp = Resp0} = Acc) ->
+    {ok, Resp1} = chttpd:send_delayed_chunk(Resp0, "]"),
+    {ok, Resp2} = chttpd:end_delayed_json_response(Resp1),
+    {ok, Acc#vacc{resp = Resp2}};
+dbs_info_callback({error, Reason}, #vacc{resp = Resp0} = Acc) ->
+    {ok, Resp1} = chttpd:send_delayed_error(Resp0, Reason),
+    {ok, Acc#vacc{resp = Resp1}}.
 
 handle_task_status_req(#httpd{method='GET'}=Req) ->
     ok = chttpd:verify_is_server_admin(Req),


[couchdb] 01/05: Track a database level view size rollup

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

davisp pushed a commit to branch prototype/fdb-layer-get-dbs-info
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 1a7e373be6adcc45831fc4ba86607be91193f32d
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Dec 3 10:24:36 2019 -0600

    Track a database level view size rollup
    
    This way we can expose the total view size for a database in the dbinfo
    JSON blob.
---
 src/couch_views/src/couch_views_fdb.erl | 17 ++++++++++++++--
 src/fabric/src/fabric2_fdb.erl          | 36 ++++++++++++++++-----------------
 2 files changed, 33 insertions(+), 20 deletions(-)

diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
index 60ce300..1972e22 100644
--- a/src/couch_views/src/couch_views_fdb.erl
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -366,8 +366,16 @@ update_kv_size(TxDb, Sig, ViewId, Increment) ->
         tx := Tx,
         db_prefix := DbPrefix
     } = TxDb,
-    Key = kv_size_key(DbPrefix, Sig, ViewId),
-    erlfdb:add(Tx, Key, Increment).
+
+    % Track a view specific size for calls to
+    % GET /dbname/_design/doc/_info`
+    IdxKey = kv_size_key(DbPrefix, Sig, ViewId),
+    erlfdb:add(Tx, IdxKey, Increment),
+
+    % Track a database level rollup for calls to
+    % GET /dbname
+    DbKey = db_kv_size_key(DbPrefix),
+    erlfdb:add(Tx, DbKey, Increment).
 
 
 seq_key(DbPrefix, Sig) ->
@@ -385,6 +393,11 @@ kv_size_key(DbPrefix, Sig, ViewId) ->
     erlfdb_tuple:pack(Key, DbPrefix).
 
 
+db_kv_size_key(DbPrefix) ->
+    Key = {?DB_STATS, <<"sizes">>, <<"views">>},
+    erlfdb_tuple:pack(Key, DbPrefix).
+
+
 id_idx_key(DbPrefix, Sig, DocId, ViewId) ->
     Key = {?DB_VIEWS, Sig, ?VIEW_ID_RANGE, DocId, ViewId},
     erlfdb_tuple:pack(Key, DbPrefix).
diff --git a/src/fabric/src/fabric2_fdb.erl b/src/fabric/src/fabric2_fdb.erl
index fb2891b..1f68b68 100644
--- a/src/fabric/src/fabric2_fdb.erl
+++ b/src/fabric/src/fabric2_fdb.erl
@@ -171,11 +171,16 @@ create(#{} = Db0, Options) ->
         {?DB_STATS, <<"doc_del_count">>, ?uint2bin(0)},
         {?DB_STATS, <<"doc_design_count">>, ?uint2bin(0)},
         {?DB_STATS, <<"doc_local_count">>, ?uint2bin(0)},
-        {?DB_STATS, <<"size">>, ?uint2bin(2)}
+        {?DB_STATS, <<"sizes">>, <<"external">>, ?uint2bin(2)},
+        {?DB_STATS, <<"sizes">>, <<"views">>, ?uint2bin(0)}
     ],
-    lists:foreach(fun({P, K, V}) ->
-        Key = erlfdb_tuple:pack({P, K}, DbPrefix),
-        erlfdb:set(Tx, Key, V)
+    lists:foreach(fun
+        ({P, K, V}) ->
+            Key = erlfdb_tuple:pack({P, K}, DbPrefix),
+            erlfdb:set(Tx, Key, V);
+        ({P, S, K, V}) ->
+            Key = erlfdb_tuple:pack({P, S, K}, DbPrefix),
+            erlfdb:set(Tx, Key, V)
     end, Defaults),
 
     UserCtx = fabric2_util:get_value(user_ctx, Options, #user_ctx{}),
@@ -345,26 +350,21 @@ get_info(#{} = Db) ->
     end,
     CProp = {update_seq, RawSeq},
 
-    MProps = lists:flatmap(fun({K, V}) ->
+    MProps = lists:foldl(fun({K, V}, Acc) ->
         case erlfdb_tuple:unpack(K, DbPrefix) of
             {?DB_STATS, <<"doc_count">>} ->
-                [{doc_count, ?bin2uint(V)}];
+                [{doc_count, ?bin2uint(V)} | Acc];
             {?DB_STATS, <<"doc_del_count">>} ->
-                [{doc_del_count, ?bin2uint(V)}];
-            {?DB_STATS, <<"size">>} ->
+                [{doc_del_count, ?bin2uint(V)} | Acc];
+            {?DB_STATS, <<"sizes">>, Name} ->
                 Val = ?bin2uint(V),
-                [
-                    {other, {[{data_size, Val}]}},
-                    {sizes, {[
-                        {active, 0},
-                        {external, Val},
-                        {file, 0}
-                    ]}}
-                ];
+                {_, {Sizes}} = lists:keyfind(sizes, 1, Acc),
+                NewSizes = lists:keystore(Name, 1, Sizes, {Name, Val}),
+                lists:keystore(sizes, 1, Acc, {sizes, {NewSizes}});
             {?DB_STATS, _} ->
-                []
+                Acc
         end
-    end, erlfdb:wait(MetaFuture)),
+    end, [{sizes, {[]}}], erlfdb:wait(MetaFuture)),
 
     [CProp | MProps].
 


[couchdb] 02/05: Implement async API for `fabric2_fdb:get_info/1`

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

davisp pushed a commit to branch prototype/fdb-layer-get-dbs-info
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 288d7b6b1e98b46828959d2d56caee43191851fa
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Dec 3 12:44:34 2019 -0600

    Implement async API for `fabric2_fdb:get_info/1`
---
 src/fabric/src/fabric2_fdb.erl | 9 +++++++++
 1 file changed, 9 insertions(+)

diff --git a/src/fabric/src/fabric2_fdb.erl b/src/fabric/src/fabric2_fdb.erl
index 1f68b68..17d3c14 100644
--- a/src/fabric/src/fabric2_fdb.erl
+++ b/src/fabric/src/fabric2_fdb.erl
@@ -29,6 +29,8 @@
     list_dbs/4,
 
     get_info/1,
+    get_info_future/2,
+    get_info_wait/1,
     set_config/3,
 
     get_stat/2,
@@ -330,7 +332,10 @@ get_info(#{} = Db) ->
         tx := Tx,
         db_prefix := DbPrefix
     } = ensure_current(Db),
+    get_info_wait(get_info_future(Tx, DbPrefix)).
 
+
+get_info_future(Tx, DbPrefix) ->
     {CStart, CEnd} = erlfdb_tuple:range({?DB_CHANGES}, DbPrefix),
     ChangesFuture = erlfdb:get_range(Tx, CStart, CEnd, [
         {streaming_mode, exact},
@@ -341,6 +346,10 @@ get_info(#{} = Db) ->
     StatsPrefix = erlfdb_tuple:pack({?DB_STATS}, DbPrefix),
     MetaFuture = erlfdb:get_range_startswith(Tx, StatsPrefix),
 
+    {DbPrefix, ChangesFuture, MetaFuture}.
+
+
+get_info_wait({DbPrefix, ChangesFuture, MetaFuture}) ->
     RawSeq = case erlfdb:wait(ChangesFuture) of
         [] ->
             vs_to_seq(fabric2_util:seq_zero_vs());