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 2020/09/24 15:59:18 UTC

[couchdb] 04/07: Reimplement db wide view size tracking

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

davisp pushed a commit to branch feature-ebtree-views-2
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit ba2a42633616bd435e47d3445dac0b7892ec9176
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Thu Sep 3 12:05:45 2020 -0500

    Reimplement db wide view size tracking
---
 src/couch_views/src/couch_views_fdb.erl        |  27 +-
 src/couch_views/src/couch_views_trees.erl      |  30 +-
 src/couch_views/test/couch_views_size_test.erl | 829 +++++++++----------------
 3 files changed, 357 insertions(+), 529 deletions(-)

diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
index e813f2b..a9dbe3e 100644
--- a/src/couch_views/src/couch_views_fdb.erl
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -135,6 +135,11 @@ clear_index(Db, Signature) ->
         db_prefix := DbPrefix
     } = Db,
 
+    % Get view size to remove from global counter
+    SizeTuple = {?DB_VIEWS, ?VIEW_INFO, ?VIEW_KV_SIZE, Signature},
+    SizeKey = erlfdb_tuple:pack(SizeTuple, DbPrefix),
+    ViewSize = ?bin2uint(erlfdb:wait(erlfdb:get(Tx, SizeKey))),
+
     % Clear index info keys
     Keys = [
         {?DB_VIEWS, ?VIEW_INFO, ?VIEW_UPDATE_SEQ, Signature},
@@ -154,7 +159,12 @@ clear_index(Db, Signature) ->
     % Clear tree data
     TreeTuple = {?DB_VIEWS, ?VIEW_TREES, Signature},
     TreePrefix = erlfdb_tuple:pack(TreeTuple, DbPrefix),
-    erlfdb:clear_range_startswith(Tx, TreePrefix).
+    erlfdb:clear_range_startswith(Tx, TreePrefix),
+
+    % Decrement db wide view size counter
+    DbSizeTuple = {?DB_STATS, <<"sizes">>, <<"views">>},
+    DbSizeKey = erlfdb_tuple:pack(DbSizeTuple, DbPrefix),
+    erlfdb:add(Tx, DbSizeKey, -ViewSize).
 
 
 persist_chunks(Tx, set, [Key, Value]) ->
@@ -181,6 +191,21 @@ persist_chunks(Tx, clear, Key) ->
     erlfdb:clear_range_startswith(Tx, Key).
 
 
+update_kv_size(TxDb, Sig, OldSize, NewSize) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = TxDb,
+
+    ViewTuple = {?DB_VIEWS, ?VIEW_INFO, ?VIEW_KV_SIZE, Sig},
+    ViewKey = erlfdb_tuple:pack(ViewTuple, DbPrefix),
+    erlfdb:set(Tx, ViewKey, ?uint2bin(NewSize)),
+
+    DbTuple = {?DB_STATS, <<"sizes">>, <<"views">>},
+    DbKey = erlfdb_tuple:pack(DbTuple, DbPrefix),
+    erlfdb:add(Tx, DbKey, NewSize - OldSize).
+
+
 seq_key(DbPrefix, Sig) ->
     Key = {?DB_VIEWS, ?VIEW_INFO, ?VIEW_UPDATE_SEQ, Sig},
     erlfdb_tuple:pack(Key, DbPrefix).
diff --git a/src/couch_views/src/couch_views_trees.erl b/src/couch_views/src/couch_views_trees.erl
index d67ccd5..85e1b91 100644
--- a/src/couch_views/src/couch_views_trees.erl
+++ b/src/couch_views/src/couch_views_trees.erl
@@ -127,6 +127,12 @@ update_views(TxDb, Mrst, Docs) ->
         tx := Tx
     } = TxDb,
 
+    % Get initial KV size
+    OldKVSize = lists:foldl(fun(View, SizeAcc) ->
+        {_, Size} = ebtree:full_reduce(Tx, View#mrview.btree),
+        SizeAcc + Size
+    end, 0, Mrst#mrst.views),
+
     % Collect update information
     #{
         ids := IdMap,
@@ -146,7 +152,15 @@ update_views(TxDb, Mrst, Docs) ->
 
         ViewMap = maps:get(ViewId, ViewMaps, #{}),
         update_btree(Tx, BTree, ViewMap, DeleteRef)
-    end, Mrst#mrst.views).
+    end, Mrst#mrst.views),
+
+    % Get new KV size after update
+    NewKVSize = lists:foldl(fun(View, SizeAcc) ->
+        {_, Size} = ebtree:full_reduce(Tx, View#mrview.btree),
+        SizeAcc + Size
+    end, 0, Mrst#mrst.views),
+
+    couch_views_fdb:update_kv_size(TxDb, Mrst#mrst.sig, OldKVSize, NewKVSize).
 
 
 open_id_tree(TxDb, Sig) ->
@@ -200,16 +214,18 @@ make_reduce_fun(_Lang, #mrview{}) ->
     fun
         (KVs, _ReReduce = false) ->
             TotalSize = lists:foldl(fun({K, V}, Acc) ->
-                KSize = couch_ejson_size:encoded_size(K),
-                VSize = case V of
+                Acc + case V of
                     {dups, Dups} ->
                         lists:foldl(fun(D, DAcc) ->
-                            DAcc + couch_ejson_size:encoded_size(D)
+                            KSize = couch_ejson_size:encoded_size(K),
+                            VSize = couch_ejson_size:encoded_size(D),
+                            DAcc + KSize + VSize
                         end, 0, Dups);
                     _ ->
-                        couch_ejson_size:encoded_size(V)
-                end,
-                KSize + VSize + Acc
+                        KSize = couch_ejson_size:encoded_size(K),
+                        VSize = couch_ejson_size:encoded_size(V),
+                        KSize + VSize
+                end
             end, 0, KVs),
             {length(KVs), TotalSize};
         (KRs, _ReReduce = true) ->
diff --git a/src/couch_views/test/couch_views_size_test.erl b/src/couch_views/test/couch_views_size_test.erl
index cc2fe39..16537a3 100644
--- a/src/couch_views/test/couch_views_size_test.erl
+++ b/src/couch_views/test/couch_views_size_test.erl
@@ -16,162 +16,38 @@
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch_mrview/include/couch_mrview.hrl").
--include_lib("fabric/include/fabric2.hrl").
 -include_lib("couch_views/include/couch_views.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
 
-% N.B., we should move to couch_ejson_size instead
-% of erlang:external_size
-%
-% to calculate view size:
-% total = 0
-% for (fdb_k, fdb_v) in VIEW_MAP_RANGE:
-%   {EncUserKey, EncUserval} = erlfdb_tuple:unpack(fdb_v),
-%   UserKey = couch_views_encoding:decode(EncUserKey),
-%   UserVal = couch_views_encoding:decode(EncUserVal),
-%   total += erlang:external_size(UserKey),
-%   total += erlang:external_size(UserVal)
-%
-% Our goal in checking the size calculations is that we cover
-% as much of the possible key mutation space as possible while
-% not relying on fuzzing out the edge cases. Conceptually we have
-% two sets of keys E and U. E is keys as currently exist in the
-% view, and U is the new set of keys corresponding to an update.
-%
-% Both sets E and U have the same possible set of state variables:
-%
-% 1. N unique keys, where 0 =< N =< infinity
-% 2. D keys with duplicates, where 0 =< D =< N,
-% 3. R repeats for each member of D, for 2 =< R =< infinity
-%
-% Given two sets S1 and S2, we then have a set of transition variables:
-%
-% 1. deltaN - shared unique keys, where 0 =< deltaN =< N
-% 2. deltaD - shared duplicates, where 0 =< deltaD =< N
-% 3. deltaR - shared repeats for each D, where 2 =< deltaR =< infinity
-%
-% To search our state transition space, we can create two functions to
-% first define our start and end states, and for each transition we have
-% a function that defines the shared overlap between states.
-%
-% Given a list of transitions are checks then become simple in that
-% we can iterate over each transition checking that our index is valid
-% after each one. Index validation will purely look at the existing
-% state of the index in fdb and validate correctness.
-
--define(NUM_SINGLE_TESTS, 100).
--define(NUM_MULTI_TESTS, 100).
-
--define(N_DOMAIN, [0, 1, 2, 5]).
--define(D_DOMAIN, [0, 1, 2, 5]).
--define(R_DOMAIN, [2, 4]).
-
--define(DELTA_N_DOMAIN, [0, 1, 2, 5]).
--define(DELTA_D_DOMAIN, [0, 1, 2, 5]).
--define(DELTA_R_DOMAIN, [1, 2, 4]).
-
-
-generate_sets() ->
-    permute(?N_DOMAIN, ?D_DOMAIN, ?R_DOMAIN, fun(N, D, R) ->
-        % We can't have more duplicates than total keys
-        case D > N of
-            true -> throw(skip);
-            false -> ok
-        end,
-
-        % Only include one of the repeat values
-        % for our zero sets
-        case D == 0 of
-            true when R == 2 -> ok;
-            true -> throw(skip);
-            false -> ok
-        end,
-
-        % Replace R with a sentinel value for sanity
-        % when there are no dupes to have repeats
-        ActualR = if D == 0 -> 0; true -> R end,
-
-        {N, D, ActualR}
-    end).
-
-
-generate_transitions() ->
-    Sets = generate_sets(),
-    Pairs = [{Set1, Set2} || Set1 <- Sets, Set2 <- Sets],
-    lists:flatmap(fun({{N1, D1, _R1} = S1, {N2, D2, _R2} = S2}) ->
-        Filter = fun(DeltaN, DeltaD, DeltaR) ->
-            % Can't share more keys than the smaller of the
-            % two sets
-            case DeltaN > min(N1, N2) of
-                true -> throw(skip);
-                false -> ok
-            end,
-
-            % For DeltaD == 0, all combinations of DeltaD and
-            % DeltaR are equivalent tests
-            case DeltaN == 0 of
-                true when DeltaD == 0, DeltaR == 1 -> ok;
-                true -> throw(skip);
-                false -> ok
-            end,
-
-            % Can't share more dupes than exist in either set
-            % or the total number of shared keys
-            case DeltaD > min(D1, D2) orelse DeltaD > DeltaN of
-                true -> throw(skip);
-                false -> ok
-            end,
-
-            % For DeltaD == 0, all DeltaR correspond to the
-            % same test so only include one instance
-            case DeltaD == 0 of
-                true when DeltaR == 1 -> ok;
-                true -> throw(skip);
-                false -> ok
-            end,
-
-            % If we have more non-repeated keys in our
-            % transition than there's "room" for in the target
-            % set it isn't a valid test case.
-            TransitionNonRepeats = DeltaN - DeltaD,
-            TargetNonRepeats = N2 - D2,
-            case TransitionNonRepeats > TargetNonRepeats of
-                true -> throw(skip);
-                false -> ok
-            end,
-
-            {S1, S2, {DeltaN, DeltaD, DeltaR}}
-        end,
-        permute(?DELTA_N_DOMAIN, ?DELTA_D_DOMAIN, ?DELTA_R_DOMAIN, Filter)
-    end, Pairs).
-
-
-permute(NList, DList, RList, Filter) ->
-    % Technically we could call into Filter in each
-    % outer loops to conditionally skip inner loops.
-    % If someone comes along looking to speed up the
-    % fixture setup time, this would likely be an
-    % easy win.
-    lists:foldl(fun(N, NAcc) ->
-        lists:foldl(fun(D, DAcc) ->
-            lists:foldl(fun(R, RAcc) ->
-                try
-                    [Filter(N, D, R) | RAcc]
-                catch throw:skip ->
-                    RAcc
-                end
-            end, DAcc, RList)
-        end, NAcc, DList)
-    end, [], NList).
-
-
-row_transition_test_() ->
+
+-define(MAP_FUN1, <<"map_fun1">>).
+-define(MAP_FUN2, <<"map_fun2">>).
+
+
+indexer_test_() ->
     {
-        "Test view size tracking",
+        "Test view indexing",
         {
             setup,
             fun setup/0,
             fun cleanup/1,
-            fun create_transition_tests/1
+            {
+                foreach,
+                fun foreach_setup/0,
+                fun foreach_teardown/1,
+                [
+                    ?TDEF_FE(empty_view),
+                    ?TDEF_FE(single_doc),
+                    ?TDEF_FE(multiple_docs),
+                    ?TDEF_FE(update_no_size_change),
+                    ?TDEF_FE(update_increases_size),
+                    ?TDEF_FE(update_decreases_size),
+                    ?TDEF_FE(deleting_docs_decreases_size),
+                    ?TDEF_FE(multi_identical_keys_count_twice),
+                    ?TDEF_FE(multiple_design_docs),
+                    ?TDEF_FE(multiple_identical_design_docs)
+                ]
+            }
         }
     }.
 
@@ -183,387 +59,298 @@ setup() ->
             couch_js,
             couch_views
         ]),
-    {ok, Db} = fabric2_db:create(?tempdb(), [{user_ctx, ?ADMIN_USER}]),
-    {Ctx, Db}.
+    Ctx.
 
 
-cleanup({Ctx, Db}) ->
-    ok = fabric2_db:delete(fabric2_db:name(Db), []),
+cleanup(Ctx) ->
     test_util:stop_couch(Ctx).
 
 
-create_transition_tests({_Ctx, Db}) ->
-    try
-        throw(disabled),
-        Transitions = generate_transitions(),
-        Single = lists:flatmap(fun(T) ->
-            Name = lists:flatten(io_lib:format("single ~s", [tname(T)])),
-            [{Name, fun() -> check_single_transition(Db, T) end}]
-        end, lists:sort(Transitions)),
-        Multi = lists:flatmap(fun(T) ->
-            Name = lists:flatten(io_lib:format("multi ~s", [tname(T)])),
-            [{Name, fun() -> check_multi_transition(Db, T) end}]
-        end, lists:sort(group(shuffle(Transitions)))),
-        subset(?NUM_SINGLE_TESTS, Single) ++ subset(?NUM_MULTI_TESTS, Multi)
-    catch throw:disabled ->
-        [{"Disabled", fun() -> ok end}]
-    end.
-
-
-check_single_transition(Db, {Set1, Set2, Transition}) ->
-    clear_views(Db),
-    InitKVs = init_set(Set1, [a, b, c, d, e]),
-    CommonKVs = reduce_set(Transition, InitKVs),
-    FinalKVs = fill_set(Set2, CommonKVs, [v, w, x, y, z]),
-    {InitJSONKVs, Bindings} = unlabel(InitKVs, #{}),
-    {FinalJSONKVs, _} = unlabel(FinalKVs, Bindings),
-
-    Sig = couch_uuids:random(),
-    DocId = couch_uuids:random(),
-
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        write_docs(TxDb, Sig, [make_doc(DocId, InitJSONKVs)])
-    end),
-
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        write_docs(TxDb, Sig, [make_doc(DocId, FinalJSONKVs)])
-    end),
-
-    validate_index(Db, Sig, #{DocId => FinalJSONKVs}).
-
-
-check_multi_transition(Db, Transitions) ->
-    clear_views(Db),
-
-    {Docs, IdMap} = lists:mapfoldl(fun({Set1, Set2, Transition}, IdMapAcc) ->
-        DocId = couch_uuids:random(),
-        InitKVs = init_set(Set1, [a, b, c, d, e]),
-        CommonKVs = reduce_set(Transition, InitKVs),
-        FinalKVs = fill_set(Set2, CommonKVs, [v, w, x, y, z]),
-        {InitJSONKVs, Bindings} = unlabel(InitKVs, #{}),
-        {FinalJSONKVs, _} = unlabel(FinalKVs, Bindings),
-        InitDoc = make_doc(DocId, InitJSONKVs),
-        FinalDoc = make_doc(DocId, FinalJSONKVs),
-        {{InitDoc, FinalDoc}, maps:put(DocId, FinalJSONKVs, IdMapAcc)}
-    end, #{}, Transitions),
-
-    {InitDocs, FinalDocs} = lists:unzip(Docs),
-
-    Sig = couch_uuids:random(),
-
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        write_docs(TxDb, Sig, InitDocs)
-    end),
-
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        write_docs(TxDb, Sig, FinalDocs)
-    end),
-
-    validate_index(Db, Sig, IdMap).
-
-
-clear_views(Db) ->
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        #{
-            tx := Tx,
-            db_prefix := DbPrefix
-        } = TxDb,
-        {Start, End} = erlfdb_tuple:range({?DB_VIEWS}, DbPrefix),
-        erlfdb:clear_range(Tx, Start, End),
-
-        GlobalKey = {?DB_STATS, <<"sizes">>, <<"views">>},
-        BinGlobalKey = erlfdb_tuple:pack(GlobalKey, DbPrefix),
-        erlfdb:set(Tx, BinGlobalKey, ?uint2bin(0))
-    end).
-
-
-write_docs(TxDb, Sig, Docs) ->
-    Mrst = #mrst{
-        sig = Sig,
-        views = [#mrview{
-            id_num = 1
-        }]
+foreach_setup() ->
+    config:set("couch_views", "view_btree_node_size", "4", false),
+    {ok, Db} = fabric2_db:create(?tempdb(), [{user_ctx, ?ADMIN_USER}]),
+    Db.
+
+
+foreach_teardown(Db) ->
+    meck:unload(),
+    config:delete("couch_views", "change_limit"),
+    ok = fabric2_db:delete(fabric2_db:name(Db), []).
+
+
+empty_view(Db) ->
+    DDoc = create_ddoc(),
+    ?assertEqual(0, view_size(Db)),
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+    ?assertEqual(0, view_size(Db)).
+
+
+single_doc(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, _} = fabric2_db:update_doc(Db, Doc1, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    % Row: key: 0, row: 0, docid: "0"
+    % Bytes: key: 1, row: 1, docid: 3
+    % Total: 1 + 1 + 3 = 5
+    ?assertEqual(5, view_size(Db)).
+
+
+multiple_docs(Db) ->
+    DDoc = create_ddoc(),
+    Docs = [doc(I) || I <- lists:seq(0, 49)],
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, _} = fabric2_db:update_docs(Db, Docs, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    % Rows 0-9: 1 + 1 + 3 = 5
+    % Rows 10->49: 2 + 2 + 4 = 8
+    % 10 * 5 + 40 * 8 = 370
+    ?assertEqual(370, view_size(Db)).
+
+
+update_no_size_change(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, {Pos, Rev}} = fabric2_db:update_doc(Db, Doc1, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    ?assertEqual(5, view_size(Db)),
+
+    Doc2 = Doc1#doc{
+        revs = {Pos, [Rev]},
+        body = {[{<<"val">>, 1}]}
     },
-    IdxState = #{
-        last_seq => <<"foo">>
+    {ok, _} = fabric2_db:update_doc(Db, Doc2, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    % Row became: key: 1, val: 1, docid: "0"
+    % 1 + 1 + 3 = 5 so samesies
+    ?assertEqual(5, view_size(Db)).
+
+
+update_increases_size(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, {Pos, Rev}} = fabric2_db:update_doc(Db, Doc1, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    ?assertEqual(5, view_size(Db)),
+
+    Doc2 = Doc1#doc{
+        revs = {Pos, [Rev]},
+        body = {[{<<"val">>, 10}]}
     },
-    couch_views_indexer:write_docs(TxDb, Mrst, Docs, IdxState).
-
-
-validate_index(Db, Sig, JSONRows) ->
-    #{
-        db_prefix := DbPrefix
-    } = Db,
-    Rows = fabric2_fdb:transactional(Db, fun(TxDb) ->
-        #{
-            tx := Tx
-        } = TxDb,
-        {Start, End} = erlfdb_tuple:range({?DB_VIEWS}, DbPrefix),
-        erlfdb:get_range(Tx, Start, End)
-    end),
-
-    InitAcc = #{
-        row_count => 0,
-        kv_size => 0,
-        ids => #{},
-        rows => []
+    {ok, _} = fabric2_db:update_doc(Db, Doc2, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    % Row became: key: 10, val: 10, docid: "0"
+    % 2 + 2 + 3 = 7
+    ?assertEqual(7, view_size(Db)).
+
+
+update_decreases_size(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(10),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, {Pos, Rev}} = fabric2_db:update_doc(Db, Doc1, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    % Row: key: 10, val: 10, docid: "10"
+    % 2 + 2 + 4 = 8
+    ?assertEqual(8, view_size(Db)),
+
+    Doc2 = Doc1#doc{
+        revs = {Pos, [Rev]},
+        body = {[{<<"val">>, 0}]}
     },
+    {ok, _} = fabric2_db:update_doc(Db, Doc2, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
 
-    MapData = lists:foldl(fun({Key, Value}, Acc) ->
-        case erlfdb_tuple:unpack(Key, DbPrefix) of
-            {?DB_VIEWS, ?VIEW_INFO, ?VIEW_UPDATE_SEQ, Sig} ->
-                ?assertEqual(<<"foo">>, Value),
-                Acc;
-            {?DB_VIEWS, ?VIEW_INFO, ?VIEW_ROW_COUNT, Sig, 1} ->
-                maps:put(row_count, ?bin2uint(Value), Acc);
-            {?DB_VIEWS, ?VIEW_INFO, ?VIEW_KV_SIZE, Sig, 1} ->
-                maps:put(kv_size, ?bin2uint(Value), Acc);
-            {?DB_VIEWS, ?VIEW_DATA, Sig, ?VIEW_ID_RANGE, DocId, 1} ->
-                [
-                    TotalKeys, TotalSize, UniqueKeys
-                ] = couch_views_encoding:decode(Value),
-                maps:update_with(ids, fun(Ids) ->
-                    false = maps:is_key(DocId, Ids),
-                    maps:put(DocId, {TotalKeys, TotalSize, UniqueKeys}, Ids)
-                end, Acc);
-            {?DB_VIEWS, ?VIEW_DATA, Sig, ?VIEW_MAP_RANGE, 1, MapKey, _DupeId} ->
-                {EncKey, DocId} = MapKey,
-                {UserKey, UserVal} = erlfdb_tuple:unpack(Value),
-
-                UserJsonKey = couch_views_encoding:decode(UserKey),
-                UserJsonVal = couch_views_encoding:decode(UserVal),
-
-                ?assertEqual(
-                        EncKey,
-                        couch_views_encoding:encode(UserJsonKey, key)
-                    ),
-
-                maps:update_with(rows, fun(RAcc) ->
-                    [{DocId, UserJsonKey, UserJsonVal} | RAcc]
-                end, Acc)
-        end
-    end, InitAcc, Rows),
-
-    #{
-        row_count := RowCount,
-        kv_size := KVSize,
-        ids := MapIds,
-        rows := MapRows
-    } = MapData,
-
-    SumFun = fun(_DocId, {TotalKVs, TotalSize, _UniqueKeys}, {KVAcc, SAcc}) ->
-        {KVAcc + TotalKVs, SAcc + TotalSize}
-    end,
-    {SumKVCount, SumKVSize} = maps:fold(SumFun, {0, 0}, MapIds),
-    ?assertEqual(RowCount, length(MapRows)),
-    ?assertEqual(RowCount, SumKVCount),
-    ?assertEqual(KVSize, SumKVSize),
-    ?assert(KVSize >= 0),
-
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        GlobalSize = get_global_size(TxDb),
-        ?assertEqual(KVSize, GlobalSize),
-
-        ViewSize = couch_views_fdb:get_kv_size(TxDb, #mrst{sig = Sig}, 1),
-        ?assertEqual(KVSize, ViewSize)
-    end),
-
-    % Compare our raw JSON rows to what was indexed
-    IdsFromJSONRows = maps:fold(fun(DocId, DocRows, IdAcc) ->
-        FinalAcc = lists:foldl(fun({JsonKey, JsonVal}, {CAcc, SAcc, UAcc}) ->
-            KeySize = erlang:external_size(JsonKey),
-            ValSize = erlang:external_size(JsonVal),
-            NewUnique = lists:usort([JsonKey | UAcc]),
-            {CAcc + 1, SAcc + KeySize + ValSize, NewUnique}
-        end, {0, 0, []}, DocRows),
-        if FinalAcc == {0, 0, []} -> IdAcc; true ->
-            maps:put(DocId, FinalAcc, IdAcc)
-        end
-    end, #{}, JSONRows),
-    ?assertEqual(MapIds, IdsFromJSONRows),
-
-    % Compare the found id entries to our row data
-    IdsFromMapRows = lists:foldl(fun({DocId, JsonKey, JsonVal}, Acc) ->
-        KeySize = erlang:external_size(JsonKey),
-        ValSize = erlang:external_size(JsonVal),
-        Default = {1, KeySize + ValSize, [JsonKey]},
-        maps:update_with(DocId, fun({TotalKVs, TotalSize, UniqueKeys}) ->
-            NewUnique = lists:usort([JsonKey | UniqueKeys]),
-            {TotalKVs + 1, TotalSize + KeySize + ValSize, NewUnique}
-        end, Default, Acc)
-    end, #{}, MapRows),
-    ?assertEqual(MapIds, IdsFromMapRows).
-
-
-make_doc(DocId, []) ->
-    case rand:uniform() < 0.5 of
-        true ->
-            #{
-                id => DocId,
-                deleted => true,
-                results => [[]]
-            };
-        false ->
-            #{
-                id => DocId,
-                deleted => false,
-                results => [[]]
-            }
-    end;
-make_doc(DocId, Results) ->
-    #{
-        id => DocId,
-        deleted => false,
-        results => [Results]
-    }.
+    % Row became: key: 0, val: 0, docid: "10"
+    % 1 + 1 + 4 = 6
+    ?assertEqual(6, view_size(Db)).
+
+
+deleting_docs_decreases_size(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, {Pos, Rev}} = fabric2_db:update_doc(Db, Doc1, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    ?assertEqual(5, view_size(Db)),
+
+    Doc2 = Doc1#doc{
+        revs = {Pos, [Rev]},
+        deleted = true,
+        body = {[{<<"val">>, 1}]}
+    },
+    {ok, _} = fabric2_db:update_doc(Db, Doc2, []),
+    {ok, []} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    ?assertEqual(0, view_size(Db)).
 
 
-get_global_size(TxDb) ->
-    #{
-        tx := Tx,
-        db_prefix := DbPrefix
-    } = TxDb,
-    GlobalKey = {?DB_STATS, <<"sizes">>, <<"views">>},
-    BinGlobalKey = erlfdb_tuple:pack(GlobalKey, DbPrefix),
-    ?bin2uint(erlfdb:wait(erlfdb:get(Tx, BinGlobalKey))).
-
-
-init_set({N, D, R}, Labels) ->
-    {Dupes, RestLabels} = fill_keys(D, Labels, []),
-    {Unique, _} = fill_keys(N - D, RestLabels, []),
-    % Sanity assertions
-    N = length(Unique) + length(Dupes),
-    D = length(Dupes),
-    {Unique, [{Key, R} || Key <- Dupes]}.
-
-
-reduce_set({DeltaN, DeltaD, DeltaR}, {Unique, Dupes}) ->
-    NewDupes = lists:sublist(Dupes, DeltaD),
-    NewUnique = lists:sublist(Unique, DeltaN - DeltaD),
-    {NewUnique, [{Key, DeltaR} || {Key, _} <- NewDupes]}.
-
-
-fill_set({N, D, R}, {Unique, Dupes}, Labels) ->
-    AddDupes = D - length(Dupes),
-    {NewDupes, RestLabels} = fill_keys(AddDupes, Labels, Dupes),
-
-    AddUnique = N - length(Unique) - length(NewDupes),
-    {NewUnique, _} = fill_keys(AddUnique, RestLabels, Unique),
-    % Sanity assertions
-    N = length(NewUnique) + length(NewDupes),
-    D = length(NewDupes),
-    {NewUnique, lists:map(fun(Dupe) ->
-        case Dupe of
-            {_, _} -> Dupe;
-            A when is_atom(A) -> {A, R}
-        end
-    end, NewDupes)}.
-
-
-fill_keys(0, Labels, Acc) ->
-    {Acc, Labels};
-fill_keys(Count, [Label | RestLabels], Acc) when Count > 0 ->
-    fill_keys(Count - 1, RestLabels, [Label | Acc]).
-
-
-unlabel({Unique, Dupes}, Bindings) ->
-    lists:foldl(fun(Item, {KVAcc, BindingsAcc}) ->
-        {KVs, NewBindingsAcc} = unlabel_item(Item, BindingsAcc),
-        {KVs ++ KVAcc, NewBindingsAcc}
-    end, {[], Bindings}, Unique ++ Dupes).
-
-
-unlabel_item(Label, Bindings) when is_atom(Label) ->
-    NewBindings = maybe_bind(Label, Bindings),
-    KV = maps:get(Label, NewBindings),
-    {[KV], NewBindings};
-unlabel_item({Label, Count}, Bindings) when is_atom(Label), is_integer(Count) ->
-    NewBindings = maybe_bind(Label, Bindings),
-    {K, _} = KV = maps:get(Label, NewBindings),
-    ToAdd = lists:map(fun(_) ->
-        {K, gen_value()}
-    end,  lists:seq(1, Count - 1)),
-    {[KV | ToAdd], NewBindings}.
-
-
-maybe_bind(Label, Bindings) ->
-    case maps:is_key(Label, Bindings) of
-        true ->
-            case rand:uniform() < 0.5 of
-                true ->
-                    rebind(Label, Bindings);
-                false ->
-                    Bindings
-            end;
-        false ->
-            bind(Label, Bindings)
-    end.
-
-
-bind(Label, Bindings) ->
-    maps:put(Label, {gen_key(), gen_value()}, Bindings).
-
-
-rebind(Label, Bindings) ->
-    {Key, _} = maps:get(Label, Bindings),
-    maps:put(Label, {Key, gen_value()}, Bindings).
-
-
-gen_key() ->
-    Unique = couch_uuids:random(),
-    case rand:uniform() of
-        N when N < 0.2 ->
-            [Unique, true, rand:uniform()];
-        N when N < 0.4 ->
-            {[{Unique, true}, {<<"foo">>, [<<"bar">>, null, 1, {[]}]}]};
-        _ ->
-            Unique
-    end.
-
-
-gen_value() ->
-    case rand:uniform() of
-        N when N < 0.2 ->
-            [false, rand:uniform(), {[]}];
-        N when N < 0.4 ->
-            {[{<<"a">>, 1}, {<<"b">>, 2}]};
-        N when N < 0.6 ->
-            rand:uniform(100);
-        N when N < 0.8 ->
-            rand:uniform();
-        _ ->
-            1
-    end.
-
-
-group(Items) ->
-    case length(Items) > 5 of
-        true ->
-            {Group, Rest} = lists:split(5, Items),
-            [lists:sort(Group) | group(Rest)];
-        false when Items == [] ->
-            [];
-        false ->
-            [lists:sort(Items)]
-    end.
-
-
-shuffle(Items) ->
-    Tagged = [{rand:uniform(), I} || I <- Items],
-    Sorted = lists:sort(Tagged),
-    [I || {_T, I} <- Sorted].
-
-
-subset(Count, Items) ->
-    Random = shuffle(Items),
-    Take = lists:sublist(Random, Count),
-    lists:sort(Take).
-
-
-tname([]) ->
-    [];
-tname([Transition | RestTransitions]) ->
-    [tname(Transition) | tname(RestTransitions)];
-tname({{N1, D1, R1}, {N2, D2, R2}, {DN, DD, DR}}) ->
-    io_lib:format("~b~b~b~b~b~b~b~b~b", [N1, D1, R1, N2, D2, R2, DN, DD, DR]).
+multi_identical_keys_count_twice(Db) ->
+    DDoc = create_ddoc(multi_emit_same),
+    Doc = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, _} = fabric2_db:update_doc(Db, Doc, []),
+    {ok, _} = run_query(Db, DDoc, ?MAP_FUN1),
+
+    % Two rows that are the same
+    ?assertEqual(10, view_size(Db)).
+
+
+multiple_design_docs(Db) ->
+    Cleanup = fun() ->
+        fabric2_fdb:transactional(Db, fun(TxDb) ->
+            DDocs = fabric2_db:get_design_docs(Db),
+            ok = couch_views:cleanup_indices(TxDb, DDocs)
+        end)
+    end,
+
+    DDoc1 = create_ddoc(simple, <<"_design/bar1">>),
+    DDoc2 = create_ddoc(multi_emit_same, <<"_design/bar2">>),
+
+    % Simple test as before
+    {ok, _} = fabric2_db:update_doc(Db, doc(0), []),
+    {ok, {Pos1, Rev1}} = fabric2_db:update_doc(Db, DDoc1, []),
+    {ok, _} = run_query(Db, DDoc1, ?MAP_FUN1),
+    ?assertEqual(5, view_size(Db)),
+
+    % Adding a second ddoc increases the size
+    {ok, {Pos2, Rev2}} = fabric2_db:update_doc(Db, DDoc2, []),
+    {ok, _} = run_query(Db, DDoc2, ?MAP_FUN1),
+    ?assertEqual(15, view_size(Db)),
+
+    % Removing the first ddoc decreases the size
+    DDoc1Del = DDoc1#doc{revs = {Pos1, [Rev1]}, deleted = true},
+    {ok, _} = fabric2_db:update_doc(Db, DDoc1Del, []),
+    Cleanup(),
+    ?assertEqual(10, view_size(Db)),
+
+    % Removing the second ddoc drops the size
+    DDoc2Del = DDoc2#doc{revs = {Pos2, [Rev2]}, deleted = true},
+    {ok, _} = fabric2_db:update_doc(Db, DDoc2Del, []),
+    Cleanup(),
+    ?assertEqual(0, view_size(Db)).
+
+
+multiple_identical_design_docs(Db) ->
+    Cleanup = fun() ->
+        fabric2_fdb:transactional(Db, fun(TxDb) ->
+            DDocs = fabric2_db:get_design_docs(Db),
+            ok = couch_views:cleanup_indices(TxDb, DDocs)
+        end)
+    end,
 
+    DDoc1 = create_ddoc(simple, <<"_design/bar1">>),
+    DDoc2 = create_ddoc(simple, <<"_design/bar2">>),
+
+    % Simple test as before
+    {ok, _} = fabric2_db:update_doc(Db, doc(0), []),
+    {ok, {Pos1, Rev1}} = fabric2_db:update_doc(Db, DDoc1, []),
+    {ok, _} = run_query(Db, DDoc1, ?MAP_FUN1),
+    ?assertEqual(5, view_size(Db)),
+
+    % Adding a second ddoc with the same sig does not double the size
+    {ok, {Pos2, Rev2}} = fabric2_db:update_doc(Db, DDoc2, []),
+    {ok, _} = run_query(Db, DDoc2, ?MAP_FUN1),
+    ?assertEqual(5, view_size(Db)),
+
+    % Removing the first ddoc does not decrease the size
+    DDoc1Del = DDoc1#doc{revs = {Pos1, [Rev1]}, deleted = true},
+    {ok, _} = fabric2_db:update_doc(Db, DDoc1Del, []),
+    Cleanup(),
+    ?assertEqual(5, view_size(Db)),
+
+    % Removing the second ddoc drops the size
+    DDoc2Del = DDoc2#doc{revs = {Pos2, [Rev2]}, deleted = true},
+    {ok, _} = fabric2_db:update_doc(Db, DDoc2Del, []),
+    Cleanup(),
+    ?assertEqual(0, view_size(Db)).
+
+
+view_size(Db) ->
+    {ok, Info} = fabric2_db:get_db_info(Db),
+    {sizes, {Sizes}} = lists:keyfind(sizes, 1, Info),
+    {<<"views">>, ViewSize} = lists:keyfind(<<"views">>, 1, Sizes),
+    ViewSize.
+
+
+create_ddoc() ->
+    create_ddoc(simple).
+
+
+create_ddoc(Type) ->
+    create_ddoc(Type, <<"_design/bar">>).
+
+
+create_ddoc(simple, DocId) when is_binary(DocId) ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, DocId},
+        {<<"views">>, {[
+            {?MAP_FUN1, {[
+                {<<"map">>, <<"function(doc) {emit(doc.val, doc.val);}">>}
+            ]}},
+            {?MAP_FUN2, {[
+                {<<"map">>, <<"function(doc) {}">>}
+            ]}}
+        ]}}
+    ]});
+
+create_ddoc(multi_emit_same, DocId) when is_binary(DocId) ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, DocId},
+        {<<"views">>, {[
+            {?MAP_FUN1, {[
+                {<<"map">>, <<"function(doc) { "
+                    "emit(doc.val, doc.val * 2); "
+                    "emit(doc.val, doc.val); "
+                    "if(doc.extra) {"
+                    "  emit(doc.val, doc.extra);"
+                    "}"
+                "}">>}
+            ]}},
+            {?MAP_FUN2, {[
+                {<<"map">>, <<"function(doc) {}">>}
+            ]}}
+        ]}}
+    ]}).
+
+
+doc(Id) ->
+    doc(Id, Id).
+
+
+doc(Id, Val) ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, list_to_binary(integer_to_list(Id))},
+        {<<"val">>, Val}
+    ]}).
+
+
+run_query(#{} = Db, DDoc, <<_/binary>> = View) ->
+    couch_views:query(Db, DDoc, View, fun fold_fun/2, [], #mrargs{}).
+
+
+fold_fun({meta, _Meta}, Acc) ->
+    {ok, Acc};
+fold_fun({row, _} = Row, Acc) ->
+    {ok, [Row | Acc]};
+fold_fun(complete, Acc) ->
+    {ok, lists:reverse(Acc)}.
\ No newline at end of file