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/07/23 20:13:09 UTC

[couchdb] branch prototype/views created (now 741a115)

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

davisp pushed a change to branch prototype/views
in repository https://gitbox.apache.org/repos/asf/couchdb.git.


      at 741a115  Minor tweaks to remove reduce function tests

This branch includes the following new commits:

     new ed46cb7  CouchDB map indexes on FDB
     new 64be6bf  fixes based on reviews
     new 1c3184f  Fix formatting of couch_views.app.src
     new bb329ab  Expose the is_replicator_db and is_user_db logic
     new bec2da3  Don't redefine records with the same name
     new d0cfb01  Don't add a fabric2_view module
     new b16b5b2  Move jobs logic to couch_view_jobs
     new 5c0b671  Simplify couch_views_worker_server
     new a4be1fb  Include the view sig in the job
     new 14321ba  Simplify worker vs. indexer distinction
     new 6678306  Fix encoding layer for different key types
     new 5be667c  Move all fdb writer logic tou couch_views_fdb.
     new 5d6068e  Move fdb logic to couch_views_fdb
     new 047ccab  fixup! Expose the is_replicator_db and is_user_db logic
     new 4dbf745  Fix compiler errors
     new 2e3da3f  Fix default fold_range bounds
     new 55214bb  Fix code and tests to pass eunit
     new 3c39ce5  More bug fixes
     new cefa111  All tests passing
     new 6119d98  Add Garren's map_test.exs
     new d6758db  Update chttpd for new API
     new 2f26a94  Support the `keys` option
     new 3ed9410  Make fabric2.hrl public
     new 81521e1  Add total row count support
     new 741a115  Minor tweaks to remove reduce function tests

The 25 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.



[couchdb] 01/25: CouchDB map indexes on FDB

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

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

commit ed46cb750969d0c42f0fa91de7d68cc08ba013bd
Author: Garren Smith <ga...@gmail.com>
AuthorDate: Mon Jun 17 15:45:10 2019 +0200

    CouchDB map indexes on FDB
    
    This adds couch_views which builds map indexes and stores them in FDB.
---
 rebar.config.script                                |   1 +
 rel/overlay/etc/default.ini                        |   6 +
 rel/reltool.config                                 |   2 +
 src/chttpd/src/chttpd_db.erl                       |   3 +-
 src/chttpd/src/chttpd_view.erl                     |   2 +-
 src/couch_mrview/src/couch_mrview_util.erl         |   2 +-
 src/couch_views/.gitignore                         |  19 +
 src/couch_views/README.md                          |  16 +
 src/couch_views/include/couch_views.hrl            |  94 ++++
 src/couch_views/rebar.config                       |  14 +
 src/couch_views/src/couch_views.app.src            |  31 ++
 src/couch_views/src/couch_views.erl                | 115 +++++
 src/couch_views/src/couch_views_app.erl            |  31 ++
 src/couch_views/src/couch_views_encoding.erl       | 108 +++++
 src/couch_views/src/couch_views_fdb.erl            | 208 +++++++++
 src/couch_views/src/couch_views_indexer.erl        | 262 +++++++++++
 src/couch_views/src/couch_views_jobs.erl           | 122 ++++++
 src/couch_views/src/couch_views_reader.erl         | 204 +++++++++
 src/couch_views/src/couch_views_sup.erl            |  46 ++
 src/couch_views/src/couch_views_util.erl           |  83 ++++
 src/couch_views/src/couch_views_worker.erl         |  44 ++
 src/couch_views/src/couch_views_worker_server.erl  | 110 +++++
 src/couch_views/test/couch_views_encoding_test.erl |  73 ++++
 src/couch_views/test/couch_views_indexer_test.erl  | 258 +++++++++++
 src/couch_views/test/couch_views_map_test.erl      | 484 +++++++++++++++++++++
 src/fabric/src/fabric2.hrl                         |   1 +
 src/fabric/src/fabric2_view.erl                    |  81 ++++
 test/elixir/test/map_test.exs                      | 222 ++++++++++
 28 files changed, 2639 insertions(+), 3 deletions(-)

diff --git a/rebar.config.script b/rebar.config.script
index 116c040..ce79728 100644
--- a/rebar.config.script
+++ b/rebar.config.script
@@ -82,6 +82,7 @@ SubDirs = [
     "src/couch_stats",
     "src/couch_peruser",
     "src/couch_tests",
+    "src/couch_views",
     "src/ddoc_cache",
     "src/fabric",
     "src/couch_jobs",
diff --git a/rel/overlay/etc/default.ini b/rel/overlay/etc/default.ini
index 8fd2261..59b7d57 100644
--- a/rel/overlay/etc/default.ini
+++ b/rel/overlay/etc/default.ini
@@ -223,6 +223,12 @@ iterations = 10 ; iterations for password hashing
 ; users_db_public = false
 ; cookie_domain = example.com
 
+; Settings for view indexing
+[couch_views]
+; type_check_period_msec = 500
+; type_check_max_jitter_msec = 500
+; change_limit = 100
+
 ; CSP (Content Security Policy) Support for _utils
 [csp]
 enable = true
diff --git a/rel/reltool.config b/rel/reltool.config
index 7b2159d..2b088be 100644
--- a/rel/reltool.config
+++ b/rel/reltool.config
@@ -42,6 +42,7 @@
         couch_stats,
         couch_event,
         couch_peruser,
+        couch_views,
         ddoc_cache,
         ets_lru,
         fabric,
@@ -100,6 +101,7 @@
     {app, couch_stats, [{incl_cond, include}]},
     {app, couch_event, [{incl_cond, include}]},
     {app, couch_peruser, [{incl_cond, include}]},
+    {app, couch_views, [{incl_cond, include}]},
     {app, ddoc_cache, [{incl_cond, include}]},
     {app, ets_lru, [{incl_cond, include}]},
     {app, fabric, [{incl_cond, include}]},
diff --git a/src/chttpd/src/chttpd_db.erl b/src/chttpd/src/chttpd_db.erl
index 0c7e4d5..785ca3f 100644
--- a/src/chttpd/src/chttpd_db.erl
+++ b/src/chttpd/src/chttpd_db.erl
@@ -334,7 +334,8 @@ handle_design_req(#httpd{
         path_parts=[_DbName, _Design, Name, <<"_",_/binary>> = Action | _Rest]
     }=Req, Db) ->
     DbName = fabric2_db:name(Db),
-    case ddoc_cache:open(DbName, <<"_design/", Name/binary>>) of
+%%    case ddoc_cache:open(DbName, <<"_design/", Name/binary>>) of
+    case fabric2_db:open_doc(Db, <<"_design/", Name/binary>>) of
     {ok, DDoc} ->
         Handler = chttpd_handlers:design_handler(Action, fun bad_action_req/3),
         Handler(Req, Db, DDoc);
diff --git a/src/chttpd/src/chttpd_view.erl b/src/chttpd/src/chttpd_view.erl
index 26107d7..849d870 100644
--- a/src/chttpd/src/chttpd_view.erl
+++ b/src/chttpd/src/chttpd_view.erl
@@ -45,7 +45,7 @@ design_doc_view(Req, Db, DDoc, ViewName, Keys) ->
     Max = chttpd:chunked_response_buffer_size(),
     VAcc = #vacc{db=Db, req=Req, threshold=Max},
     Options = [{user_ctx, Req#httpd.user_ctx}],
-    {ok, Resp} = fabric:query_view(Db, Options, DDoc, ViewName,
+    {ok, Resp} = fabric2_view:query(Db, Options, DDoc, ViewName,
             fun view_cb/2, VAcc, Args),
     {ok, Resp#vacc.resp}.
 
diff --git a/src/couch_mrview/src/couch_mrview_util.erl b/src/couch_mrview/src/couch_mrview_util.erl
index eb68124..18a4be1 100644
--- a/src/couch_mrview/src/couch_mrview_util.erl
+++ b/src/couch_mrview/src/couch_mrview_util.erl
@@ -497,7 +497,7 @@ fold_reduce({NthRed, Lang, View}, Fun,  Acc, Options) ->
 
 
 validate_args(Db, DDoc, Args0) ->
-    {ok, State} = couch_mrview_index:init(Db, DDoc),
+    {ok, State} = couch_mrview_util:ddoc_to_mrst(fabric2_db:name(Db), DDoc),
     Args1 = apply_limit(State#mrst.partitioned, Args0),
     validate_args(State, Args1).
 
diff --git a/src/couch_views/.gitignore b/src/couch_views/.gitignore
new file mode 100644
index 0000000..f1c4554
--- /dev/null
+++ b/src/couch_views/.gitignore
@@ -0,0 +1,19 @@
+.rebar3
+_*
+.eunit
+*.o
+*.beam
+*.plt
+*.swp
+*.swo
+.erlang.cookie
+ebin
+log
+erl_crash.dump
+.rebar
+logs
+_build
+.idea
+*.iml
+rebar3.crashdump
+*~
diff --git a/src/couch_views/README.md b/src/couch_views/README.md
new file mode 100644
index 0000000..dba0fcf2
--- /dev/null
+++ b/src/couch_views/README.md
@@ -0,0 +1,16 @@
+CouchDB Views
+=====
+
+This is the new application that builds and runs Map/reduce views against FoundationDB.
+Currently only map indexes are supported and it will always return the full index.
+
+Code layout:
+
+* `couch_views` - Main entry point to query a view
+* `couch_views_reader` - Reads from the index.
+* `couch_views_indexer` - Queries the changes feed from the last sequence and updates the index
+* `couch_views_fdb` - a wrapper around erlfdb
+* `couch_views_encoding` - Emitted key encoding to keep CouchDB sorting rules
+* `couch_views_worker_server` - checks for indexing jobs and spawns a worker to build it
+* `couch_views_worker` - runs couch_views_indexer and builds index along with sending updates back to jobs
+* `couch_views_jobs` - a wrapper around couch_jobs
diff --git a/src/couch_views/include/couch_views.hrl b/src/couch_views/include/couch_views.hrl
new file mode 100644
index 0000000..f5a9c8b
--- /dev/null
+++ b/src/couch_views/include/couch_views.hrl
@@ -0,0 +1,94 @@
+% 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.
+
+% indexing
+-define(VIEW_UPDATE_SEQ, 1).
+-define(VIEW_ID_RANGE,   2).
+-define(VIEW_MAP_RANGE,  3).
+-define(VIEW_BUILDS,     4).
+-define(VIEW_STATUS,     5).
+-define(VIEW_WATCH,      6).
+-define(VIEW_ROW_KEY,    7).
+-define(VIEW_ROW_VALUE,  8).
+
+% jobs api
+-define(INDEX_JOB_TYPE, <<"views">>).
+
+
+-record(mrst, {
+    sig=nil,
+    fd=nil,
+    db_name,
+    idx_name,
+    language,
+    design_opts=[],
+    seq_indexed=false,
+    keyseq_indexed=false,
+    partitioned=false,
+    lib,
+    views,
+    % update_seq=0,
+    % purge_seq=0,
+    % first_build,
+    % partial_resp_pid,
+    % doc_acc,
+    % doc_queue,
+    % write_queue,
+    qserver=nil
+}).
+
+
+-record(mrview, {
+    id_num,
+    % update_seq=0,
+    % purge_seq=0,
+    map_names=[],
+    reduce_funs=[],
+    def,
+    seq_indexed=false,
+    keyseq_indexed=false,
+    options=[]
+}).
+
+
+-define(MAX_VIEW_LIMIT, 16#10000000).
+
+
+-record(mrargs, {
+    view_type,
+    % reduce,
+
+    % preflight_fun,
+
+    start_key,
+    start_key_docid,
+    end_key,
+    end_key_docid,
+    keys,
+
+    direction = fwd,
+    limit = ?MAX_VIEW_LIMIT,
+    skip = 0,
+    % group_level = 0,
+    % group = undefined,
+    stable = false,
+    update = true,
+    multi_get = false,
+    inclusive_end = true,
+    include_docs = false,
+    doc_options = [],
+    update_seq=false,
+    conflicts,
+    % callback,
+    sorted = true
+    % extra = []
+}).
diff --git a/src/couch_views/rebar.config b/src/couch_views/rebar.config
new file mode 100644
index 0000000..362c878
--- /dev/null
+++ b/src/couch_views/rebar.config
@@ -0,0 +1,14 @@
+% 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.
+
+{cover_enabled, true}.
+{cover_print_enabled, true}.
diff --git a/src/couch_views/src/couch_views.app.src b/src/couch_views/src/couch_views.app.src
new file mode 100644
index 0000000..9e1bbe7
--- /dev/null
+++ b/src/couch_views/src/couch_views.app.src
@@ -0,0 +1,31 @@
+% 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.
+
+{application, couch_views,
+ [{description, "CouchDB Views on FDB"},
+  {vsn, git},
+  {mod, {couch_views_app, []}},
+  {registered, [
+    couch_views_sup,
+    couch_views_worker_server
+  ]},
+  {applications, [
+    kernel,
+    stdlib,
+    erlfdb,
+    couch_log,
+    config,
+    couch_stats,
+    fabric,
+    couch_jobs
+   ]}
+ ]}.
diff --git a/src/couch_views/src/couch_views.erl b/src/couch_views/src/couch_views.erl
new file mode 100644
index 0000000..4ccf0fa
--- /dev/null
+++ b/src/couch_views/src/couch_views.erl
@@ -0,0 +1,115 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views).
+
+-export([
+    map_query/6
+]).
+
+-include("couch_views.hrl").
+
+
+map_query(Db, DDoc, ViewName, Callback, Acc0, Args0) ->
+    Args = process_args(Args0),
+    #{name := DbName} = Db,
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+    maybe_build_index(Db, Mrst, Args),
+    Resp = couch_views_reader:read(Db, DDoc, ViewName, Callback, Acc0, Args),
+
+    UpdateAfter = maps:get(update, Args) == lazy,
+    if UpdateAfter == false -> ok; true ->
+        maybe_add_couch_job(Db, Mrst)
+    end,
+    Resp.
+
+
+process_args(#{} = Args) ->
+    Args1 = maps:filter(fun (_, V) -> V /= undefined end, Args),
+
+    maps:merge(#{
+        direction => fwd,
+        inclusive_end => true,
+        update => true,
+        skip => 0,
+        limit => ?MAX_VIEW_LIMIT
+    }, Args1).
+
+
+maybe_build_index(_Db, _Mrst, #{update := false}) ->
+    false;
+
+maybe_build_index(_Db, _Mrst, #{update := lazy}) ->
+    false;
+
+maybe_build_index(Db, Mrst, _Args) ->
+    {Status, Seq} = fabric2_fdb:transactional(Db, fun(TxDb) ->
+        case view_up_to_date(TxDb, Mrst) of
+            {true, UpdateSeq} ->
+                {ready, UpdateSeq};
+            {false, LatestSeq} ->
+                maybe_add_couch_job(TxDb, Mrst),
+                {false, LatestSeq}
+        end
+    end),
+
+    if Status == ready -> true; true ->
+        subscribe_and_wait_for_index(Db, Mrst, Seq)
+    end.
+
+
+view_up_to_date(Db, Mrst) ->
+    fabric2_fdb:transactional(Db, fun(TxDb) ->
+        UpdateSeq = couch_views_fdb:get_update_seq(TxDb, Mrst),
+        LastChange = fabric2_fdb:get_last_change(TxDb),
+        {UpdateSeq == LastChange, LastChange}
+    end).
+
+
+maybe_add_couch_job(TxDb, Mrst) ->
+    case couch_views_jobs:status(TxDb, Mrst) of
+        running ->
+            ok;
+        pending ->
+            ok;
+        Status when Status == finished orelse Status == not_found ->
+            couch_views_jobs:add(TxDb, Mrst)
+    end.
+
+
+subscribe_and_wait_for_index(Db, Mrst, Seq) ->
+    case couch_views_jobs:subscribe(Db, Mrst) of
+        {error, Error} ->
+            throw({error, Error});
+        {ok, finished, _} ->
+            ready;
+        {ok, Subscription, _JobState, _} ->
+            wait_for_index_ready(Subscription, Db, Mrst, Seq)
+    end.
+
+
+wait_for_index_ready(Subscription, Db, Mrst, Seq) ->
+    Out = couch_views_jobs:wait(Subscription),
+    case Out of
+        {finished, _JobData} ->
+            ready;
+        {pending, _JobData} ->
+            wait_for_index_ready(Subscription, Db, Mrst, Seq);
+        {running, #{last_seq := LastSeq}} ->
+            if LastSeq =< Seq -> ready; true ->
+                wait_for_index_ready(Subscription, Db, Mrst, Seq)
+            end;
+        {running, _JobData} ->
+            wait_for_index_ready(Subscription, Db, Mrst, Seq);
+        {error, Error} ->
+            throw({error, Error})
+    end.
diff --git a/src/couch_views/src/couch_views_app.erl b/src/couch_views/src/couch_views_app.erl
new file mode 100644
index 0000000..5ede5ef
--- /dev/null
+++ b/src/couch_views/src/couch_views_app.erl
@@ -0,0 +1,31 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+
+-module(couch_views_app).
+
+
+-behaviour(application).
+
+
+-export([
+    start/2,
+    stop/1
+]).
+
+
+start(_StartType, StartArgs) ->
+    couch_views_sup:start_link(StartArgs).
+
+
+stop(_State) ->
+    ok.
diff --git a/src/couch_views/src/couch_views_encoding.erl b/src/couch_views/src/couch_views_encoding.erl
new file mode 100644
index 0000000..3af6d7f
--- /dev/null
+++ b/src/couch_views/src/couch_views_encoding.erl
@@ -0,0 +1,108 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_encoding).
+
+
+-export([
+    encode/1,
+    decode/1
+]).
+
+
+-define(NULL, 16#00).
+-define(FALSE, 16#26).
+-define(TRUE, 16#27).
+-define(NUMBER, 16#40).
+-define(STRING, 16#41).
+-define(LIST, 16#42).
+-define(OBJECT, 16#43).
+
+
+encode(X) ->
+    Encoded = encode_int(X),
+    erlfdb_tuple:pack(Encoded).
+
+
+decode(EncodedVal) ->
+    Val = erlfdb_tuple:unpack(EncodedVal),
+    decode_int(Val).
+
+
+encode_int(X) when is_atom(X) -> encode_atom(X);
+encode_int(X) when is_number(X) -> encode_number(X);
+encode_int(X) when is_binary(X) -> encode_binary(X);
+encode_int(X) when is_list(X) -> encode_list(X);
+encode_int(X) when is_tuple(X) -> encode_object(X).
+
+
+encode_atom(null) ->
+    {?NULL};
+
+encode_atom(false) ->
+    {?FALSE};
+
+encode_atom(true) ->
+    {?TRUE}.
+
+
+encode_number(Val) ->
+    {?NUMBER, float(Val)}.
+
+
+encode_binary(Val) ->
+    % TODO add sort strings
+    {?STRING, Val}.
+
+
+encode_list(List) ->
+    EncodedItems = lists:map(fun encode_int/1, List),
+    {?LIST, list_to_tuple(EncodedItems)}.
+
+
+encode_object({Props}) ->
+    EncodedProps = lists:map(fun({K, V}) -> 
+        EncodedK = encode_int(K),
+        EncodedV = encode_int(V),
+        {EncodedK, EncodedV}
+    end, Props),
+    {?OBJECT, list_to_tuple(EncodedProps)}.
+
+
+decode_int({?NULL}) ->
+    null;
+
+decode_int({?FALSE}) ->
+    false;
+
+decode_int({?TRUE}) ->
+    true;
+
+decode_int({?STRING, String}) ->
+    String;
+
+decode_int({?NUMBER, Number}) ->
+    case Number - trunc(Number) of
+        0 -> trunc(Number); % convert to integer
+        _ -> Number
+    end;
+
+decode_int({?LIST, List}) ->
+    lists:map(fun decode_int/1, tuple_to_list(List));
+
+decode_int({?OBJECT, Object}) ->
+    Props = lists:map(fun({EncodedK, EncodedV}) ->
+        K = decode_int(EncodedK),
+        V = decode_int(EncodedV),
+        {K, V}
+    end, tuple_to_list(Object)),
+    {Props}.
diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
new file mode 100644
index 0000000..0791ffa
--- /dev/null
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -0,0 +1,208 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_fdb).
+
+-export([
+    get_update_seq/2,
+    update_view_seq/3,
+    get_seq_key/2,
+
+    clear_id_index/4,
+    set_id_index/5,
+    get_id_index/4,
+    create_id_index_key/4,
+
+    clear_map_index/5,
+    set_map_index_results/5,
+    get_map_index_key/4,
+    get_map_range_keys/3,
+    get_map_range/4,
+    unpack_map_row/3
+]).
+
+
+-define(LIST_VALUE, 0).
+-define(JSON_VALUE, 1).
+-define(VALUE, 2).
+
+
+-include_lib("fabric/src/fabric2.hrl").
+-include("couch_views.hrl").
+
+% View Build Sequence Access
+% (<db>, ?DB_VIEWS, Sig, ?VIEW_UPDATE_SEQ) = Sequence
+
+get_update_seq(Db, #mrst{sig = Sig}) ->
+    #{
+        db_prefix := DbPrefix
+    } = Db,
+
+    fabric2_fdb:transactional(Db, fun(TxDb) ->
+        Key = get_seq_key(Sig, DbPrefix),
+        Tx = maps:get(tx, TxDb),
+        case erlfdb:wait(erlfdb:get(Tx, Key)) of
+            not_found -> 0;
+            UpdateSeq -> UpdateSeq
+        end
+    end).
+
+
+update_view_seq(Db, Sig, Seq) ->
+    fabric2_fdb:transactional(Db, fun(TxDb) ->
+        #{
+            db_prefix := DbPrefix,
+            tx := Tx
+        } = TxDb,
+        SeqKey = get_seq_key(Sig, DbPrefix),
+        erlfdb:set(Tx, SeqKey, Seq)
+    end).
+
+
+get_seq_key(Sig, DbPrefix) ->
+    erlfdb_tuple:pack({?DB_VIEWS, Sig, ?VIEW_UPDATE_SEQ}, DbPrefix).
+
+
+% Id Index access
+
+% (<db>, ?VIEWS, <sig>, ?VIEW_ID_INDEX, <_id>, <view_id>) -> [emitted keys]
+
+clear_id_index(TxDb, Sig, DocId, IdxName) ->
+    #{
+        db_prefix := DbPrefix,
+        tx := Tx
+    } = TxDb,
+    IdKey = create_id_index_key(DbPrefix, Sig, DocId, IdxName),
+    ok = erlfdb:clear(Tx, IdKey).
+
+
+set_id_index(TxDb, Sig, IdxName, DocId, IdxKey) ->
+    #{
+        db_prefix := DbPrefix,
+        tx := Tx
+    } = TxDb,
+    IdKey = create_id_index_key(DbPrefix, Sig, DocId, IdxName),
+    erlfdb:set(Tx, IdKey, couch_views_encoding:encode(IdxKey)).
+
+
+get_id_index(TxDb, Sig, Id, IdxName) ->
+    #{
+        db_prefix := DbPrefix,
+        tx := Tx
+    } = TxDb,
+    IdKey = create_id_index_key(DbPrefix, Sig, Id, IdxName),
+    case erlfdb:wait(erlfdb:get(Tx, IdKey)) of
+        not_found -> not_found;
+        IdxKey -> couch_views_encoding:decode(IdxKey)
+    end.
+
+
+create_id_index_key(DbPrefix, Sig, DocId, IdxName) ->
+    BaseIdKey = {?DB_VIEWS, Sig, ?VIEW_ID_RANGE, DocId, IdxName},
+    erlfdb_tuple:pack(BaseIdKey, DbPrefix).
+
+
+% Map Index Access
+% {<db>, ?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, Idx, Key, DocId,
+%   RowType, Counter} = Values
+% RowType = Emitted Keys or Emitted Value
+
+
+clear_map_index(TxDb, Sig, IdxName, DocId, IdxKeys) when is_list(IdxKeys) ->
+    lists:foreach(fun (IdxKey) ->
+        clear_map_index(TxDb, Sig, IdxName, DocId, IdxKey)
+    end, IdxKeys);
+
+clear_map_index(TxDb, Sig, IdxName, DocId, IdxKey) ->
+    #{db_prefix := DbPrefix, tx := Tx} = TxDb,
+    Key = couch_views_encoding:encode(IdxKey),
+    BaseKey = {?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, IdxName, Key, DocId},
+    {StartKey, EndKey} = erlfdb_tuple:range(BaseKey, DbPrefix),
+    ok = erlfdb:clear_range(Tx, StartKey, EndKey).
+
+
+set_map_index_results(TxDb, Sig, IdxName, DocId, Results) ->
+    #{db_prefix := DbPrefix, tx := Tx} = TxDb,
+    lists:foldl(fun ({IdxKey, IdxValue}, Counter) ->
+        RowKey = create_map_key(DbPrefix, Sig, IdxName, IdxKey, DocId,
+            ?VIEW_ROW_KEY, Counter),
+        RowValue = create_map_key(DbPrefix, Sig, IdxName, IdxKey, DocId,
+            ?VIEW_ROW_VALUE, Counter),
+
+        EncodedKey = pack_value(IdxKey),
+        EncodedValue = pack_value(IdxValue),
+
+        ok = erlfdb:set(Tx, RowKey, EncodedKey),
+        ok = erlfdb:set(Tx, RowValue, EncodedValue),
+        Counter + 1
+    end, 0, Results).
+
+
+get_map_index_key(#{db_prefix := DbPrefix}, Sig, IdxName, Key) ->
+    EncKey = couch_views_encoding:encode(Key),
+    erlfdb_tuple:pack({?DB_VIEWS, Sig, ?VIEW_MAP_RANGE,
+            IdxName, EncKey}, DbPrefix).
+
+
+get_map_range_keys(#{db_prefix := DbPrefix}, Sig, IdxName) ->
+    erlfdb_tuple:range({?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, IdxName}, DbPrefix).
+
+
+get_map_range(TxDb, Start, End, Opts) ->
+    #{tx := Tx} = TxDb,
+    erlfdb:get_range(Tx, Start, End, Opts).
+
+
+unpack_map_row(#{db_prefix := DbPrefix}, Key, Value) ->
+    case erlfdb_tuple:unpack(Key, DbPrefix) of
+        {?DB_VIEWS, _Sig, ?VIEW_MAP_RANGE, _Idx, _RowKey, Id,
+            ?VIEW_ROW_KEY, _Counter} ->
+            RowKey = unpack_value(Value),
+            {key, Id, RowKey};
+
+        {?DB_VIEWS, _Sig, ?VIEW_MAP_RANGE, _Idx, _RowValue, Id,
+            ?VIEW_ROW_VALUE, _Counter} ->
+            RowValue = unpack_value(Value),
+            {value, Id, RowValue}
+    end.
+
+
+create_map_key(DbPrefix, Sig, IdxName, IdxKey, DocId, RowType, Counter) ->
+    Key = couch_views_encoding:encode(IdxKey),
+    BaseKey = {?DB_VIEWS, Sig, ?VIEW_MAP_RANGE,
+        IdxName, Key, DocId, RowType, Counter},
+    erlfdb_tuple:pack(BaseKey, DbPrefix).
+
+
+% Internal used to packed and unpack Values
+
+
+pack_value(Val) when is_list(Val) ->
+    erlfdb_tuple:pack({?LIST_VALUE, list_to_tuple(Val)});
+
+pack_value(Val) when is_tuple(Val) ->
+    {Props} = Val,
+    erlfdb_tuple:pack({?JSON_VALUE, list_to_tuple(Props)});
+
+pack_value(Val) ->
+    erlfdb_tuple:pack({?VALUE, Val}).
+
+
+unpack_value(Bin) ->
+    case erlfdb_tuple:unpack(Bin) of
+        {?LIST_VALUE, Val} ->
+            tuple_to_list(Val);
+        {?JSON_VALUE, Val} ->
+            {tuple_to_list(Val)};
+        {?VALUE, Val} ->
+            Val
+    end.
diff --git a/src/couch_views/src/couch_views_indexer.erl b/src/couch_views/src/couch_views_indexer.erl
new file mode 100644
index 0000000..e9f0b41
--- /dev/null
+++ b/src/couch_views/src/couch_views_indexer.erl
@@ -0,0 +1,262 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_indexer).
+
+-export([
+    update/2,
+    update/4,
+
+    % For tests
+    map_docs/2,
+    write_doc/4
+]).
+
+
+-include("couch_views.hrl").
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("fabric/src/fabric2.hrl").
+-include_lib("eunit/include/eunit.hrl").
+
+% TODO: 
+%  * Handle timeouts of transaction and other errors
+
+update(Db, Mrst) ->
+    Noop = fun (_) -> ok end,
+    update(Db, Mrst, Noop, []).
+
+
+update(#{} = Db, Mrst, ProgressCallback, ProgressArgs)
+        when is_function(ProgressCallback, 6) ->
+    try
+        Seq = couch_views_fdb:get_update_seq(Db, Mrst),
+        State = #{
+            since_seq => Seq,
+            count => 0,
+            limit => config:get_integer("couch_views", "change_limit", 100),
+            doc_acc => [],
+            last_seq => Seq,
+            callback => ProgressCallback,
+            callback_args => ProgressArgs,
+            mrst => Mrst
+        },
+        update_int(Db, State)
+    catch error:database_does_not_exist ->
+        #{db_prefix := DbPrefix} = Db,
+        couch_log:notice("couch_views_indexer stopped"
+        "- ~p database does not exist", [DbPrefix])
+    end.
+
+
+update_int(#{} = Db, State) ->
+    {ok, FinalState} = fabric2_fdb:transactional(Db, fun(TxDb) ->
+        State1 = maps:put(tx_db, TxDb, State),
+        fold_changes(State1)
+    end),
+
+    #{
+        count := Count,
+        limit := Limit,
+        doc_acc := DocAcc,
+        last_seq := LastSeq,
+        callback := Cb,
+        callback_args := CallbackArgs,
+        mrst := Mrst
+    } = FinalState,
+
+    {MappedResults, Mrst1} = map_docs(Mrst, DocAcc),
+    write_docs(Db, Mrst1, MappedResults, FinalState),
+
+    case Count < Limit of
+        true ->
+            Cb(undefined, finished, CallbackArgs, Db, Mrst, LastSeq);
+        false ->
+            NextState = maps:merge(FinalState, #{
+                limit => Limit,
+                count => 0,
+                doc_acc => [],
+                since_seq => LastSeq,
+                last_seq => 0,
+                mrst => Mrst1
+            }),
+            update_int(Db, NextState)
+    end.
+
+
+fold_changes(State) ->
+    #{
+        since_seq := SinceSeq,
+        limit := Limit,
+        tx_db := TxDb
+    } = State,
+
+    fabric2_db:fold_changes(TxDb, SinceSeq,
+        fun process_changes/2, State, [{limit, Limit}]).
+
+
+process_changes(Change, Acc) ->
+    #{
+        doc_acc := DocAcc,
+        count := Count,
+        tx_db := TxDb,
+        mrst := Mrst
+    } = Acc,
+
+    #{
+        id := Id,
+        sequence := LastSeq,
+        deleted := Deleted
+    } = Change,
+
+    IncludeDesign = lists:keymember(<<"include_design">>, 1,
+        Mrst#mrst.design_opts),
+
+    Acc1 = case {Id, IncludeDesign} of
+        {<<"_design/", _/binary>>, false} ->
+            % {ok, Doc} = fabric2_db:open_doc(Db, Id),
+            maps:merge(Acc, #{
+                count => Count + 1,
+                last_seq => LastSeq
+                });
+        _ ->
+
+            % Making a note here that we should make fetching all the docs
+            % a parallel fdb operation
+            Doc = if Deleted -> []; true ->
+                case fabric2_db:open_doc(TxDb, Id) of
+                    {ok, Doc0} -> Doc0;
+                    {not_found, _} -> []
+                end
+            end,
+
+            Change1 = maps:put(doc, Doc, Change),
+            maps:merge(Acc, #{
+                doc_acc => DocAcc ++ [Change1],
+                count => Count + 1,
+                last_seq => LastSeq
+            })
+    end,
+    {ok, Acc1}.
+
+
+map_docs(Mrst, Docs) ->
+    % Run all the non deleted docs through the view engine and
+    Mrst1 = get_query_server(Mrst),
+    QServer = Mrst1#mrst.qserver,
+
+    MapFun = fun
+        (#{deleted := true} = Change) ->
+            maps:put(results, [], Change);
+
+        (Change) ->
+            #{doc := Doc} = Change,
+            couch_stats:increment_counter([couchdb, mrview, map_doc]),
+            {ok, RawResults} = couch_query_servers:map_doc_raw(QServer, Doc),
+            JsonResults = couch_query_servers:raw_to_ejson(RawResults),
+            ListResults = [[list_to_tuple(Res) || Res <- FunRs]
+                || FunRs <- JsonResults],
+            maps:put(results, ListResults, Change)
+    end,
+    MappedResults = lists:map(MapFun, Docs),
+    {MappedResults, Mrst1}.
+
+
+start_query_server(#mrst{} = Mrst) ->
+    #mrst{
+        language=Language,
+        lib=Lib,
+        views=Views
+    } = Mrst,
+    Defs = [View#mrview.def || View <- Views],
+    {ok, QServer} = couch_query_servers:start_doc_map(Language, Defs, Lib),
+    Mrst#mrst{qserver=QServer}.
+
+
+get_query_server(#mrst{} = Mrst) ->
+    case Mrst#mrst.qserver of
+        nil -> start_query_server(Mrst);
+        _ -> Mrst
+    end.
+
+
+write_docs(Db, Mrst, Docs, State) ->
+    #mrst{
+        views = Views,
+        sig = Sig
+    } = Mrst,
+
+    #{
+        callback := Cb,
+        callback_args := CallbackArgs
+    } = State,
+
+    IdxNames = lists:map(fun (View) ->
+        View#mrview.id_num
+    end, Views),
+
+    lists:foreach(fun (Doc) ->
+        #{sequence := Seq} = Doc,
+        fabric2_fdb:transactional(Db, fun(TxDb) ->
+            couch_views_fdb:update_view_seq(TxDb, Sig, Seq),
+            Cb(TxDb, update, CallbackArgs, Db, Mrst, Seq),
+            write_doc(TxDb, Sig, Doc, IdxNames)
+        end)
+    end, Docs).
+
+
+write_doc(TxDb, Sig, #{deleted := true} = Doc, ViewIds) ->
+    #{id := DocId} = Doc,
+    lists:foreach(fun (IdxName) ->
+        maybe_clear_id_and_map_index(TxDb, Sig, DocId, IdxName)
+    end, ViewIds);
+
+write_doc(TxDb, Sig, Doc, ViewIds) ->
+    #{id := DocId, results := Results} = Doc,
+    lists:foreach(fun
+        ({IdxName, []}) ->
+            maybe_clear_id_and_map_index(TxDb, Sig, DocId, IdxName);
+        ({IdxName, IdxResults}) ->
+            lists:foldl(fun (IdxResult, DocIdsCleared) ->
+                {IdxKey, _} = IdxResult,
+                OldIdxKey = couch_views_fdb:get_id_index(TxDb, Sig,
+                    DocId, IdxName),
+                IsAlreadyCleared = lists:member(DocId, DocIdsCleared),
+                case OldIdxKey == not_found orelse IsAlreadyCleared == true of
+                    true ->
+                        couch_views_fdb:set_id_index(TxDb, Sig, IdxName,
+                            DocId, IdxKey),
+                        couch_views_fdb:set_map_index_results(TxDb, Sig,
+                            IdxName, DocId, IdxResults);
+                    false ->
+                        couch_views_fdb:clear_id_index(TxDb, Sig,
+                            DocId, IdxName),
+                        couch_views_fdb:clear_map_index(TxDb, Sig, IdxName,
+                            DocId, OldIdxKey),
+                        couch_views_fdb:set_id_index(TxDb, Sig, DocId,
+                            IdxName, IdxKey),
+                        couch_views_fdb:set_map_index_results(TxDb, Sig,
+                            IdxName, DocId, IdxResults)
+                end,
+                [DocId | DocIdsCleared]
+            end, [], IdxResults)
+    end, lists:zip(ViewIds, Results)).
+
+
+maybe_clear_id_and_map_index(TxDb, Sig, DocId, IdxName) ->
+    OldIdxKey = couch_views_fdb:get_id_index(TxDb, Sig,
+        DocId, IdxName),
+    if OldIdxKey == not_found -> ok; true ->
+        couch_views_fdb:clear_id_index(TxDb, Sig,
+            DocId, IdxName),
+        couch_views_fdb:clear_map_index(TxDb, Sig, IdxName,
+            DocId, OldIdxKey)
+    end.
diff --git a/src/couch_views/src/couch_views_jobs.erl b/src/couch_views/src/couch_views_jobs.erl
new file mode 100644
index 0000000..ff99475
--- /dev/null
+++ b/src/couch_views/src/couch_views_jobs.erl
@@ -0,0 +1,122 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_jobs).
+
+-export([
+    status/2,
+    add/2,
+
+    accept/0,
+    get_job_data/1,
+    update/5,
+    finish/5,
+    set_timeout/0,
+
+    subscribe/2,
+    wait/1,
+    unsubscribe/1,
+
+    create_job_id/2
+]).
+
+
+-include("couch_views.hrl").
+
+
+% Query request usage of jobs
+
+
+status(TxDb, Mrst) ->
+    JobId = create_job_id(TxDb, Mrst),
+
+    case couch_jobs:get_job_state(TxDb, ?INDEX_JOB_TYPE, JobId) of
+        {ok, State} -> State;
+        {error, not_found} -> not_found;
+        Error -> Error
+    end.
+
+
+add(TxDb, Mrst) ->
+    JobData = create_job_data(TxDb, Mrst, 0),
+
+    JobId = create_job_id(TxDb, Mrst),
+    JTx = couch_jobs_fdb:get_jtx(),
+    couch_jobs:add(JTx, ?INDEX_JOB_TYPE, JobId, JobData).
+
+
+% couch_views_worker api
+
+
+accept() ->
+    couch_jobs:accept(?INDEX_JOB_TYPE).
+
+
+get_job_data(JobId) ->
+    couch_jobs:get_job_data(undefined, ?INDEX_JOB_TYPE, JobId).
+
+
+update(JTx, Job, Db, Mrst, LastSeq) ->
+    JobData = create_job_data(Db, Mrst, LastSeq),
+    couch_jobs:update(JTx, Job, JobData).
+
+
+finish(JTx, Job, Db, Mrst, LastSeq) ->
+    JobData = create_job_data(Db, Mrst, LastSeq),
+    couch_jobs:finish(JTx, Job, JobData).
+
+
+set_timeout() ->
+    couch_jobs:set_type_timeout(?INDEX_JOB_TYPE, 6 * 1000).
+
+
+% Watcher Job api
+
+
+subscribe(Db, Mrst) ->
+    JobId = create_job_id(Db, Mrst),
+    couch_jobs:subscribe(?INDEX_JOB_TYPE, JobId).
+
+
+wait(JobSubscription) ->
+    case couch_jobs:wait(JobSubscription, infinity) of
+        {?INDEX_JOB_TYPE, _JobId, JobState, JobData} -> {JobState, JobData};
+        {timeout} -> {error, timeout}
+    end.
+
+
+unsubscribe(JobSubscription) ->
+    couch_jobs:unsubscribe(JobSubscription).
+
+
+% Internal
+
+
+create_job_id(#{name := DbName}, #mrst{sig = Sig}) ->
+    create_job_id(DbName, Sig);
+
+create_job_id(DbName, Sig) ->
+    <<DbName/binary, Sig/binary>>.
+
+
+create_job_data(Db, Mrst, LastSeq) ->
+    #{name := DbName} = Db,
+
+    #mrst{
+        idx_name = DDocId
+    } = Mrst,
+
+    #{
+        db_name => DbName,
+        ddoc_id => DDocId,
+        last_seq => LastSeq
+    }.
diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
new file mode 100644
index 0000000..2ddb5b6
--- /dev/null
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -0,0 +1,204 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_reader).
+
+-export([
+    read/6
+]).
+
+
+-include("couch_views.hrl").
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("fabric/src/fabric2.hrl").
+
+
+read(Db, DDoc, ViewName, Callback, Acc0, Args) ->
+    #{name := DbName} = Db,
+
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+    #mrst{
+        sig = Sig,
+        views = Views
+    } = Mrst,
+
+    IdxName = get_idx_name(ViewName, Views),
+    State0 = #{
+        acc => Acc0,
+        skip => maps:get(skip, Args, 0),
+        include_docs => maps:get(include_docs, Args, false),
+        db => Db
+    },
+
+    DefaultOpts = [{streaming_mode, want_all}],
+    {Start, End, QueryOpts} = convert_args_to_fdb(Db, Sig, IdxName, Args,
+        DefaultOpts),
+    Opts = QueryOpts ++ DefaultOpts,
+
+    fabric2_fdb:transactional(Db, fun(TxDb) ->
+        Future = couch_views_fdb:get_map_range(TxDb, Start, End, Opts),
+
+        UnPack = get_unpack_fun(TxDb, Opts, Callback),
+        State1 = lists:foldl(UnPack, State0, erlfdb:wait(Future)),
+
+        #{acc := Acc1} = State1,
+        Callback(complete, Acc1)
+    end).
+
+
+get_idx_name(ViewName, Views) ->
+    {value, View} = lists:search(fun (View) ->
+        lists:member(ViewName, View#mrview.map_names)
+    end, Views),
+    View#mrview.id_num.
+
+
+convert_args_to_fdb(Db, Sig, IdxName, Args, Opts) ->
+    #{
+        direction := Direction
+    } = Args,
+
+    {Start1, End1} = get_range_keys(Db, Sig, IdxName, Args),
+
+    Opts1 = case maps:is_key(limit, Args) of
+        false ->
+            Opts;
+        true ->
+            Skip = maps:get(skip, Args, 0),
+            Limit = maps:get(limit, Args),
+            % Limit is multiplied by two because there are two rows per key
+            % value.
+            % Skip is added because that is done in the fold so we need
+            % to fetch the number of documents
+            % along with the docs we would skip.
+            % Limit = (Doc limit + Skip) * Num of Rows per Map KV
+            [{limit, (Limit + Skip) * 2} | Opts]
+    end,
+
+    Opts2 = case Direction of
+        fwd ->
+            Opts1;
+        rev ->
+            [{reverse, true} | Opts1]
+    end,
+    {Start1, End1, Opts2}.
+
+
+get_range_keys(Db, Sig, IdxName, Args) ->
+    #{
+        inclusive_end := InclusiveEnd,
+        direction := Direction
+    } = Args,
+
+    {MapStartKey, MapEndKey} = case Direction of
+        fwd -> {start_key, end_key};
+        rev -> {end_key, start_key}
+    end,
+
+    {Start0, End0} = couch_views_fdb:get_map_range_keys(Db, Sig, IdxName),
+
+    Start1 = case maps:is_key(MapStartKey, Args) of
+        false ->
+            Start0;
+        true ->
+            StartKey = maps:get(MapStartKey, Args),
+            Start = couch_views_fdb:get_map_index_key(Db, Sig, IdxName,
+                StartKey),
+            erlfdb_key:first_greater_or_equal(Start)
+    end,
+
+    End1 = case maps:is_key(MapEndKey, Args) of
+        false ->
+            End0;
+        true ->
+            EndKey = maps:get(MapEndKey, Args),
+            EndBin = couch_views_fdb:get_map_index_key(Db, Sig, IdxName,
+                EndKey),
+            EndBin1 = case InclusiveEnd of
+                true -> <<EndBin/binary, 16#FF>>;
+                false -> EndBin
+            end,
+            erlfdb_key:first_greater_than(EndBin1)
+    end,
+    {Start1, End1}.
+
+
+get_unpack_fun(TxDb, Opts, Callback) ->
+    UnPackFwd = fun({K, V}, State) ->
+        case couch_views_fdb:unpack_map_row(TxDb, K, V) of
+            {key, _Id, RowKey} ->
+                maps:put(current_key, RowKey, State);
+            {value, Id, RowValue} ->
+                #{
+                    current_key := RowKey,
+                    acc := Acc,
+                    skip := Skip,
+                    db := Db
+                } = State,
+
+                case Skip > 0 of
+                    true ->
+                        maps:put(skip, Skip - 1, State);
+                    false ->
+                        Row = [{id, Id}, {key, RowKey}, {value, RowValue}],
+
+                        IncludeDoc = maps:get(include_docs, State, false),
+                        Row1 = maybe_include_doc(Db, Id, Row, IncludeDoc),
+
+                        {ok, AccNext} = Callback({row, Row1}, Acc),
+                        maps:put(acc, AccNext, State)
+                end
+        end
+    end,
+
+    UnPackRev = fun({K, V}, State) ->
+        case couch_views_fdb:unpack_map_row(TxDb, K, V) of
+            {key, Id, RowKey} ->
+                #{
+                    current_value := RowValue,
+                    acc := Acc,
+                    skip := Skip,
+                    db := Db
+                } = State,
+
+                case Skip > 0 of
+                    true ->
+                        maps:put(skip, Skip - 1, State);
+                    false ->
+                        Row = [{id, Id}, {key, RowKey}, {value, RowValue}],
+
+                        IncludeDoc = maps:get(include_docs, State, false),
+                        Row1 = maybe_include_doc(Db, Id, Row, IncludeDoc),
+
+                        {ok, AccNext} = Callback({row, Row1}, Acc),
+                        maps:put(acc, AccNext, State)
+                end;
+            {value, _Id, RowValue} ->
+                maps:put(current_value, RowValue, State)
+        end
+    end,
+
+    case lists:keyfind(reverse, 1, Opts) of
+        {reverse, true} -> UnPackRev;
+        _ -> UnPackFwd
+    end.
+
+
+maybe_include_doc(_Db, _Id, Row, false) ->
+    Row;
+
+maybe_include_doc(Db, Id, Row, true) ->
+    Doc1 = case fabric2_db:open_doc(Db, Id) of
+        {ok, Doc} -> couch_doc:to_json_obj(Doc, []);
+        {not_found, _} -> []
+    end,
+    Row ++ [{doc, Doc1}].
diff --git a/src/couch_views/src/couch_views_sup.erl b/src/couch_views/src/couch_views_sup.erl
new file mode 100644
index 0000000..da7d796
--- /dev/null
+++ b/src/couch_views/src/couch_views_sup.erl
@@ -0,0 +1,46 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+
+-module(couch_views_sup).
+
+
+-behaviour(supervisor).
+
+
+-export([
+    start_link/1
+]).
+
+
+-export([
+    init/1
+]).
+
+
+start_link(Args) ->
+    supervisor:start_link({local, ?MODULE}, ?MODULE, Args).
+
+
+init([]) ->
+    Flags = #{
+        strategy => one_for_one,
+        intensity => 1,
+        period => 5
+    },
+    Children = [
+        #{
+            id => couch_views_worker_server,
+            start => {couch_views_worker_server, start_link, []}
+        }
+    ],
+    {ok, {Flags, Children}}.
diff --git a/src/couch_views/src/couch_views_util.erl b/src/couch_views/src/couch_views_util.erl
new file mode 100644
index 0000000..d7ed29f
--- /dev/null
+++ b/src/couch_views/src/couch_views_util.erl
@@ -0,0 +1,83 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_util).
+
+
+-export([
+    ddoc_to_mrst/2
+]).
+
+
+-include_lib("couch/include/couch_db.hrl").
+-include("couch_views.hrl").
+
+
+ddoc_to_mrst(DbName, #doc{id=Id, body={Fields}}) ->
+    MakeDict = fun({Name, {MRFuns}}, DictBySrcAcc) ->
+        case couch_util:get_value(<<"map">>, MRFuns) of
+            MapSrc when MapSrc /= undefined ->
+                RedSrc = couch_util:get_value(<<"reduce">>, MRFuns, null),
+                {ViewOpts} = couch_util:get_value(<<"options">>, MRFuns, {[]}),
+                View = case dict:find({MapSrc, ViewOpts}, DictBySrcAcc) of
+                    {ok, View0} -> View0;
+                    error -> #mrview{def=MapSrc, options=ViewOpts}
+                end,
+                {MapNames, RedSrcs} = case RedSrc of
+                    null ->
+                        MNames = [Name | View#mrview.map_names],
+                        {MNames, View#mrview.reduce_funs};
+                    _ ->
+                        RedFuns = [{Name, RedSrc} | View#mrview.reduce_funs],
+                        {View#mrview.map_names, RedFuns}
+                end,
+                View2 = View#mrview{map_names=MapNames, reduce_funs=RedSrcs},
+                dict:store({MapSrc, ViewOpts}, View2, DictBySrcAcc);
+            undefined ->
+                DictBySrcAcc
+        end;
+        ({Name, Else}, DictBySrcAcc) ->
+            couch_log:error("design_doc_to_view_group ~s views ~p",
+                [Name, Else]),
+            DictBySrcAcc
+    end,
+    {DesignOpts} = proplists:get_value(<<"options">>, Fields, {[]}),
+    SeqIndexed = proplists:get_value(<<"seq_indexed">>, DesignOpts, false),
+    KeySeqIndexed = proplists:get_value(<<"keyseq_indexed">>,
+        DesignOpts, false),
+    Partitioned = proplists:get_value(<<"partitioned">>, DesignOpts, false),
+
+    {RawViews} = couch_util:get_value(<<"views">>, Fields, {[]}),
+    BySrc = lists:foldl(MakeDict, dict:new(), RawViews),
+
+    NumViews = fun({_, View}, N) ->
+            {View#mrview{id_num=N, seq_indexed=SeqIndexed,
+                keyseq_indexed=KeySeqIndexed}, N+1}
+    end,
+    {Views, _} = lists:mapfoldl(NumViews, 0, lists:sort(dict:to_list(BySrc))),
+
+    Language = couch_util:get_value(<<"language">>, Fields, <<"javascript">>),
+    Lib = couch_util:get_value(<<"lib">>, RawViews, {[]}),
+
+    IdxState = #mrst{
+        db_name=DbName,
+        idx_name=Id,
+        lib=Lib,
+        views=Views,
+        language=Language,
+        design_opts=DesignOpts,
+        seq_indexed=SeqIndexed,
+        keyseq_indexed=KeySeqIndexed,
+        partitioned=Partitioned
+    },
+    SigInfo = {Views, Language, DesignOpts, couch_index_util:sort_lib(Lib)},
+    {ok, IdxState#mrst{sig=couch_hash:md5_hash(term_to_binary(SigInfo))}}.
diff --git a/src/couch_views/src/couch_views_worker.erl b/src/couch_views/src/couch_views_worker.erl
new file mode 100644
index 0000000..fa641d5
--- /dev/null
+++ b/src/couch_views/src/couch_views_worker.erl
@@ -0,0 +1,44 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_worker).
+
+-export([
+    start/2,
+    job_progress/6
+]).
+
+
+start(Job, JobData) ->
+    {ok, Db, Mrst} = get_indexing_info(JobData),
+    % maybe we should spawn here
+    couch_views_indexer:update(Db, Mrst, fun job_progress/6, Job).
+
+
+job_progress(Tx, Progress, Job, Db, Mrst, LastSeq) ->
+    case Progress of
+        update ->
+            couch_views_jobs:update(Tx, Job, Db, Mrst, LastSeq);
+        finished ->
+            couch_views_jobs:finish(Tx, Job, Db, Mrst, LastSeq)
+    end.
+
+
+get_indexing_info(JobData) ->
+    #{
+        <<"db_name">> := DbName,
+        <<"ddoc_id">> := DDocId
+    } = JobData,
+    {ok, Db} = fabric2_db:open(DbName, []),
+    {ok, DDoc} = fabric2_db:open_doc(Db, DDocId),
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+    {ok, Db, Mrst}.
diff --git a/src/couch_views/src/couch_views_worker_server.erl b/src/couch_views/src/couch_views_worker_server.erl
new file mode 100644
index 0000000..1c815e5
--- /dev/null
+++ b/src/couch_views/src/couch_views_worker_server.erl
@@ -0,0 +1,110 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_worker_server).
+
+
+-behaviour(gen_server).
+
+
+-export([
+    start_link/0
+]).
+
+
+-export([
+    init/1,
+    terminate/2,
+    handle_call/3,
+    handle_cast/2,
+    handle_info/2,
+    code_change/3
+]).
+
+
+-define(TYPE_CHECK_PERIOD_DEFAULT, 500).
+-define(MAX_JITTER_DEFAULT, 100).
+
+
+start_link() ->
+    gen_server:start_link({local, ?MODULE}, ?MODULE, [], []).
+
+
+init(_) ->
+    couch_views_jobs:set_timeout(),
+    schedule_check(),
+    {ok, #{}}.
+
+
+terminate(_, _St) ->
+    ok.
+
+
+handle_call(Msg, _From, St) ->
+    {stop, {bad_call, Msg}, {bad_call, Msg}, St}.
+
+
+handle_cast(Msg, St) ->
+    {stop, {bad_cast, Msg}, St}.
+
+
+handle_info(check_for_jobs, State) ->
+    accept_jobs(),
+    schedule_check(),
+    {noreply, State};
+
+handle_info({'DOWN', _Ref, process, Pid, Reason}, St) ->
+    LogMsg = "~p : process ~p exited with ~p",
+    couch_log:error(LogMsg, [?MODULE, Pid, Reason]),
+    {noreply, St};
+
+handle_info(Msg, St) ->
+    couch_log:notice("~s ignoring info ~w", [?MODULE, Msg]),
+    {noreply, St}.
+
+
+code_change(_OldVsn, St, _Extra) ->
+    {ok, St}.
+
+
+accept_jobs() ->
+    case couch_views_jobs:accept() of
+        not_found ->
+            ok;
+        {ok, Job, JobData} ->
+            start_worker(Job, JobData),
+            % keep accepting jobs until not_found
+            accept_jobs()
+    end.
+
+
+start_worker(Job, JobData) ->
+    % TODO Should I monitor it, or let jobs do that?
+    spawn_monitor(fun () -> couch_views_worker:start(Job, JobData) end),
+    ok.
+
+
+schedule_check() ->
+    Timeout = get_period_msec(),
+    MaxJitter = max(Timeout div 2, get_max_jitter_msec()),
+    Wait = Timeout + rand:uniform(max(1, MaxJitter)),
+    timer:send_after(Wait, self(), check_for_jobs).
+
+
+get_period_msec() ->
+    config:get_integer("couch_views", "type_check_period_msec",
+        ?TYPE_CHECK_PERIOD_DEFAULT).
+
+
+get_max_jitter_msec() ->
+    config:get_integer("couch_views", "type_check_max_jitter_msec",
+        ?MAX_JITTER_DEFAULT).
diff --git a/src/couch_views/test/couch_views_encoding_test.erl b/src/couch_views/test/couch_views_encoding_test.erl
new file mode 100644
index 0000000..a73cb42
--- /dev/null
+++ b/src/couch_views/test/couch_views_encoding_test.erl
@@ -0,0 +1,73 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_encoding_test).
+
+-include_lib("eunit/include/eunit.hrl").
+
+val_encoding_test() ->
+    Values = [
+        null,
+        true,
+        1.0,
+        <<"a">>,
+        {[{<<"a">>, 1.0}, {<<"b">>, <<"hello">>}]}
+    ],
+    lists:foreach(fun (Val) ->
+        EncVal = couch_views_encoding:encode(Val),
+        ?assertEqual(Val, couch_views_encoding:decode(EncVal))
+    end, Values).
+
+
+correct_ordering_test() ->
+    Ordered = [
+        %  Special values sort before all other types
+        null,
+        false,
+        true,
+
+        %  Then numbers
+        % 1,
+        % 2,
+        % 3.0,
+        % 4,
+
+        1.0,
+        2.0,
+        3.0,
+        4.0,
+
+        [<<"a">>],
+        [<<"b">>],
+        [<<"b">>, <<"c">>],
+        [<<"b">>, <<"c">>, <<"a">>],
+        [<<"b">>, <<"d">>],
+        [<<"b">>, <<"d">>, <<"e">>],
+
+        % Then objects, compared each key value in the list until different.
+        % Larger objects sort after their subset objects
+        {[{<<"a">>, 1.0}]},
+        {[{<<"a">>, 2.0}]},
+        {[{<<"b">>, 1.0}]},
+        {[{<<"b">>, 2.0}]},
+
+        % Member order does matter for collation
+        {[{<<"b">>, 2.0}, {<<"a">>, 1.0}]},
+        {[{<<"b">>, 2.0}, {<<"c">>, 2.0}]}
+
+    ],
+
+    BinList = lists:map(fun couch_views_encoding:encode/1, Ordered),
+    SortedBinList = lists:sort(BinList),
+    DecodedBinList = lists:map(fun couch_views_encoding:decode/1,
+        SortedBinList),
+    ?assertEqual(Ordered, DecodedBinList).
diff --git a/src/couch_views/test/couch_views_indexer_test.erl b/src/couch_views/test/couch_views_indexer_test.erl
new file mode 100644
index 0000000..2d192a6
--- /dev/null
+++ b/src/couch_views/test/couch_views_indexer_test.erl
@@ -0,0 +1,258 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_indexer_test).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("eunit/include/eunit.hrl").
+-include_lib("couch_mrview/include/couch_mrview.hrl").
+
+
+-define(TDEF(A), {atom_to_list(A), fun A/0}).
+
+setup() ->
+    test_util:start_couch([fabric]).
+
+
+teardown(State) ->
+    test_util:stop_couch(State).
+
+
+foreach_setup() ->
+    ok.
+
+
+foreach_teardown(_) ->
+    meck:unload().
+
+
+index_server_test_() ->
+    {
+        "Test Couch Views indexer",
+        {
+            setup,
+            fun setup/0,
+            fun teardown/1,
+            {
+                foreach,
+                fun foreach_setup/0, fun foreach_teardown/1,
+                [
+                    ?TDEF(map_docs_no_results_for_deleted),
+                    ?TDEF(map_docs_returns_sorted_results),
+                    ?TDEF(write_doc_clears_for_deleted_doc),
+                    ?TDEF(write_doc_adds_for_new_doc),
+                    ?TDEF(write_doc_clears_and_sets_for_update),
+                    ?TDEF(write_doc_clears_for_no_new_update),
+                    ?TDEF(write_doc_clears_and_updates_duplicates)
+                ]
+            }
+
+        }
+    }.
+
+
+map_docs_no_results_for_deleted() ->
+    DbName = ?tempdb,
+
+    DDoc = create_ddoc(),
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+
+    Doc = #{
+        id => <<"id">>,
+        sequence => <<1111>>,
+        rev_id => <<"1-123">>,
+        deleted => true
+    },
+
+    meck:expect(couch_query_servers, start_doc_map, fun(_, _, _) ->
+        {ok, fake}
+    end),
+
+    {Results, _} = couch_views_indexer:map_docs(Mrst, [Doc]),
+
+    [#{results := DocResult}] = Results,
+    ?assertEqual([], DocResult).
+
+
+map_docs_returns_sorted_results() ->
+    DbName = ?tempdb,
+    Doc = #{
+        id => <<"id">>,
+        sequence => <<1111>>,
+        rev_id => <<"1-123">>,
+        doc => doc(1)
+    },
+
+    CompleteResult = [[{1, 1}], []],
+
+    DDoc = create_ddoc(),
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+
+
+    {Results, _} = couch_views_indexer:map_docs(Mrst, [Doc]),
+    [#{results := DocResult}] = Results,
+    ?assertEqual(CompleteResult, DocResult).
+
+
+write_doc_clears_for_deleted_doc() ->
+    TxDb = #{},
+    Sig = <<123>>,
+    Doc = #{deleted => true, id => 1},
+    ViewIds = [1],
+    OldIdxKey = old_key,
+
+    meck:expect(couch_views_fdb, get_id_index, 4, old_key),
+    meck:expect(couch_views_fdb, clear_id_index, 4, ok),
+    meck:expect(couch_views_fdb, clear_map_index, 5, ok),
+
+    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
+    ?assert(meck:called(couch_views_fdb, get_id_index, [TxDb, Sig, 1, 1])),
+    ?assert(meck:called(couch_views_fdb, clear_id_index, [TxDb, Sig, 1, 1])),
+    ?assert(meck:called(couch_views_fdb, clear_map_index,
+        [TxDb, Sig, 1, 1, OldIdxKey])),
+    ?assertEqual(length(meck:history(couch_views_fdb)), 3).
+
+
+write_doc_adds_for_new_doc() ->
+    TxDb = #{},
+    Sig = <<123>>,
+    Key = <<"key">>,
+    Value = 1,
+    Results = [{Key, Value}],
+    Doc = #{
+        deleted => false,
+        id => 1,
+        results => [Results]
+    },
+    ViewIds = [1],
+
+    meck:expect(couch_views_fdb, get_id_index, 4, not_found),
+    meck:expect(couch_views_fdb, set_id_index, 5, ok),
+    meck:expect(couch_views_fdb, set_map_index_results, 5, ok),
+
+    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
+    ?assert(meck:called(couch_views_fdb, get_id_index, [TxDb, Sig, 1, 1])),
+    ?assert(meck:called(couch_views_fdb, set_id_index,
+        [TxDb, Sig, 1, 1, Key])),
+    ?assert(meck:called(couch_views_fdb, set_map_index_results,
+        [TxDb, Sig, 1, 1, Results])),
+    ?assertEqual(length(meck:history(couch_views_fdb)), 3).
+
+
+write_doc_clears_and_sets_for_update() ->
+    TxDb = #{},
+    Sig = <<123>>,
+    Key = <<"key">>,
+    Value = 1,
+    Results = [{Key, Value}],
+    Doc = #{
+        deleted => false,
+        id => 1,
+        results => [Results]
+    },
+    ViewIds = [1],
+    OldKey = oldkey,
+
+    meck:expect(couch_views_fdb, get_id_index, 4, OldKey),
+    meck:expect(couch_views_fdb, clear_id_index, 4, ok),
+    meck:expect(couch_views_fdb, clear_map_index, 5, ok),
+    meck:expect(couch_views_fdb, set_id_index, 5, ok),
+    meck:expect(couch_views_fdb, set_map_index_results, 5, ok),
+
+    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
+    ?assert(meck:called(couch_views_fdb, get_id_index, [TxDb, Sig, 1, 1])),
+    ?assert(meck:called(couch_views_fdb, clear_id_index, [TxDb, Sig, 1, 1])),
+    ?assert(meck:called(couch_views_fdb, clear_map_index,
+        [TxDb, Sig, 1, 1, OldKey])),
+    ?assert(meck:called(couch_views_fdb, set_id_index,
+        [TxDb, Sig, 1, 1, Key])),
+    ?assert(meck:called(couch_views_fdb, set_map_index_results,
+        [TxDb, Sig, 1, 1, Results])),
+    ?assertEqual(length(meck:history(couch_views_fdb)), 5).
+
+
+write_doc_clears_for_no_new_update() ->
+    TxDb = #{},
+    Sig = <<123>>,
+    Results = [],
+    Doc = #{
+        deleted => false,
+        id => 1,
+        results => [Results]
+    },
+    ViewIds = [1],
+    OldKey = oldkey,
+
+    meck:expect(couch_views_fdb, get_id_index, 4, OldKey),
+    meck:expect(couch_views_fdb, clear_id_index, 4, ok),
+    meck:expect(couch_views_fdb, clear_map_index, 5, ok),
+
+    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
+    ?assert(meck:called(couch_views_fdb, get_id_index, [TxDb, Sig, 1, 1])),
+    ?assert(meck:called(couch_views_fdb, clear_id_index, [TxDb, Sig, 1, 1])),
+    ?assert(meck:called(couch_views_fdb, clear_map_index,
+        [TxDb, Sig, 1, 1, OldKey])),
+    ?assertEqual(length(meck:history(couch_views_fdb)), 3).
+
+
+write_doc_clears_and_updates_duplicates() ->
+    TxDb = #{},
+    Sig = <<123>>,
+    Key = <<"key">>,
+    Results = [{Key, 1}, {Key, 2}],
+    Doc = #{
+        deleted => false,
+        id => 1,
+        results => [Results]
+    },
+    ViewIds = [1],
+    OldKey = oldkey,
+
+    meck:expect(couch_views_fdb, get_id_index, 4, OldKey),
+    meck:expect(couch_views_fdb, clear_id_index, 4, ok),
+    meck:expect(couch_views_fdb, clear_map_index, 5, ok),
+    meck:expect(couch_views_fdb, set_id_index, 5, ok),
+    meck:expect(couch_views_fdb, set_map_index_results, 5, ok),
+
+    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
+    ?assertEqual(meck:num_calls(couch_views_fdb, get_id_index,
+        [TxDb, Sig, 1, 1]), 2),
+    ?assertEqual(meck:num_calls(couch_views_fdb, clear_id_index,
+        [TxDb, Sig, 1, 1]), 1),
+    ?assertEqual(meck:num_calls(couch_views_fdb, set_id_index,
+        [TxDb, Sig, 1, 1, Key]), 2),
+    ?assertEqual(meck:num_calls(couch_views_fdb, clear_map_index,
+        [TxDb, Sig, 1, 1, OldKey]), 1),
+    ?assertEqual(meck:num_calls(couch_views_fdb, set_map_index_results,
+        [TxDb, Sig, 1, 1, Results]), 2),
+    ?assertEqual(length(meck:history(couch_views_fdb)), 8).
+
+
+create_ddoc() ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, <<"_design/bar">>},
+        {<<"views">>, {[
+            {<<"map_fun1">>, {[
+                {<<"map">>, <<"function(doc) {emit(doc.val, doc.val);}">>}
+            ]}},
+            {<<"map_fun2">>, {[
+                {<<"map">>, <<"function(doc) {}">>}
+            ]}}
+        ]}}
+    ]}).
+
+
+doc(Id) ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, list_to_binary(integer_to_list(Id))},
+        {<<"val">>, Id}
+    ]}).
diff --git a/src/couch_views/test/couch_views_map_test.erl b/src/couch_views/test/couch_views_map_test.erl
new file mode 100644
index 0000000..bbad93f
--- /dev/null
+++ b/src/couch_views/test/couch_views_map_test.erl
@@ -0,0 +1,484 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(couch_views_map_test).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("couch/include/couch_db.hrl").
+
+
+-define(TDEF(A), {atom_to_list(A), fun A/0}).
+
+
+setup() ->
+    test_util:start_couch([fabric, couch_jobs, couch_views]).
+
+
+teardown(State) ->
+    test_util:stop_couch(State).
+
+
+map_views_test_() ->
+    {
+        "Map views",
+        {
+            setup,
+            fun setup/0,
+            fun teardown/1,
+            [
+                ?TDEF(should_map),
+                ?TDEF(should_map_with_startkey),
+                ?TDEF(should_map_with_endkey),
+                ?TDEF(should_map_with_endkey_not_inclusive),
+                ?TDEF(should_map_reverse_and_limit),
+                ?TDEF(should_map_with_range_reverse),
+                ?TDEF(should_map_with_limit_and_skip),
+                ?TDEF(should_map_with_limit_and_skip_reverse),
+                ?TDEF(should_map_with_include_docs),
+                ?TDEF(should_map_with_include_docs_reverse),
+                ?TDEF(should_map_with_startkey_with_key_array),
+                ?TDEF(should_map_with_startkey_and_endkey_with_key_array),
+                ?TDEF(should_map_empty_views),
+                ?TDEF(should_map_duplicate_keys),
+                ?TDEF(should_map_with_doc_emit),
+                ?TDEF(should_map_update_is_false),
+                ?TDEF(should_map_update_is_lazy)
+                % fun should_give_ext_size_seq_indexed_test/1
+            ]
+        }
+    }.
+
+
+should_map() ->
+    Result = run_query(<<"baz">>, #{}),
+    Expect = {ok, [
+        {row, [{id, <<"1">>}, {key, 1}, {value, 1}]},
+        {row, [{id, <<"2">>}, {key, 2}, {value, 2}]},
+        {row, [{id, <<"3">>}, {key, 3}, {value, 3}]},
+        {row, [{id, <<"4">>}, {key, 4}, {value, 4}]},
+        {row, [{id, <<"5">>}, {key, 5}, {value, 5}]},
+        {row, [{id, <<"6">>}, {key, 6}, {value, 6}]},
+        {row, [{id, <<"7">>}, {key, 7}, {value, 7}]},
+        {row, [{id, <<"8">>}, {key, 8}, {value, 8}]},
+        {row, [{id, <<"9">>}, {key, 9}, {value, 9}]},
+        {row, [{id, <<"10">>}, {key, 10}, {value, 10}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_startkey() ->
+    Result = run_query(<<"baz">>, #{start_key => 4}),
+    Expect = {ok, [
+        {row, [{id, <<"4">>}, {key, 4}, {value, 4}]},
+        {row, [{id, <<"5">>}, {key, 5}, {value, 5}]},
+        {row, [{id, <<"6">>}, {key, 6}, {value, 6}]},
+        {row, [{id, <<"7">>}, {key, 7}, {value, 7}]},
+        {row, [{id, <<"8">>}, {key, 8}, {value, 8}]},
+        {row, [{id, <<"9">>}, {key, 9}, {value, 9}]},
+        {row, [{id, <<"10">>}, {key, 10}, {value, 10}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_endkey() ->
+    Result = run_query(<<"baz">>, #{end_key => 5}),
+    Expect = {ok, [
+        {row, [{id, <<"1">>}, {key, 1}, {value, 1}]},
+        {row, [{id, <<"2">>}, {key, 2}, {value, 2}]},
+        {row, [{id, <<"3">>}, {key, 3}, {value, 3}]},
+        {row, [{id, <<"4">>}, {key, 4}, {value, 4}]},
+        {row, [{id, <<"5">>}, {key, 5}, {value, 5}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_endkey_not_inclusive() ->
+    Result = run_query(<<"baz">>, #{
+        end_key => 5,
+        inclusive_end => false
+    }),
+    Expect = {ok, [
+        {row, [{id, <<"1">>}, {key, 1}, {value, 1}]},
+        {row, [{id, <<"2">>}, {key, 2}, {value, 2}]},
+        {row, [{id, <<"3">>}, {key, 3}, {value, 3}]},
+        {row, [{id, <<"4">>}, {key, 4}, {value, 4}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_reverse_and_limit() ->
+    Result = run_query(<<"baz">>, #{
+        direction => rev,
+        limit => 3
+    }),
+    Expect = {ok, [
+        {row, [{id, <<"10">>}, {key, 10}, {value, 10}]},
+        {row, [{id, <<"9">>}, {key, 9}, {value, 9}]},
+        {row, [{id, <<"8">>}, {key, 8}, {value, 8}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_range_reverse() ->
+    Result = run_query(<<"baz">>, #{
+        direction => rev,
+        start_key => 5,
+        end_key => 3,
+        inclusive_end => true
+    }),
+    Expect = {ok, [
+        {row, [{id, <<"5">>}, {key, 5}, {value, 5}]},
+        {row, [{id, <<"4">>}, {key, 4}, {value, 4}]},
+        {row, [{id, <<"3">>}, {key, 3}, {value, 3}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_limit_and_skip() ->
+    Result = run_query(<<"baz">>, #{
+        start_key => 2,
+        limit => 3,
+        skip => 3
+    }),
+    Expect = {ok, [
+        {row, [{id, <<"5">>}, {key, 5}, {value, 5}]},
+        {row, [{id, <<"6">>}, {key, 6}, {value, 6}]},
+        {row, [{id, <<"7">>}, {key, 7}, {value, 7}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_limit_and_skip_reverse() ->
+    Result = run_query(<<"baz">>, #{
+        start_key => 10,
+        limit => 3,
+        skip => 3,
+        direction => rev
+    }),
+    Expect = {ok, [
+        {row, [{id, <<"7">>}, {key, 7}, {value, 7}]},
+        {row, [{id, <<"6">>}, {key, 6}, {value, 6}]},
+        {row, [{id, <<"5">>}, {key, 5}, {value, 5}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_include_docs() ->
+    Result = run_query(<<"baz">>, #{
+        start_key => 8,
+        end_key => 8,
+        include_docs => true
+    }),
+    Doc = {[
+        {<<"_id">>, <<"8">>},
+        {<<"_rev">>, <<"1-55b9a29311341e07ec0a7ca13bc1b59f">>},
+        {<<"val">>, 8}
+    ]},
+    Expect = {ok, [
+        {row, [{id, <<"8">>}, {key, 8}, {value, 8}, {doc, Doc}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_include_docs_reverse() ->
+    Result = run_query(<<"baz">>, #{
+        start_key => 8,
+        end_key => 8,
+        include_docs => true,
+        direction => rev
+    }),
+    Doc = {[
+        {<<"_id">>, <<"8">>},
+        {<<"_rev">>, <<"1-55b9a29311341e07ec0a7ca13bc1b59f">>},
+        {<<"val">>, 8}
+    ]},
+    Expect = {ok, [
+        {row, [{id, <<"8">>}, {key, 8}, {value, 8}, {doc, Doc}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_startkey_with_key_array() ->
+    Rows = [
+        {row, [{id, <<"4">>}, {key, [<<"4">>, 4]}, {value, 4}]},
+        {row, [{id, <<"5">>}, {key, [<<"5">>, 5]}, {value, 5}]},
+        {row, [{id, <<"6">>}, {key, [<<"6">>, 6]}, {value, 6}]},
+        {row, [{id, <<"7">>}, {key, [<<"7">>, 7]}, {value, 7}]},
+        {row, [{id, <<"8">>}, {key, [<<"8">>, 8]}, {value, 8}]},
+        {row, [{id, <<"9">>}, {key, [<<"9">>, 9]}, {value, 9}]}
+    ],
+
+    Result = run_query(<<"boom">>, #{
+        start_key => [<<"4">>]
+    }),
+
+    ?assertEqual({ok, Rows}, Result),
+
+    ResultRev = run_query(<<"boom">>, #{
+        start_key => [<<"9">>, 9],
+        direction => rev,
+        limit => 6
+    }),
+
+    ?assertEqual({ok, lists:reverse(Rows)}, ResultRev).
+
+
+should_map_with_startkey_and_endkey_with_key_array() ->
+    Rows = [
+        {row, [{id, <<"4">>}, {key, [<<"4">>, 4]}, {value, 4}]},
+        {row, [{id, <<"5">>}, {key, [<<"5">>, 5]}, {value, 5}]},
+        {row, [{id, <<"6">>}, {key, [<<"6">>, 6]}, {value, 6}]},
+        {row, [{id, <<"7">>}, {key, [<<"7">>, 7]}, {value, 7}]},
+        {row, [{id, <<"8">>}, {key, [<<"8">>, 8]}, {value, 8}]}
+    ],
+
+    Result = run_query(<<"boom">>, #{
+        start_key => [<<"4">>],
+        end_key => [<<"8">>, []]
+    }),
+
+    ?assertEqual({ok, Rows}, Result),
+
+    ResultRev = run_query(<<"boom">>, #{
+        start_key => [<<"8">>, []],
+        end_key => [<<"4">>],
+        direction => rev
+    }),
+
+    ?assertEqual({ok, lists:reverse(Rows)}, ResultRev),
+
+    ResultRev2 = run_query(<<"boom">>, #{
+        start_key => [<<"9">>, 9],
+        end_key => [<<"4">>],
+        direction => rev,
+        inclusive_end => false
+    }),
+
+    ?assertEqual({ok, lists:reverse(Rows)}, ResultRev2).
+
+
+should_map_empty_views() ->
+    Result = run_query(<<"bing">>, #{}),
+    Expect = {ok, []},
+    ?assertEqual(Expect, Result).
+
+
+should_map_with_doc_emit() ->
+    Result = run_query(<<"doc_emit">>, #{
+        start_key => 8,
+        limit => 1
+    }),
+    Doc = {[
+        {<<"_id">>, <<"8">>},
+        {<<"_rev">>, <<"1-55b9a29311341e07ec0a7ca13bc1b59f">>},
+        {<<"val">>, 8}
+    ]},
+    Expect = {ok, [
+        {row, [{id, <<"8">>}, {key, 8}, {value, Doc}]}
+    ]},
+    ?assertEqual(Expect, Result).
+
+
+should_map_duplicate_keys() ->
+    Result = run_query(<<"duplicate_keys">>, #{
+        limit => 6
+    }),
+    Expect = {ok, [
+        {row, [{id, <<"1">>}, {key, <<"1">>}, {value, 1}]},
+        {row, [{id, <<"1">>}, {key, <<"1">>}, {value, 2}]},
+        {row, [{id, <<"10">>}, {key, <<"10">>}, {value, 10}]},
+        {row, [{id, <<"10">>}, {key, <<"10">>}, {value, 11}]},
+        {row, [{id, <<"2">>}, {key, <<"2">>}, {value, 2}]},
+        {row, [{id, <<"2">>}, {key, <<"2">>}, {value, 3}]}
+    ]},
+    ?debugFmt("EXPE ~p ~n", [Expect]),
+    ?assertEqual(Expect, Result).
+
+
+should_map_update_is_false() ->
+    Expect = {ok, [
+        {row, [{id, <<"8">>},  {key, 8},  {value, 8}]},
+        {row, [{id, <<"9">>},  {key, 9},  {value, 9}]},
+        {row, [{id, <<"10">>}, {key, 10}, {value, 10}]}
+    ]},
+
+    Expect1 = {ok, [
+        {row, [{id, <<"8">>},  {key, 8},  {value, 8}]},
+        {row, [{id, <<"9">>},  {key, 9},  {value, 9}]},
+        {row, [{id, <<"10">>}, {key, 10}, {value, 10}]},
+        {row, [{id, <<"11">>}, {key, 11}, {value, 11}]}
+    ]},
+
+    Idx = <<"baz">>,
+    DbName = ?tempdb(),
+
+    {ok, Db} = fabric2_db:create(DbName, [{user_ctx, ?ADMIN_USER}]),
+
+    DDoc = create_ddoc(),
+    Docs = make_docs(10),
+    fabric2_db:update_docs(Db, [DDoc | Docs]),
+
+    Args1 = #{
+        start_key => 8
+    },
+
+    Result1 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+        [], Args1),
+    ?assertEqual(Expect, Result1),
+
+    Doc = doc(11),
+    fabric2_db:update_doc(Db, Doc),
+
+    Args2 = #{
+        start_key => 8,
+        update => false
+    },
+
+    Result2 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+        [], Args2),
+    ?assertEqual(Expect, Result2),
+
+    Result3 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+        [], Args1),
+    ?assertEqual(Expect1, Result3).
+
+
+should_map_update_is_lazy() ->
+    Expect = {ok, [
+        {row, [{id, <<"8">>},  {key, 8},  {value, 8}]},
+        {row, [{id, <<"9">>},  {key, 9},  {value, 9}]},
+        {row, [{id, <<"10">>}, {key, 10}, {value, 10}]}
+    ]},
+
+    Idx = <<"baz">>,
+    DbName = ?tempdb(),
+
+    {ok, Db} = fabric2_db:create(DbName, [{user_ctx, ?ADMIN_USER}]),
+
+    DDoc = create_ddoc(),
+    Docs = make_docs(10),
+
+    fabric2_db:update_docs(Db, [DDoc | Docs]),
+
+    Args1 = #{
+        start_key => 8,
+        update => lazy
+    },
+
+    Result1 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+        [], Args1),
+    ?assertEqual({ok, []}, Result1),
+
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+    {ok, Subscription, _, _} = couch_views_jobs:subscribe(Db, Mrst),
+    couch_jobs:wait(Subscription, finished, 1000),
+
+    Args2 = #{
+        start_key => 8,
+        update => false
+    },
+
+    Result2 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+        [], Args2),
+    ?assertEqual(Expect, Result2).
+
+
+% should_give_ext_size_seq_indexed_test(Db) ->
+%     DDoc = couch_doc:from_json_obj({[
+%         {<<"_id">>, <<"_design/seqdoc">>},
+%         {<<"options">>, {[{<<"seq_indexed">>, true}]}},
+%         {<<"views">>, {[
+%                 {<<"view1">>, {[
+%                     {<<"map">>, <<"function(doc){emit(doc._id, doc._id);}">>}
+%                 ]}}
+%             ]}
+%         }
+%     ]}),
+%     {ok, _} = couch_db:update_doc(Db, DDoc, []),
+%     {ok, Db1} = couch_db:open_int(couch_db:name(Db), []),
+%     {ok, DDoc1} = couch_db:open_doc(Db1, <<"_design/seqdoc">>, [ejson_body]),
+%     couch_mrview:query_view(Db1, DDoc1, <<"view1">>, [{update, true}]),
+%     {ok, Info} = couch_mrview:get_info(Db1, DDoc),
+%     Size = couch_util:get_nested_json_value({Info}, [sizes, external]),
+%     ok = couch_db:close(Db1),
+%     ?assert(is_number(Size)).
+
+
+run_query(Idx, Args) ->
+    DbName = ?tempdb(),
+    {ok, Db} = fabric2_db:create(DbName, [{user_ctx, ?ADMIN_USER}]),
+    DDoc = create_ddoc(),
+    Docs = make_docs(10),
+    fabric2_db:update_docs(Db, [DDoc | Docs]),
+    couch_views:map_query(Db, DDoc, Idx, fun default_cb/2, [], Args).
+
+
+default_cb(complete, Acc) ->
+    {ok, lists:reverse(Acc)};
+default_cb({final, Info}, []) ->
+    {ok, [Info]};
+default_cb({final, _}, Acc) ->
+    {ok, Acc};
+default_cb(ok, ddoc_updated) ->
+    {ok, ddoc_updated};
+default_cb(Row, Acc) ->
+    {ok, [Row | Acc]}.
+
+
+create_ddoc() ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, <<"_design/bar">>},
+        {<<"views">>, {[
+            {<<"baz">>, {[
+                {<<"map">>, <<"function(doc) {emit(doc.val, doc.val);}">>}
+            ]}},
+            {<<"boom">>, {[
+                {<<"map">>, <<
+                    "function(doc) {\n"
+                    "   emit([doc.val.toString(), doc.val], doc.val);\n"
+                    "}"
+                >>}
+            ]}},
+            {<<"bing">>, {[
+                {<<"map">>, <<"function(doc) {}">>}
+            ]}},
+            {<<"doc_emit">>, {[
+                {<<"map">>, <<"function(doc) {emit(doc.val, doc)}">>}
+            ]}},
+            {<<"duplicate_keys">>, {[
+                {<<"map">>, <<
+                    "function(doc) {\n"
+                    "   emit(doc._id, doc.val);\n"
+                    "   emit(doc._id, doc.val + 1);\n"
+                    "}">>}
+            ]}},
+            {<<"zing">>, {[
+                {<<"map">>, <<
+                    "function(doc) {\n"
+                    "  if(doc.foo !== undefined)\n"
+                    "    emit(doc.foo, 0);\n"
+                    "}"
+                >>}
+            ]}}
+        ]}}
+    ]}).
+
+
+make_docs(Count) ->
+    [doc(I) || I <- lists:seq(1, Count)].
+
+
+doc(Id) ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, list_to_binary(integer_to_list(Id))},
+        {<<"val">>, Id}
+    ]}).
diff --git a/src/fabric/src/fabric2.hrl b/src/fabric/src/fabric2.hrl
index de1d3d1..6392d12 100644
--- a/src/fabric/src/fabric2.hrl
+++ b/src/fabric/src/fabric2.hrl
@@ -46,6 +46,7 @@
 -define(DB_DOCS, 21).
 -define(DB_LOCAL_DOCS, 22).
 -define(DB_ATTS, 23).
+-define(DB_VIEWS, 24).
 
 
 % Versions
diff --git a/src/fabric/src/fabric2_view.erl b/src/fabric/src/fabric2_view.erl
new file mode 100644
index 0000000..01c9ab0
--- /dev/null
+++ b/src/fabric/src/fabric2_view.erl
@@ -0,0 +1,81 @@
+% Licensed under the Apache License, Version 2.0 (the "License"); you may not
+% use this file except in compliance with the License. You may obtain a copy of
+% the License at
+%
+%   http://www.apache.org/licenses/LICENSE-2.0
+%
+% Unless required by applicable law or agreed to in writing, software
+% distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+% WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+% License for the specific language governing permissions and limitations under
+% the License.
+
+-module(fabric2_view).
+
+-export([
+    query/7
+]).
+
+-include_lib("couch_mrview/include/couch_mrview.hrl").
+
+%% @doc execute a given view.
+%%      There are many additional query args that can be passed to a view,
+%%      see <a href="http://wiki.apache.org/couchdb/HTTP_view_API#Querying_Options">
+%%      query args</a> for details.
+% -spec query(db(), [{atom(), any()}] | [],
+%         #doc{} | binary(), iodata(), callback(), any(), #mrargs{}) -> any().
+query(Db, Options, DDoc, ViewName, Callback, Acc0, QueryArgs0) ->
+    DbName = fabric2_db:name(Db),
+%%    View = name(ViewName),
+    case fabric_util:is_users_db(DbName) of
+        true ->
+            FakeDb = fabric_util:open_cluster_db(DbName, Options),
+            couch_users_db:after_doc_read(DDoc, FakeDb);
+        false ->
+            ok
+    end,
+%%    {ok, #mrst{views=Views, language=Lang}} =
+%%        couch_views_util:ddoc_to_mrst(DbName, DDoc),
+%%    QueryArgs1 = couch_mrview_util:set_view_type(QueryArgs0, View, Views),
+%%    QueryArgs1 = fabric_util:validate_args(Db, DDoc, QueryArgs0),
+    QueryArgs1 = couch_mrview_util:validate_args(Db, DDoc, QueryArgs0),
+%%    VInfo = couch_mrview_util:extract_view(Lang, QueryArgs1, View, Views),
+    case is_reduce_view(QueryArgs1) of
+        true ->
+            throw({not_implemented});
+        false ->
+            MapQueryArgs = mrargs_to_map((QueryArgs1)),
+            couch_views:map_query(Db, DDoc, ViewName, Callback,
+                Acc0, MapQueryArgs)
+    end.
+
+
+is_reduce_view(_) ->
+    false.
+
+
+name(Thing) ->
+    couch_util:to_binary(Thing).
+
+
+mrargs_to_map(#mrargs{} = Args) ->
+    #{
+        start_key => Args#mrargs.start_key,
+        start_key_docid => Args#mrargs.start_key_docid,
+        end_key => Args#mrargs.end_key,
+        end_key_docid => Args#mrargs.end_key_docid,
+        keys => Args#mrargs.keys,
+        direction => Args#mrargs.direction,
+        limit => Args#mrargs.limit,
+        skip => Args#mrargs.skip,
+        update => Args#mrargs.update,
+        multi_get => Args#mrargs.multi_get,
+        inclusive_end => Args#mrargs.inclusive_end,
+        include_docs => Args#mrargs.include_docs,
+        doc_options => Args#mrargs.doc_options,
+        update_seq => Args#mrargs.update_seq,
+        conflicts => Args#mrargs.conflicts,
+        sorted => Args#mrargs.sorted
+    }.
+
+
diff --git a/test/elixir/test/map_test.exs b/test/elixir/test/map_test.exs
new file mode 100644
index 0000000..b7a809d
--- /dev/null
+++ b/test/elixir/test/map_test.exs
@@ -0,0 +1,222 @@
+defmodule ViewMapTest do
+  use CouchTestCase
+
+  @moduledoc """
+  Test Map functionality for views
+  """
+  def get_ids(resp) do
+    %{:body => %{"rows" => rows}} = resp
+    Enum.map(rows, fn row -> row["id"] end)
+  end
+
+  defp create_map_docs(db_name) do
+    docs =
+      for i <- 1..10 do
+        group =
+          if rem(i, 3) == 0 do
+            "one"
+          else
+            "two"
+          end
+
+        doc = %{
+          :_id => "doc-id-#{i}",
+          :value => i,
+          :some => "field",
+          :group => group
+        }
+      end
+
+    resp = Couch.post("/#{db_name}/_bulk_docs", body: %{:docs => docs})
+    assert resp.status_code == 201
+  end
+
+  setup do
+    db_name = random_db_name()
+    {:ok, _} = create_db(db_name)
+    on_exit(fn -> delete_db(db_name) end)
+
+    create_map_docs(db_name)
+
+    map_fun1 = """
+      function(doc) {
+        if (doc.some) {
+            emit(doc.value , doc.value);
+        }
+
+        if (doc._id.indexOf("_design") > -1) {
+            emit(0, "ddoc")
+        }
+      }
+    """
+
+    map_fun2 = """
+      function(doc) {
+        if (doc.group) {
+          emit([doc.some, doc.group], 1);
+        }
+      }
+    """
+
+    body = %{
+      :docs => [
+        %{
+          _id: "_design/map",
+          views: %{
+            some: %{map: map_fun1},
+            map_some: %{map: map_fun2}
+          }
+        },
+        %{
+          _id: "_design/include_ddocs",
+          views: %{some: %{map: map_fun1}},
+          options: %{include_design: true}
+        }
+      ]
+    }
+
+    resp = Couch.post("/#{db_name}/_bulk_docs", body: body)
+    Enum.each(resp.body, &assert(&1["ok"]))
+
+    #        ddoc = %{
+    #            :_id => "_design/map",
+    #            views: %{
+    #                some: %{map: map_fun1},
+    #                map_some: %{map: map_fun2}
+    #            }
+    #        }
+    #        resp = Couch.put("/#{db_name}/#{ddoc._id}", body: ddoc)
+    #        IO.inspect resp
+    #        assert resp.status_code == 201
+
+    {:ok, [db_name: db_name]}
+  end
+
+  def get_reduce_result(resp) do
+    %{:body => %{"rows" => rows}} = resp
+    rows
+  end
+
+  test "query returns docs", context do
+    db_name = context[:db_name]
+
+    url = "/#{db_name}/_design/map/_view/some"
+    resp = Couch.get(url)
+    assert resp.status_code == 200
+
+    ids = get_ids(resp)
+
+    assert ids == [
+             "doc-id-1",
+             "doc-id-2",
+             "doc-id-3",
+             "doc-id-4",
+             "doc-id-5",
+             "doc-id-6",
+             "doc-id-7",
+             "doc-id-8",
+             "doc-id-9",
+             "doc-id-10"
+           ]
+
+    url = "/#{db_name}/_design/map/_view/map_some"
+    resp = Couch.get(url)
+    assert resp.status_code == 200
+
+    ids = get_ids(resp)
+
+    assert ids == [
+             "doc-id-3",
+             "doc-id-6",
+             "doc-id-9",
+             "doc-id-1",
+             "doc-id-10",
+             "doc-id-2",
+             "doc-id-4",
+             "doc-id-5",
+             "doc-id-7",
+             "doc-id-8"
+           ]
+  end
+
+  test "updated docs rebuilds index", context do
+    db_name = context[:db_name]
+
+    url = "/#{db_name}/_design/map/_view/some"
+    resp = Couch.get(url)
+    assert resp.status_code == 200
+    ids = get_ids(resp)
+
+    assert ids == [
+             "doc-id-1",
+             "doc-id-2",
+             "doc-id-3",
+             "doc-id-4",
+             "doc-id-5",
+             "doc-id-6",
+             "doc-id-7",
+             "doc-id-8",
+             "doc-id-9",
+             "doc-id-10"
+           ]
+
+    update_doc_value(db_name, "doc-id-5", 0)
+    update_doc_value(db_name, "doc-id-6", 100)
+
+    resp = Couch.get("/#{db_name}/doc-id-3")
+    doc3 = convert(resp.body)
+    resp = Couch.delete("/#{db_name}/#{doc3["_id"]}", query: %{rev: doc3["_rev"]})
+    assert resp.status_code == 200
+    #
+    resp = Couch.get("/#{db_name}/doc-id-4")
+    doc4 = convert(resp.body)
+    doc4 = Map.delete(doc4, "some")
+    resp = Couch.put("/#{db_name}/#{doc4["_id"]}", body: doc4)
+    assert resp.status_code == 201
+    #
+    resp = Couch.get("/#{db_name}/doc-id-1")
+    doc1 = convert(resp.body)
+    doc1 = Map.put(doc1, "another", "value")
+    resp = Couch.put("/#{db_name}/#{doc1["_id"]}", body: doc1)
+    assert resp.status_code == 201
+
+    url = "/#{db_name}/_design/map/_view/some"
+    resp = Couch.get(url)
+    assert resp.status_code == 200
+    ids = get_ids(resp)
+
+    assert ids == [
+             "doc-id-5",
+             "doc-id-1",
+             "doc-id-2",
+             "doc-id-7",
+             "doc-id-8",
+             "doc-id-9",
+             "doc-id-10",
+             "doc-id-6"
+           ]
+  end
+
+  test "can index design docs", context do
+    db_name = context[:db_name]
+
+    url = "/#{db_name}/_design/include_ddocs/_view/some"
+    resp = Couch.get(url, query: %{limit: 3})
+    assert resp.status_code == 200
+    ids = get_ids(resp)
+
+    assert ids == ["_design/include_ddocs", "_design/map", "doc-id-1"]
+  end
+
+  def update_doc_value(db_name, id, value) do
+    resp = Couch.get("/#{db_name}/#{id}")
+    doc = convert(resp.body)
+    doc = Map.put(doc, "value", value)
+    resp = Couch.put("/#{db_name}/#{id}", body: doc)
+    assert resp.status_code == 201
+  end
+
+  def convert(value) do
+    :jiffy.decode(:jiffy.encode(value), [:return_maps])
+  end
+end


[couchdb] 07/25: Move jobs logic to couch_view_jobs

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

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

commit b16b5b25304281aa9529d663c860327ff4e168f2
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Wed Jul 17 10:23:59 2019 -0500

    Move jobs logic to couch_view_jobs
    
    Now that the couch_jobs API is full baked we can remove the thin wrapper
    API in couch_views_jobs and just use couch_jobs directly.
---
 src/couch_views/src/couch_views.erl      |  80 ++++---------------
 src/couch_views/src/couch_views_jobs.erl | 131 +++++++++++++------------------
 2 files changed, 68 insertions(+), 143 deletions(-)

diff --git a/src/couch_views/src/couch_views.erl b/src/couch_views/src/couch_views.erl
index c059204..65af1bf 100644
--- a/src/couch_views/src/couch_views.erl
+++ b/src/couch_views/src/couch_views.erl
@@ -44,86 +44,36 @@ query(Db, DDoc, ViewName, Callback, Acc0, QueryArgs0) ->
     end,
 
     Args = mrargs_to_map(QueryArgs2),
-
-    maybe_build_view(Db, MrSt, Args),
+    ok = maybe_update_view(Db, Mrst, Args),
 
     try
         couch_views_reader:read(Db, Mrst, ViewName, Callback, Acc0, Args)
     after
         UpdateAfter = maps:get(update, Args) == lazy,
         if UpdateAfter == false -> ok; true ->
-            maybe_add_couch_job(Db, Mrst)
+            couch_views_jobs:build_view_async(Db, Mrst)
         end
     end.
 
 
-maybe_build_index(_Db, _Mrst, #{update := false}) ->
-    false;
+maybe_update_view(_Db, _Mrst, #{update := false}) ->
+    ok;
 
-maybe_build_index(_Db, _Mrst, #{update := lazy}) ->
-    false;
+maybe_update_view(_Db, _Mrst, #{update := lazy}) ->
+    ok;
 
-maybe_build_index(Db, Mrst, _Args) ->
-    {Status, Seq} = fabric2_fdb:transactional(Db, fun(TxDb) ->
-        case view_up_to_date(TxDb, Mrst) of
-            {true, UpdateSeq} ->
-                {ready, UpdateSeq};
-            {false, LatestSeq} ->
-                maybe_add_couch_job(TxDb, Mrst),
-                {false, LatestSeq}
+maybe_update_view(Db, Mrst, _Args) ->
+    WaitSeq = fabric2_fdb:transactional(Db, fun(TxDb) ->
+        DbSeq = fabric2_db:get_update_seq(TxDb),
+        ViewSeq = couch_views_fdb:get_update_seq(TxDb, Mrst),
+        case DbSeq == ViewSeq of
+            true -> ready;
+            false -> DbSeq
         end
     end),
 
-    if Status == ready -> true; true ->
-        subscribe_and_wait_for_index(Db, Mrst, Seq)
-    end.
-
-
-view_up_to_date(Db, Mrst) ->
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        UpdateSeq = couch_views_fdb:get_update_seq(TxDb, Mrst),
-        LastChange = fabric2_fdb:get_last_change(TxDb),
-        {UpdateSeq == LastChange, LastChange}
-    end).
-
-
-maybe_add_couch_job(TxDb, Mrst) ->
-    case couch_views_jobs:status(TxDb, Mrst) of
-        running ->
-            ok;
-        pending ->
-            ok;
-        Status when Status == finished orelse Status == not_found ->
-            couch_views_jobs:add(TxDb, Mrst)
-    end.
-
-
-subscribe_and_wait_for_index(Db, Mrst, Seq) ->
-    case couch_views_jobs:subscribe(Db, Mrst) of
-        {error, Error} ->
-            throw({error, Error});
-        {ok, finished, _} ->
-            ready;
-        {ok, Subscription, _JobState, _} ->
-            wait_for_index_ready(Subscription, Db, Mrst, Seq)
-    end.
-
-
-wait_for_index_ready(Subscription, Db, Mrst, Seq) ->
-    Out = couch_views_jobs:wait(Subscription),
-    case Out of
-        {finished, _JobData} ->
-            ready;
-        {pending, _JobData} ->
-            wait_for_index_ready(Subscription, Db, Mrst, Seq);
-        {running, #{last_seq := LastSeq}} ->
-            if LastSeq =< Seq -> ready; true ->
-                wait_for_index_ready(Subscription, Db, Mrst, Seq)
-            end;
-        {running, _JobData} ->
-            wait_for_index_ready(Subscription, Db, Mrst, Seq);
-        {error, Error} ->
-            throw({error, Error})
+    if WaitSeq == ready -> ok; true ->
+        couch_views_jobs:build_view(Db, Mrst, DbSeq)
     end.
 
 
diff --git a/src/couch_views/src/couch_views_jobs.erl b/src/couch_views/src/couch_views_jobs.erl
index 31ab728..d9c5157 100644
--- a/src/couch_views/src/couch_views_jobs.erl
+++ b/src/couch_views/src/couch_views_jobs.erl
@@ -13,110 +13,85 @@
 -module(couch_views_jobs).
 
 -export([
-    status/2,
-    add/2,
-
-    accept/0,
-    get_job_data/1,
-    update/5,
-    finish/5,
     set_timeout/0,
-
-    subscribe/2,
-    wait/1,
-    unsubscribe/1,
-
-    create_job_id/2
+    build_view/3,
+    build_view_async/2
 ]).
 
 
 -include("couch_views.hrl").
 
 
-% Query request usage of jobs
-
-
-status(TxDb, Mrst) ->
-    JobId = create_job_id(TxDb, Mrst),
-
-    case couch_jobs:get_job_state(TxDb, ?INDEX_JOB_TYPE, JobId) of
-        {ok, State} -> State;
-        {error, not_found} -> not_found;
-        Error -> Error
-    end.
-
-
-add(TxDb, Mrst) ->
-    JobData = create_job_data(TxDb, Mrst, 0),
-
-    JobId = create_job_id(TxDb, Mrst),
-    JTx = couch_jobs_fdb:get_jtx(TxDb),
-    couch_jobs:add(JTx, ?INDEX_JOB_TYPE, JobId, JobData).
-
-
-% couch_views_worker api
-
-
-accept() ->
-    couch_jobs:accept(?INDEX_JOB_TYPE).
-
-
-get_job_data(JobId) ->
-    couch_jobs:get_job_data(undefined, ?INDEX_JOB_TYPE, JobId).
-
-
-update(JTx, Job, Db, Mrst, LastSeq) ->
-    JobData = create_job_data(Db, Mrst, LastSeq),
-    couch_jobs:update(JTx, Job, JobData).
-
-
-finish(JTx, Job, Db, Mrst, LastSeq) ->
-    JobData = create_job_data(Db, Mrst, LastSeq),
-    couch_jobs:finish(JTx, Job, JobData).
-
-
 set_timeout() ->
     couch_jobs:set_type_timeout(?INDEX_JOB_TYPE, 6 * 1000).
 
 
-% Watcher Job api
-
-
-subscribe(Db, Mrst) ->
-    JobId = create_job_id(Db, Mrst),
-    couch_jobs:subscribe(?INDEX_JOB_TYPE, JobId).
-
-
-wait(JobSubscription) ->
-    case couch_jobs:wait(JobSubscription, infinity) of
-        {?INDEX_JOB_TYPE, _JobId, JobState, JobData} -> {JobState, JobData};
-        {timeout} -> {error, timeout}
+build_view(Db, Mrst, UpdateSeq) ->
+    {ok, JobId} = build_view_async(Db, Mrst),
+    case wait_for_job(JobId, UpdateSeq) of
+        ok -> ok;
+        retry -> build_view(Db, Mrst, UpdateSeq)
     end.
 
 
-unsubscribe(JobSubscription) ->
-    couch_jobs:unsubscribe(JobSubscription).
+build_view_async(Db, Mrst) ->
+    JobId = create_job_id(TxDb, Mrst),
+    JobData = create_job_data(TxDb, Mrst),
+    ok = couch_jobs:add(undefined, ?INDEX_JOB_TYPE, JobId, JobData).
+    {ok, JobId}.
+
+
+
+wait_for_job(JobId, UpdateSeq) ->
+    case couch_jobs:subscribe(?INDEX_JOB_TYPE, JobId) of
+        {ok, Subscription, _State, _Data} ->
+            wait_for_job(JobId, Subscription, UpdateSeq)
+        {ok, finished, Data} ->
+            case Data of
+                #{view_seq := ViewSeq} when ViewSeq >= UpdateSeq ->
+                    ok;
+                _ ->
+                    retry
+            end
+    end.
 
 
-% Internal
+wait_for_job(JobId, Subscription, UpdateSeq) ->
+    case wait(Subscription, infinity) of
+        {error, Error} ->
+            erlang:error(Error);
+        {finished, #{view_seq := ViewSeq}} when ViewSeq >= UpdateSeq ->
+            ok;
+        {finished, _} ->
+            wait_for_job(JobId, UpdateSeq);
+        {_State, #{view_seq := ViewSeq}} when ViewSeq >= UpdateSeq ->
+            couch_jobs:unsubscribe(Subscription),
+            ok;
+        {_, _} ->
+            wait_for_job(JobId, Subscription, UpdateSeq)
+    end.
 
 
-create_job_id(#{name := DbName}, #mrst{sig = Sig}) ->
+get_id(#{name := DbName}, #mrst{sig = Sig}) ->
     create_job_id(DbName, Sig);
 
-create_job_id(DbName, Sig) ->
+get_id(DbName, Sig) ->
     <<DbName/binary, Sig/binary>>.
 
 
-create_job_data(Db, Mrst, LastSeq) ->
-    #{name := DbName} = Db,
-
+create_job_data(Db, Mrst) ->
     #mrst{
         idx_name = DDocId
     } = Mrst,
 
     #{
-        db_name => DbName,
-        ddoc_id => DDocId,
-        last_seq => LastSeq
+        db_name => fabric2_db:name(Db),
+        ddoc_id => DDocId
     }.
+
+
+wait(Subscription) ->
+    case couch_jobs:wait(JobSubscription, infinity) of
+        {?INDEX_JOB_TYPE, _JobId, JobState, JobData} -> {JobState, JobData};
+        timeout -> {error, timeout}
+    end.


[couchdb] 12/25: Move all fdb writer logic tou couch_views_fdb.

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

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

commit 5be667c4b1e424d031262404ac6f1bab1f5172ce
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Thu Jul 18 12:29:14 2019 -0500

    Move all fdb writer logic tou couch_views_fdb.
---
 src/couch_views/include/couch_views.hrl |   5 +-
 src/couch_views/src/couch_views_fdb.erl | 259 ++++++++++++++++----------------
 2 files changed, 133 insertions(+), 131 deletions(-)

diff --git a/src/couch_views/include/couch_views.hrl b/src/couch_views/include/couch_views.hrl
index 99a62b0..4fcc57e 100644
--- a/src/couch_views/include/couch_views.hrl
+++ b/src/couch_views/include/couch_views.hrl
@@ -17,8 +17,9 @@
 -define(VIEW_BUILDS,     4).
 -define(VIEW_STATUS,     5).
 -define(VIEW_WATCH,      6).
--define(VIEW_ROW_KEY,    7).
--define(VIEW_ROW_VALUE,  8).
+
+-define(VIEW_ROW_KEY,    0).
+-define(VIEW_ROW_VALUE,  1).
 
 % jobs api
 -define(INDEX_JOB_TYPE, <<"views">>).
diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
index 0791ffa..f47f1b1 100644
--- a/src/couch_views/src/couch_views_fdb.erl
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -14,20 +14,9 @@
 
 -export([
     get_update_seq/2,
-    update_view_seq/3,
-    get_seq_key/2,
-
-    clear_id_index/4,
-    set_id_index/5,
-    get_id_index/4,
-    create_id_index_key/4,
-
-    clear_map_index/5,
-    set_map_index_results/5,
-    get_map_index_key/4,
-    get_map_range_keys/3,
-    get_map_range/4,
-    unpack_map_row/3
+    set_update_seq/3,
+
+    write_rows/4
 ]).
 
 
@@ -42,167 +31,179 @@
 % View Build Sequence Access
 % (<db>, ?DB_VIEWS, Sig, ?VIEW_UPDATE_SEQ) = Sequence
 
-get_update_seq(Db, #mrst{sig = Sig}) ->
+
+get_update_seq(TxDb, #mrst{sig = Sig}) ->
     #{
+        tx := Tx,
         db_prefix := DbPrefix
-    } = Db,
+    } = TxDb,
 
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        Key = get_seq_key(Sig, DbPrefix),
-        Tx = maps:get(tx, TxDb),
-        case erlfdb:wait(erlfdb:get(Tx, Key)) of
-            not_found -> 0;
-            UpdateSeq -> UpdateSeq
-        end
-    end).
+    Key = get_seq_key(Sig, DbPrefix),
+    case erlfdb:wait(erlfdb:get(Tx, Key)) of
+        not_found -> <<>>;
+        UpdateSeq -> UpdateSeq
+    end.
 
 
-update_view_seq(Db, Sig, Seq) ->
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        #{
-            db_prefix := DbPrefix,
-            tx := Tx
-        } = TxDb,
-        SeqKey = get_seq_key(Sig, DbPrefix),
-        erlfdb:set(Tx, SeqKey, Seq)
-    end).
+set_view_seq(TxDb, Sig, Seq) ->
+    #{
+        tx := Tx
+        db_prefix := DbPrefix,
+    } = TxDb,
+    SeqKey = get_seq_key(Sig, DbPrefix),
+    ok = erlfdb:set(Tx, SeqKey, Seq).
 
 
-get_seq_key(Sig, DbPrefix) ->
-    erlfdb_tuple:pack({?DB_VIEWS, Sig, ?VIEW_UPDATE_SEQ}, DbPrefix).
+write_doc(TxDb, Sig, #{deleted := true} = Doc, ViewIds) ->
+    #{
+        id := DocId
+    } = Doc,
 
+    ViewKeys = get_view_keys(TxDb, Sig, DocId),
 
-% Id Index access
+    clear_id_idx(TxDb, Sig, DocId),
+    lists:foreach(fun({ViewId, ViewKeys}) ->
+        clear_map_idx(TxDb, Sig, ViewId, ViewKeys)
+    end, ViewKeys).
 
-% (<db>, ?VIEWS, <sig>, ?VIEW_ID_INDEX, <_id>, <view_id>) -> [emitted keys]
 
-clear_id_index(TxDb, Sig, DocId, IdxName) ->
+write_doc(TxDb, Sig, Doc, ViewIds) ->
     #{
-        db_prefix := DbPrefix,
-        tx := Tx
+        db_prefix := DbPrefix
     } = TxDb,
-    IdKey = create_id_index_key(DbPrefix, Sig, DocId, IdxName),
-    ok = erlfdb:clear(Tx, IdKey).
-
 
-set_id_index(TxDb, Sig, IdxName, DocId, IdxKey) ->
     #{
-        db_prefix := DbPrefix,
-        tx := Tx
-    } = TxDb,
-    IdKey = create_id_index_key(DbPrefix, Sig, DocId, IdxName),
-    erlfdb:set(Tx, IdKey, couch_views_encoding:encode(IdxKey)).
+        id := DocId,
+        results := Results
+    } = Doc,
+
+    ExistingViewKeys = get_view_keys(TxDb, Sig, DocId),
+
+    ok = clear_id_idx(TxDb, Sig, DocId),
+
+    lists:foreach(fun({ViewId, NewRows}) ->
+        ExistingKeys = fabric2_util:get_value(ViewId, ExistingViewKeys, []),
+        update_id_idx(TxDb, Sig, ViewId, DocId, NewRows),
+        update_map_idx(TxDb, Sig, ViewId, DocId, ExitingKeys, NewRows)
+    end, lists:zip(ViewIds, Results)).
 
 
-get_id_index(TxDb, Sig, Id, IdxName) ->
+clear_id_idx(TxDb, Sig, DocId) ->
     #{
-        db_prefix := DbPrefix,
-        tx := Tx
+        tx := Tx,
+        db_prefix := DbPrefix
     } = TxDb,
-    IdKey = create_id_index_key(DbPrefix, Sig, Id, IdxName),
-    case erlfdb:wait(erlfdb:get(Tx, IdKey)) of
-        not_found -> not_found;
-        IdxKey -> couch_views_encoding:decode(IdxKey)
-    end.
 
+    {Start, End} = id_idx_range(DbPrefix, Sig, DocId),
+    ok = erlfdb:clear_range(Start, End).
 
-create_id_index_key(DbPrefix, Sig, DocId, IdxName) ->
-    BaseIdKey = {?DB_VIEWS, Sig, ?VIEW_ID_RANGE, DocId, IdxName},
-    erlfdb_tuple:pack(BaseIdKey, DbPrefix).
 
+clear_map_idx(TxDb, Sig, ViewId, ViewKeys) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = TxDb,
 
-% Map Index Access
-% {<db>, ?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, Idx, Key, DocId,
-%   RowType, Counter} = Values
-% RowType = Emitted Keys or Emitted Value
+    lists:foreach(fun(ViewKey) ->
+        {Start, End} = map_idx_range(DbPrefix, Sig, ViewId, ViewKey, DocId),
+        ok = erlfdb:clear_range(Tx, Start, End)
+    end, ViewKeys).
 
 
-clear_map_index(TxDb, Sig, IdxName, DocId, IdxKeys) when is_list(IdxKeys) ->
-    lists:foreach(fun (IdxKey) ->
-        clear_map_index(TxDb, Sig, IdxName, DocId, IdxKey)
-    end, IdxKeys);
+update_id_idx(TxDb, Sig, ViewId, DocId, NewRows) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = TxDb,
 
-clear_map_index(TxDb, Sig, IdxName, DocId, IdxKey) ->
-    #{db_prefix := DbPrefix, tx := Tx} = TxDb,
-    Key = couch_views_encoding:encode(IdxKey),
-    BaseKey = {?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, IdxName, Key, DocId},
-    {StartKey, EndKey} = erlfdb_tuple:range(BaseKey, DbPrefix),
-    ok = erlfdb:clear_range(Tx, StartKey, EndKey).
+    Unique = lists:usort([K || {K, _V} <- NewRows]),
 
+    Key = id_idx_key(DbPrefix, Sig, ViewId, DocId),
+    Val = couch_views_encoding:encode(Unique),
+    ok = erlfdb:set(Tx, Key, Val).
 
-set_map_index_results(TxDb, Sig, IdxName, DocId, Results) ->
-    #{db_prefix := DbPrefix, tx := Tx} = TxDb,
-    lists:foldl(fun ({IdxKey, IdxValue}, Counter) ->
-        RowKey = create_map_key(DbPrefix, Sig, IdxName, IdxKey, DocId,
-            ?VIEW_ROW_KEY, Counter),
-        RowValue = create_map_key(DbPrefix, Sig, IdxName, IdxKey, DocId,
-            ?VIEW_ROW_VALUE, Counter),
 
-        EncodedKey = pack_value(IdxKey),
-        EncodedValue = pack_value(IdxValue),
+update_map_idx(TxDb, Sig, ViewId, DocId, ExistingKeys, NewRows) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = TxDb,
 
-        ok = erlfdb:set(Tx, RowKey, EncodedKey),
-        ok = erlfdb:set(Tx, RowValue, EncodedValue),
-        Counter + 1
-    end, 0, Results).
+    Unique = lists:usort([K || {K, _V} <- NewRows]),
 
+    KeysToRem = ExistingKeys -- Unique,
+    lists:foreach(fun(RemKey) ->
+        {Start, End} = map_idx_range(DbPrefix, Sig, ViewId, RemKey, DocId),
+        ok = erlfdb:clear_range(Tx, Start, End)
+    end, KeysToRem),
 
-get_map_index_key(#{db_prefix := DbPrefix}, Sig, IdxName, Key) ->
-    EncKey = couch_views_encoding:encode(Key),
-    erlfdb_tuple:pack({?DB_VIEWS, Sig, ?VIEW_MAP_RANGE,
-            IdxName, EncKey}, DbPrefix).
+    KVsToAdd = process_rows(NewRows),
+    MapIdxPrefix = map_idx_prefix(DbPrefix, Sig, ViewId),
 
+    lists:foreach(fun({DupeId, Key1, Key2, Val}) ->
+        KeyKey = map_idx_key(MapIdxPrefix, Key1, DocId, DupeId, ?VIEW_ROW_KEY),
+        ValKey = map_idx_key(MapIdxPrefix, Key1, DocId, DupeId, ?VIEW_ROW_VAL),
+        ok = erlfdn:store(Tx, KeyKey, Key2),
+        ok = erlfdb:store(Tx, ValKey, Val)
+    end, KVsToAdd).
 
-get_map_range_keys(#{db_prefix := DbPrefix}, Sig, IdxName) ->
-    erlfdb_tuple:range({?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, IdxName}, DbPrefix).
 
+get_view_keys(TxDb, Sig, DocId) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = TxDb,
+    {Start, End} = id_idx_range(DbPrefix, Sig, DocId)
+    lists:map(fun({K, V}) ->
+        {?DB_VIEWS, Sig, ?VIEW_ID_RANGE, DocId, ViewId} =
+                erlfdb_tuple:unpack(K, DbPrefix),
+        ViewKeys = couch_views_encoding:decode(V)
+        {ViewId, ViewKeys}
+    end, erlfdb:get_range(Tx, Start, End, [])).
 
-get_map_range(TxDb, Start, End, Opts) ->
-    #{tx := Tx} = TxDb,
-    erlfdb:get_range(Tx, Start, End, Opts).
 
+id_idx_key(DbPrefix, Sig, DocId, ViewId) ->
+    Key = {?DB_VIEWS, Sig, ?VIEW_ID_RANGE, DocId, ViewId},
+    erlfdb_tuple:pack(Key, DbPrefix).
 
-unpack_map_row(#{db_prefix := DbPrefix}, Key, Value) ->
-    case erlfdb_tuple:unpack(Key, DbPrefix) of
-        {?DB_VIEWS, _Sig, ?VIEW_MAP_RANGE, _Idx, _RowKey, Id,
-            ?VIEW_ROW_KEY, _Counter} ->
-            RowKey = unpack_value(Value),
-            {key, Id, RowKey};
 
-        {?DB_VIEWS, _Sig, ?VIEW_MAP_RANGE, _Idx, _RowValue, Id,
-            ?VIEW_ROW_VALUE, _Counter} ->
-            RowValue = unpack_value(Value),
-            {value, Id, RowValue}
-    end.
+id_idx_range(DbPrefix, Sig, DocId) ->
+    Key = {?DB_VIEWS, Sig, ?VIEW_ID_RANGE, DocId},
+    erlfdb_tuple:range(Key, DbPrefix).
 
 
-create_map_key(DbPrefix, Sig, IdxName, IdxKey, DocId, RowType, Counter) ->
-    Key = couch_views_encoding:encode(IdxKey),
-    BaseKey = {?DB_VIEWS, Sig, ?VIEW_MAP_RANGE,
-        IdxName, Key, DocId, RowType, Counter},
-    erlfdb_tuple:pack(BaseKey, DbPrefix).
+map_idx_prefix(DbPrefix, Sig, ViewId) ->
+    Key = {?DB_VIES, Sig, ?VIEW_MAP_RANGE, ViewId},
+    erlfdb_tuple:pack(Key).
 
 
-% Internal used to packed and unpack Values
+map_idx_key(MapIdxPrefix, MapKey, DocId, DupeId, Type)
+    Key = {MapKey, DocId, DupeId, Type},
+    erldb_tuple:encode(Key, MapIdxPrefix).
 
 
-pack_value(Val) when is_list(Val) ->
-    erlfdb_tuple:pack({?LIST_VALUE, list_to_tuple(Val)});
+map_idx_range(DbPrefix, Sig, ViewId, MapKey, DocId) ->
+    Encoded = couch_views_encoding:encode(MapKey, key),
+    Key = {?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, ViewId, Encoded, DocId},
+    erlfdb_tuple:range(Key, DbPrefix).
 
-pack_value(Val) when is_tuple(Val) ->
-    {Props} = Val,
-    erlfdb_tuple:pack({?JSON_VALUE, list_to_tuple(Props)});
 
-pack_value(Val) ->
-    erlfdb_tuple:pack({?VALUE, Val}).
+process_rows(Rows) ->
+    Encoded = lists:map(fun({K, V}) ->
+        EK1 = couch_views_encoding:encode(K, key),
+        EK2 = couch_views_encoding:encode(K, value),
+        EV = couch_views_encoding:encode(V, value),
+        {EKK, EKV, EV}
+    end, Rows),
 
+    Grouped = lists:foldl(fun({K1, K2, V}, Acc) ->
+        dict:append(K1, {K2, V}, Acc)
+    end, dict:new(), Encoded),
 
-unpack_value(Bin) ->
-    case erlfdb_tuple:unpack(Bin) of
-        {?LIST_VALUE, Val} ->
-            tuple_to_list(Val);
-        {?JSON_VALUE, Val} ->
-            {tuple_to_list(Val)};
-        {?VALUE, Val} ->
-            Val
-    end.
+    {_, Labeled} = dict:fold(fun(K1, Vals) ->
+        lists:foldl(fun({K2, V}, {Count, Acc}) ->
+            {Count + 1, [{Count, K1, K2, V} | Acc]}
+        end, {0, []}, Vals)
+    end, [], Grouped),
+
+    Labeled.


[couchdb] 18/25: More bug fixes

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

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

commit 3c39ce548a9ae8e4ee3a55b554cb29223ec87530
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 23 11:29:26 2019 -0500

    More bug fixes
---
 src/couch_views/src/couch_views.erl           |  2 +-
 src/couch_views/src/couch_views_jobs.erl      |  5 +++++
 src/couch_views/src/couch_views_reader.erl    | 18 ++++++++++++------
 src/couch_views/test/couch_views_map_test.erl | 11 ++++++-----
 4 files changed, 24 insertions(+), 12 deletions(-)

diff --git a/src/couch_views/src/couch_views.erl b/src/couch_views/src/couch_views.erl
index e619a67..8c565a6 100644
--- a/src/couch_views/src/couch_views.erl
+++ b/src/couch_views/src/couch_views.erl
@@ -57,7 +57,7 @@ query(Db, DDoc, ViewName, Callback, Acc0, Args0) ->
 maybe_update_view(_Db, _Mrst, #mrargs{update = false}) ->
     ok;
 
-maybe_update_view(_Db, _Mrst, #mrargs{update = laze}) ->
+maybe_update_view(_Db, _Mrst, #mrargs{update = lazy}) ->
     ok;
 
 maybe_update_view(Db, Mrst, _Args) ->
diff --git a/src/couch_views/src/couch_views_jobs.erl b/src/couch_views/src/couch_views_jobs.erl
index 85264c6..15852a1 100644
--- a/src/couch_views/src/couch_views_jobs.erl
+++ b/src/couch_views/src/couch_views_jobs.erl
@@ -18,6 +18,11 @@
     build_view_async/2
 ]).
 
+-ifdef(TEST).
+-compile(export_all).
+-compile(nowarn_export_all).
+-endif.
+
 
 -include_lib("couch_mrview/include/couch_mrview.hrl").
 -include("couch_views.hrl").
diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index 41a37eb..2deb9f6 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -135,20 +135,26 @@ mrargs_to_fdb_options(Args) ->
         couch_views_encoding:encode(EndKey0)
     end,
 
-    EndKeyOpts = case {EndKey1, EndKeyDocId} of
-        {undefined, _} ->
+    EndKeyOpts = case {EndKey1, EndKeyDocId, Direction} of
+        {undefined, _, _} ->
             [];
-        {EndKey1, <<255>>} when not InclusiveEnd ->
+        {EndKey1, <<>>, rev} when not InclusiveEnd ->
+            % When we iterate in reverse with
+            % inclusive_end=false we have to set the
+            % EndKeyDocId to <<255>> so that we don't
+            % include matching rows.
+            [{end_key_gt, {EndKey1, <<255>>}}];
+        {EndKey1, <<255>>, _} when not InclusiveEnd ->
             % When inclusive_end=false we need to
             % elide the default end_key_docid so as
             % to not sort past the docids with the
             % given end key.
             [{end_key_gt, {EndKey1}}];
-        {EndKey1, EndKeyDocId} when not InclusiveEnd ->
+        {EndKey1, EndKeyDocId, _} when not InclusiveEnd ->
             [{end_key_gt, {EndKey1, EndKeyDocId}}];
-        {EndKey1, EndKeyDocId} when InclusiveEnd ->
+        {EndKey1, EndKeyDocId, _} when InclusiveEnd ->
             [{end_key, {EndKey1, EndKeyDocId}}];
-        {EndKey1, EndKeyDocId} when InclusiveEnd ->
+        {EndKey1, EndKeyDocId, _} when InclusiveEnd ->
             [{end_key_gt, {EndKey1, EndKeyDocId}}]
     end,
 
diff --git a/src/couch_views/test/couch_views_map_test.erl b/src/couch_views/test/couch_views_map_test.erl
index ab3000e..c5db382 100644
--- a/src/couch_views/test/couch_views_map_test.erl
+++ b/src/couch_views/test/couch_views_map_test.erl
@@ -76,7 +76,7 @@ should_map() ->
 
 
 should_map_with_startkey() ->
-    Result = run_query(<<"baz">>, #{start_key => 4}, true),
+    Result = run_query(<<"baz">>, #{start_key => 4}),
     Expect = {ok, [
         {row, [{id, <<"4">>}, {key, 4}, {value, 4}]},
         {row, [{id, <<"5">>}, {key, 5}, {value, 5}]},
@@ -276,7 +276,7 @@ should_map_with_startkey_and_endkey_with_key_array() ->
     % expect rows 9-4
     ?assertEqual({ok, lists:reverse(Rows2)}, ResultRev2),
 
-    ResultRev2 = run_query(<<"boom">>, #{
+    ResultRev3 = run_query(<<"boom">>, #{
         start_key => [<<"9">>, 9],
         end_key => [<<"4">>, 4],
         direction => rev,
@@ -285,7 +285,7 @@ should_map_with_startkey_and_endkey_with_key_array() ->
 
     % Here, specifying [<<"4">>, 4] as the key will prevent
     % us from including that row which leaves rows 9-5
-    ?assertEqual({ok, lists:reverse(lists:nthtail(1, Rows2))}, ResultRev2).
+    ?assertEqual({ok, lists:reverse(lists:nthtail(1, Rows2))}, ResultRev3).
 
 
 
@@ -446,8 +446,9 @@ run_query(Idx, Args, DebugCluster) ->
     Docs = make_docs(10),
     fabric2_db:update_docs(Db, [DDoc | Docs]),
     if not DebugCluster -> ok; true ->
-        %% couch_views:query(Db, DDoc, Idx, fun default_cb/2, [], #{}),
-        %% fabric2_fdb:debug_cluster()
+        io:format(standard_error, "Database: ~s~n", [DbName]),
+        couch_views:query(Db, DDoc, Idx, fun default_cb/2, [], #{}),
+        fabric2_fdb:debug_cluster(),
         ok
     end,
     couch_views:query(Db, DDoc, Idx, fun default_cb/2, [], Args).


[couchdb] 23/25: Make fabric2.hrl public

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

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

commit 3ed94108075bade99c011dd75ec526b217f2803b
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 23 15:12:18 2019 -0500

    Make fabric2.hrl public
---
 src/couch_views/src/couch_views_fdb.erl     | 2 +-
 src/couch_views/src/couch_views_indexer.erl | 5 +++--
 src/couch_views/src/couch_views_reader.erl  | 2 +-
 src/fabric/{src => include}/fabric2.hrl     | 0
 4 files changed, 5 insertions(+), 4 deletions(-)

diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
index 16da0fc..09a9802 100644
--- a/src/couch_views/src/couch_views_fdb.erl
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -32,7 +32,7 @@
 
 
 -include_lib("couch_mrview/include/couch_mrview.hrl").
--include_lib("fabric/src/fabric2.hrl").
+-include_lib("fabric/include/fabric2.hrl").
 -include("couch_views.hrl").
 
 % View Build Sequence Access
diff --git a/src/couch_views/src/couch_views_indexer.erl b/src/couch_views/src/couch_views_indexer.erl
index decec42..0000700 100644
--- a/src/couch_views/src/couch_views_indexer.erl
+++ b/src/couch_views/src/couch_views_indexer.erl
@@ -21,10 +21,11 @@
     init/0
 ]).
 
+
+-include("couch_views.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_mrview/include/couch_mrview.hrl").
--include_lib("fabric/src/fabric2.hrl").
--include("couch_views.hrl").
+-include_lib("fabric/include/fabric2.hrl").
 
 % TODO:
 %  * Handle timeouts of transaction and other errors
diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index c26b313..ce5097ba 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -20,7 +20,7 @@
 -include("couch_views.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("couch_mrview/include/couch_mrview.hrl").
--include_lib("fabric/src/fabric2.hrl").
+-include_lib("fabric/include/fabric2.hrl").
 
 
 read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
diff --git a/src/fabric/src/fabric2.hrl b/src/fabric/include/fabric2.hrl
similarity index 100%
rename from src/fabric/src/fabric2.hrl
rename to src/fabric/include/fabric2.hrl


[couchdb] 14/25: fixup! Expose the is_replicator_db and is_user_db logic

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

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

commit 047ccabe3b452c5777aec0f2376681c33ba92e81
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Thu Jul 18 15:04:37 2019 -0500

    fixup! Expose the is_replicator_db and is_user_db logic
---
 src/fabric/src/fabric2_db.erl   | 2 +-
 src/fabric/src/fabric2_util.erl | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/src/fabric/src/fabric2_db.erl b/src/fabric/src/fabric2_db.erl
index a89b098..c926da9 100644
--- a/src/fabric/src/fabric2_db.erl
+++ b/src/fabric/src/fabric2_db.erl
@@ -385,7 +385,7 @@ is_replicator_db(#{name := DbName}) ->
     is_replicator_db(DbName);
 
 is_replicator_db(DbName) when is_binary(DbName) ->
-    fabric2_util:dbname_ends_with(Db, <<"_replicator">>).
+    fabric2_util:dbname_ends_with(DbName, <<"_replicator">>).
 
 
 is_users_db(#{name := DbName}) ->
diff --git a/src/fabric/src/fabric2_util.erl b/src/fabric/src/fabric2_util.erl
index 1921bca..2b8e49e 100644
--- a/src/fabric/src/fabric2_util.erl
+++ b/src/fabric/src/fabric2_util.erl
@@ -125,7 +125,7 @@ validate_json_list_of_strings(Member, Props) ->
 
 
 dbname_ends_with(#{} = Db, Suffix) ->
-    dbname_ends_with(fabric2_db:name(Db), Suffix).
+    dbname_ends_with(fabric2_db:name(Db), Suffix);
 
 dbname_ends_with(DbName, Suffix) when is_binary(DbName), is_binary(Suffix) ->
     Suffix == filename:basename(DbName).


[couchdb] 16/25: Fix default fold_range bounds

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

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

commit 2e3da3f876808595482ebf56c370733252132242
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Fri Jul 19 12:41:02 2019 -0500

    Fix default fold_range bounds
---
 src/fabric/src/fabric2_fdb.erl | 31 ++++++++++++++++---------------
 1 file changed, 16 insertions(+), 15 deletions(-)

diff --git a/src/fabric/src/fabric2_fdb.erl b/src/fabric/src/fabric2_fdb.erl
index 670ce8b..71cb68f 100644
--- a/src/fabric/src/fabric2_fdb.erl
+++ b/src/fabric/src/fabric2_fdb.erl
@@ -985,18 +985,19 @@ get_fold_opts(RangePrefix, Options) ->
 
     % Set the maximum bounds for the start and endkey
     StartKey2 = case StartKey1 of
-        undefined -> <<>>;
-        SK2 -> SK2
+        undefined ->
+            <<RangePrefix/binary, 16#00>>;
+        SK2 ->
+            erlfdb_tuple:pack({SK2}, RangePrefix)
     end,
 
     EndKey2 = case EndKey1 of
-        undefined -> <<255>>;
-        EK2 -> EK2
+        undefined ->
+            <<RangePrefix/binary, 16#FF>>;
+        EK2 ->
+            erlfdb_tuple:pack({EK2}, RangePrefix)
     end,
 
-    StartKey3 = erlfdb_tuple:pack({StartKey2}, RangePrefix),
-    EndKey3 = erlfdb_tuple:pack({EndKey2}, RangePrefix),
-
     % FoundationDB ranges are applied as SK <= key < EK
     % By default, CouchDB is SK <= key <= EK with the
     % optional inclusive_end=false option changing that
@@ -1006,20 +1007,20 @@ get_fold_opts(RangePrefix, Options) ->
     % Thus we have this wonderful bit of logic to account
     % for all of those combinations.
 
-    StartKey4 = case {Reverse, InclusiveEnd} of
+    StartKey3 = case {Reverse, InclusiveEnd} of
         {true, false} ->
-            erlfdb_key:first_greater_than(StartKey3);
+            erlfdb_key:first_greater_than(StartKey2);
         _ ->
-            StartKey3
+            StartKey2
     end,
 
-    EndKey4 = case {Reverse, InclusiveEnd} of
+    EndKey3 = case {Reverse, InclusiveEnd} of
         {false, true} when EndKey0 /= undefined ->
-            erlfdb_key:first_greater_than(EndKey3);
+            erlfdb_key:first_greater_than(EndKey2);
         {true, _} ->
-            erlfdb_key:first_greater_than(EndKey3);
+            erlfdb_key:first_greater_than(EndKey2);
         _ ->
-            EndKey3
+            EndKey2
     end,
 
     Skip = case fabric2_util:get_value(skip, Options) of
@@ -1053,7 +1054,7 @@ get_fold_opts(RangePrefix, Options) ->
             ++ StreamingMode
             ++ Snapshot,
 
-    {StartKey4, EndKey4, Skip, OutOpts}.
+    {StartKey3, EndKey3, Skip, OutOpts}.
 
 
 fold_range_cb(KV, {skip, 0, Callback, Acc}) ->


[couchdb] 06/25: Don't add a fabric2_view module

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

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

commit d0cfb019ee44d2015511fadd6e12698536cb4221
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 16 15:12:27 2019 -0500

    Don't add a fabric2_view module
    
    There was barely any logic contained there that's easily included in the
    couch_views module. Its mostly historical that fabric includes all of
    the view logic anyway. Now that we have multiple indexers it's more
    hygenic to move all the code to a single application.
    
    For the chttpd level I'm going to leave the logic in chttpd_view.erl for
    the time being. When we go to clean up code in terms of deleting unused
    code and moving used code out of mostly deleted applications.
---
 src/couch_views/src/couch_views.erl | 101 +++++++++++++++++++++++++++---------
 src/fabric/src/fabric2_view.erl     |  81 -----------------------------
 2 files changed, 76 insertions(+), 106 deletions(-)

diff --git a/src/couch_views/src/couch_views.erl b/src/couch_views/src/couch_views.erl
index 69d6765..c059204 100644
--- a/src/couch_views/src/couch_views.erl
+++ b/src/couch_views/src/couch_views.erl
@@ -13,41 +13,48 @@
 -module(couch_views).
 
 -export([
-    map_query/6
+    query/6
 ]).
 
--include("couch_views.hrl").
+-include("couch_mrview/include/couch_mrview.hrl").
 
 
-map_query(Db, DDoc, ViewName, Callback, Acc0, Args0) ->
-    Args = process_args(Args0),
-    #{name := DbName} = Db,
-    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
-    maybe_build_index(Db, Mrst, Args),
-    Resp = couch_views_reader:read(Db, DDoc, ViewName, Callback, Acc0, Args),
-
-    UpdateAfter = maps:get(update, Args) == lazy,
-    if UpdateAfter == false -> ok; true ->
-        maybe_add_couch_job(Db, Mrst)
+query(Db, DDoc, ViewName, Callback, Acc0, QueryArgs0) ->
+    case fabric2_db:is_users_db(Db) of
+        true ->
+            fabric2_users_db:after_doc_read(DDoc, Db);
+        false ->
+            ok
     end,
-    Resp.
 
+    DbName = fabric2_db:name(Db),
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
 
-process_args(#{} = Args) ->
-    Args1 = remove_ununsed_values(Args),
-    Defaults = #{
-            direction => fwd,
-            inclusive_end => true,
-            update => true,
-            skip => 0,
-            limit => ?MAX_VIEW_LIMIT
-        },
+    #mrst{
+        views = Views,
+        language = Lang
+    } = Mrst,
+
+    QueryArgs1 = couch_mrview_util:set_view_type(QueryArgs0, View, Views),
+    QueryArgs2 = couch_mrview_util:validate_args(QueryArgs1),
+    VInfo = couch_mrview_util:extract_view(Lang, QueryArgs2, View, Views),
+    case is_reduce_view(QueryArgs2) of
+        true -> throw({not_implemented});
+        false -> ok
+    end,
 
-    maps:merge(Defaults, Args1).
+    Args = mrargs_to_map(QueryArgs2),
 
+    maybe_build_view(Db, MrSt, Args),
 
-remove_ununsed_values(Args) ->
-    maps:filter(fun (_, V) -> V /= undefined end, Args).
+    try
+        couch_views_reader:read(Db, Mrst, ViewName, Callback, Acc0, Args)
+    after
+        UpdateAfter = maps:get(update, Args) == lazy,
+        if UpdateAfter == false -> ok; true ->
+            maybe_add_couch_job(Db, Mrst)
+        end
+    end.
 
 
 maybe_build_index(_Db, _Mrst, #{update := false}) ->
@@ -118,3 +125,47 @@ wait_for_index_ready(Subscription, Db, Mrst, Seq) ->
         {error, Error} ->
             throw({error, Error})
     end.
+
+
+is_reduce_view(#mrargs{view_type = ViewType}) ->
+    ViewType =:= red;
+is_reduce_view({Reduce, _, _}) ->
+    Reduce =:= red.
+
+
+mrargs_to_map(#mrargs{} = Args) ->
+    process_args(#{
+        start_key => Args#mrargs.start_key,
+        start_key_docid => Args#mrargs.start_key_docid,
+        end_key => Args#mrargs.end_key,
+        end_key_docid => Args#mrargs.end_key_docid,
+        keys => Args#mrargs.keys,
+        direction => Args#mrargs.direction,
+        limit => Args#mrargs.limit,
+        skip => Args#mrargs.skip,
+        update => Args#mrargs.update,
+        multi_get => Args#mrargs.multi_get,
+        inclusive_end => Args#mrargs.inclusive_end,
+        include_docs => Args#mrargs.include_docs,
+        doc_options => Args#mrargs.doc_options,
+        update_seq => Args#mrargs.update_seq,
+        conflicts => Args#mrargs.conflicts,
+        sorted => Args#mrargs.sorted
+    }).
+
+
+process_args(#{} = Args) ->
+    Args1 = remove_ununsed_values(Args),
+    Defaults = #{
+            direction => fwd,
+            inclusive_end => true,
+            update => true,
+            skip => 0,
+            limit => ?MAX_VIEW_LIMIT
+        },
+
+    maps:merge(Defaults, Args1).
+
+
+remove_ununsed_values(Args) ->
+    maps:filter(fun (_, V) -> V /= undefined end, Args).
diff --git a/src/fabric/src/fabric2_view.erl b/src/fabric/src/fabric2_view.erl
deleted file mode 100644
index 01c9ab0..0000000
--- a/src/fabric/src/fabric2_view.erl
+++ /dev/null
@@ -1,81 +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.
-
--module(fabric2_view).
-
--export([
-    query/7
-]).
-
--include_lib("couch_mrview/include/couch_mrview.hrl").
-
-%% @doc execute a given view.
-%%      There are many additional query args that can be passed to a view,
-%%      see <a href="http://wiki.apache.org/couchdb/HTTP_view_API#Querying_Options">
-%%      query args</a> for details.
-% -spec query(db(), [{atom(), any()}] | [],
-%         #doc{} | binary(), iodata(), callback(), any(), #mrargs{}) -> any().
-query(Db, Options, DDoc, ViewName, Callback, Acc0, QueryArgs0) ->
-    DbName = fabric2_db:name(Db),
-%%    View = name(ViewName),
-    case fabric_util:is_users_db(DbName) of
-        true ->
-            FakeDb = fabric_util:open_cluster_db(DbName, Options),
-            couch_users_db:after_doc_read(DDoc, FakeDb);
-        false ->
-            ok
-    end,
-%%    {ok, #mrst{views=Views, language=Lang}} =
-%%        couch_views_util:ddoc_to_mrst(DbName, DDoc),
-%%    QueryArgs1 = couch_mrview_util:set_view_type(QueryArgs0, View, Views),
-%%    QueryArgs1 = fabric_util:validate_args(Db, DDoc, QueryArgs0),
-    QueryArgs1 = couch_mrview_util:validate_args(Db, DDoc, QueryArgs0),
-%%    VInfo = couch_mrview_util:extract_view(Lang, QueryArgs1, View, Views),
-    case is_reduce_view(QueryArgs1) of
-        true ->
-            throw({not_implemented});
-        false ->
-            MapQueryArgs = mrargs_to_map((QueryArgs1)),
-            couch_views:map_query(Db, DDoc, ViewName, Callback,
-                Acc0, MapQueryArgs)
-    end.
-
-
-is_reduce_view(_) ->
-    false.
-
-
-name(Thing) ->
-    couch_util:to_binary(Thing).
-
-
-mrargs_to_map(#mrargs{} = Args) ->
-    #{
-        start_key => Args#mrargs.start_key,
-        start_key_docid => Args#mrargs.start_key_docid,
-        end_key => Args#mrargs.end_key,
-        end_key_docid => Args#mrargs.end_key_docid,
-        keys => Args#mrargs.keys,
-        direction => Args#mrargs.direction,
-        limit => Args#mrargs.limit,
-        skip => Args#mrargs.skip,
-        update => Args#mrargs.update,
-        multi_get => Args#mrargs.multi_get,
-        inclusive_end => Args#mrargs.inclusive_end,
-        include_docs => Args#mrargs.include_docs,
-        doc_options => Args#mrargs.doc_options,
-        update_seq => Args#mrargs.update_seq,
-        conflicts => Args#mrargs.conflicts,
-        sorted => Args#mrargs.sorted
-    }.
-
-


[couchdb] 22/25: Support the `keys` option

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

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

commit 2f26a949c25e332d78f3502fe5d0c7136af76c1c
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 23 14:25:09 2019 -0500

    Support the `keys` option
---
 src/couch_views/src/couch_views_reader.erl | 37 ++++++++++++++++++++++--------
 1 file changed, 27 insertions(+), 10 deletions(-)

diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index 25144ba..c26b313 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -31,7 +31,6 @@ read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
     } = Mrst,
 
     ViewId = get_view_id(Lang, Args, ViewName, Views),
-    Opts = mrargs_to_fdb_options(Args),
     Fun = fun handle_row/4,
 
     try
@@ -43,19 +42,25 @@ read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
             Acc0 = #{
                 db => TxDb,
                 skip => Args#mrargs.skip,
-                mrargs => Args,
+                mrargs => undefined,
                 callback => UserCallback,
                 acc => UserAcc1
             },
 
-            Acc1 = couch_views_fdb:fold_map_idx(
-                    TxDb,
-                    Sig,
-                    ViewId,
-                    Opts,
-                    Fun,
-                    Acc0
-                ),
+            Acc1 = lists:foldl(fun(KeyArgs, KeyAcc0) ->
+                Opts = mrargs_to_fdb_options(KeyArgs),
+                KeyAcc1 = KeyAcc0#{
+                    mrargs := KeyArgs
+                },
+                couch_views_fdb:fold_map_idx(
+                        TxDb,
+                        Sig,
+                        ViewId,
+                        Opts,
+                        Fun,
+                        KeyAcc1
+                    )
+            end, Acc0, expand_keys_args(Args)),
 
             #{
                 acc := UserAcc2
@@ -108,6 +113,18 @@ get_view_id(Lang, Args, ViewName, Views) ->
     end.
 
 
+expand_keys_args(#mrargs{keys = undefined} = Args) ->
+    [Args];
+
+expand_keys_args(#mrargs{keys = Keys} = Args) ->
+    lists:map(fun(Key) ->
+        Args#mrargs{
+            start_key = Key,
+            end_key = Key
+        }
+    end, Keys).
+
+
 mrargs_to_fdb_options(Args) ->
     #mrargs{
         start_key = StartKey0,


[couchdb] 02/25: fixes based on reviews

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

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

commit 64be6bf4afbdadd857e1791ed004b1cb5c3399fe
Author: Garren Smith <ga...@gmail.com>
AuthorDate: Mon Jul 8 16:40:29 2019 +0200

    fixes based on reviews
---
 rel/overlay/etc/default.ini                       |  4 +-
 src/couch_views/src/couch_views.erl               | 23 ++++---
 src/couch_views/src/couch_views_jobs.erl          |  2 +-
 src/couch_views/src/couch_views_reader.erl        | 63 ++++++++---------
 src/couch_views/src/couch_views_worker_server.erl | 84 +++++++++++++----------
 src/couch_views/test/couch_views_map_test.erl     |  1 -
 test/elixir/test/map_test.exs                     | 13 +---
 7 files changed, 92 insertions(+), 98 deletions(-)

diff --git a/rel/overlay/etc/default.ini b/rel/overlay/etc/default.ini
index 59b7d57..11bd611 100644
--- a/rel/overlay/etc/default.ini
+++ b/rel/overlay/etc/default.ini
@@ -225,9 +225,7 @@ iterations = 10 ; iterations for password hashing
 
 ; Settings for view indexing
 [couch_views]
-; type_check_period_msec = 500
-; type_check_max_jitter_msec = 500
-; change_limit = 100
+; max_workers = 100
 
 ; CSP (Content Security Policy) Support for _utils
 [csp]
diff --git a/src/couch_views/src/couch_views.erl b/src/couch_views/src/couch_views.erl
index 4ccf0fa..69d6765 100644
--- a/src/couch_views/src/couch_views.erl
+++ b/src/couch_views/src/couch_views.erl
@@ -34,15 +34,20 @@ map_query(Db, DDoc, ViewName, Callback, Acc0, Args0) ->
 
 
 process_args(#{} = Args) ->
-    Args1 = maps:filter(fun (_, V) -> V /= undefined end, Args),
-
-    maps:merge(#{
-        direction => fwd,
-        inclusive_end => true,
-        update => true,
-        skip => 0,
-        limit => ?MAX_VIEW_LIMIT
-    }, Args1).
+    Args1 = remove_ununsed_values(Args),
+    Defaults = #{
+            direction => fwd,
+            inclusive_end => true,
+            update => true,
+            skip => 0,
+            limit => ?MAX_VIEW_LIMIT
+        },
+
+    maps:merge(Defaults, Args1).
+
+
+remove_ununsed_values(Args) ->
+    maps:filter(fun (_, V) -> V /= undefined end, Args).
 
 
 maybe_build_index(_Db, _Mrst, #{update := false}) ->
diff --git a/src/couch_views/src/couch_views_jobs.erl b/src/couch_views/src/couch_views_jobs.erl
index ff99475..31ab728 100644
--- a/src/couch_views/src/couch_views_jobs.erl
+++ b/src/couch_views/src/couch_views_jobs.erl
@@ -50,7 +50,7 @@ add(TxDb, Mrst) ->
     JobData = create_job_data(TxDb, Mrst, 0),
 
     JobId = create_job_id(TxDb, Mrst),
-    JTx = couch_jobs_fdb:get_jtx(),
+    JTx = couch_jobs_fdb:get_jtx(TxDb),
     couch_jobs:add(JTx, ?INDEX_JOB_TYPE, JobId, JobData).
 
 
diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index 2ddb5b6..f4e768a 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -136,27 +136,12 @@ get_unpack_fun(TxDb, Opts, Callback) ->
     UnPackFwd = fun({K, V}, State) ->
         case couch_views_fdb:unpack_map_row(TxDb, K, V) of
             {key, _Id, RowKey} ->
-                maps:put(current_key, RowKey, State);
+                State#{current_key => RowKey};
             {value, Id, RowValue} ->
                 #{
-                    current_key := RowKey,
-                    acc := Acc,
-                    skip := Skip,
-                    db := Db
+                    current_key := RowKey
                 } = State,
-
-                case Skip > 0 of
-                    true ->
-                        maps:put(skip, Skip - 1, State);
-                    false ->
-                        Row = [{id, Id}, {key, RowKey}, {value, RowValue}],
-
-                        IncludeDoc = maps:get(include_docs, State, false),
-                        Row1 = maybe_include_doc(Db, Id, Row, IncludeDoc),
-
-                        {ok, AccNext} = Callback({row, Row1}, Acc),
-                        maps:put(acc, AccNext, State)
-                end
+                process_map_row(Id, RowKey, RowValue, State, Callback)
         end
     end,
 
@@ -164,26 +149,11 @@ get_unpack_fun(TxDb, Opts, Callback) ->
         case couch_views_fdb:unpack_map_row(TxDb, K, V) of
             {key, Id, RowKey} ->
                 #{
-                    current_value := RowValue,
-                    acc := Acc,
-                    skip := Skip,
-                    db := Db
+                    current_value := RowValue
                 } = State,
-
-                case Skip > 0 of
-                    true ->
-                        maps:put(skip, Skip - 1, State);
-                    false ->
-                        Row = [{id, Id}, {key, RowKey}, {value, RowValue}],
-
-                        IncludeDoc = maps:get(include_docs, State, false),
-                        Row1 = maybe_include_doc(Db, Id, Row, IncludeDoc),
-
-                        {ok, AccNext} = Callback({row, Row1}, Acc),
-                        maps:put(acc, AccNext, State)
-                end;
+                process_map_row(Id, RowKey, RowValue, State, Callback);
             {value, _Id, RowValue} ->
-                maps:put(current_value, RowValue, State)
+                State#{current_value => RowValue}
         end
     end,
 
@@ -193,6 +163,27 @@ get_unpack_fun(TxDb, Opts, Callback) ->
     end.
 
 
+process_map_row(Id, RowKey, RowValue, State, Callback) ->
+    #{
+        acc := Acc,
+        skip := Skip,
+        db := Db
+    } = State,
+
+    case Skip > 0 of
+        true ->
+            State#{skip := Skip -1};
+        false ->
+            Row = [{id, Id}, {key, RowKey}, {value, RowValue}],
+
+            IncludeDoc = maps:get(include_docs, State, false),
+            Row1 = maybe_include_doc(Db, Id, Row, IncludeDoc),
+
+            {ok, AccNext} = Callback({row, Row1}, Acc),
+            State#{acc := AccNext}
+    end.
+
+
 maybe_include_doc(_Db, _Id, Row, false) ->
     Row;
 
diff --git a/src/couch_views/src/couch_views_worker_server.erl b/src/couch_views/src/couch_views_worker_server.erl
index 1c815e5..13bd9aa 100644
--- a/src/couch_views/src/couch_views_worker_server.erl
+++ b/src/couch_views/src/couch_views_worker_server.erl
@@ -31,8 +31,7 @@
 ]).
 
 
--define(TYPE_CHECK_PERIOD_DEFAULT, 500).
--define(MAX_JITTER_DEFAULT, 100).
+-define(MAX_WORKERS, 100).
 
 
 start_link() ->
@@ -41,8 +40,12 @@ start_link() ->
 
 init(_) ->
     couch_views_jobs:set_timeout(),
-    schedule_check(),
-    {ok, #{}}.
+    State0 = #{
+        workers => #{},
+        acceptor_pid => undefined
+    },
+    State = spawn_acceptor(State0),
+    {ok, State}.
 
 
 terminate(_, _St) ->
@@ -53,19 +56,20 @@ handle_call(Msg, _From, St) ->
     {stop, {bad_call, Msg}, {bad_call, Msg}, St}.
 
 
+handle_cast({job, Job, JobData}, State) ->
+    State1 = start_worker(State, Job, JobData),
+    State2 = spawn_acceptor(State1),
+    {noreply, State2};
+
 handle_cast(Msg, St) ->
     {stop, {bad_cast, Msg}, St}.
 
 
-handle_info(check_for_jobs, State) ->
-    accept_jobs(),
-    schedule_check(),
-    {noreply, State};
-
-handle_info({'DOWN', _Ref, process, Pid, Reason}, St) ->
+handle_info({'DOWN', _Ref, process, Pid, Reason}, State) ->
     LogMsg = "~p : process ~p exited with ~p",
     couch_log:error(LogMsg, [?MODULE, Pid, Reason]),
-    {noreply, St};
+    State1 = check_finished_process(State, Pid),
+    {noreply, State1};
 
 handle_info(Msg, St) ->
     couch_log:notice("~s ignoring info ~w", [?MODULE, Msg]),
@@ -76,35 +80,43 @@ code_change(_OldVsn, St, _Extra) ->
     {ok, St}.
 
 
-accept_jobs() ->
+start_worker(State, Job, JobData) ->
+    #{workers := Workers} = State,
+    {Pid, _Ref} = spawn_monitor(fun () -> couch_views_worker:start(Job, JobData) end),
+    Workers1 = Workers#{Pid => true},
+    State#{workers := Workers1}.
+
+
+spawn_acceptor(State) ->
+    #{
+        workers := Workers,
+        acceptor_pid := Pid
+    } = State,
+    MaxWorkers = config:get_integer("couch_views", "max_workers", ?MAX_WORKERS),
+    case maps:size(Workers) >= MaxWorkers of
+        false when not is_pid(Pid) ->
+            Parent = self(),
+            {Pid1, _Ref} = spawn_monitor(fun() -> blocking_acceptor(Parent) end),
+            State#{acceptor_pid := Pid1};
+        _ ->
+            State
+    end.
+
+
+blocking_acceptor(Parent) ->
     case couch_views_jobs:accept() of
         not_found ->
-            ok;
+            blocking_acceptor(Parent);
         {ok, Job, JobData} ->
-            start_worker(Job, JobData),
-            % keep accepting jobs until not_found
-            accept_jobs()
+            gen_server:cast(Parent, {job, Job, JobData})
     end.
 
 
-start_worker(Job, JobData) ->
-    % TODO Should I monitor it, or let jobs do that?
-    spawn_monitor(fun () -> couch_views_worker:start(Job, JobData) end),
-    ok.
-
-
-schedule_check() ->
-    Timeout = get_period_msec(),
-    MaxJitter = max(Timeout div 2, get_max_jitter_msec()),
-    Wait = Timeout + rand:uniform(max(1, MaxJitter)),
-    timer:send_after(Wait, self(), check_for_jobs).
-
-
-get_period_msec() ->
-    config:get_integer("couch_views", "type_check_period_msec",
-        ?TYPE_CHECK_PERIOD_DEFAULT).
-
+check_finished_process(#{acceptor_pid := Pid} = State, Pid) ->
+    State1 = State#{acceptor_pid := undefined},
+    spawn_acceptor(State1);
 
-get_max_jitter_msec() ->
-    config:get_integer("couch_views", "type_check_max_jitter_msec",
-        ?MAX_JITTER_DEFAULT).
+check_finished_process(State, Pid) ->
+    #{workers := Workers} = State,
+    Workers1 = maps:remove(Pid, Workers),
+    State#{workers := Workers1}.
diff --git a/src/couch_views/test/couch_views_map_test.erl b/src/couch_views/test/couch_views_map_test.erl
index bbad93f..e7be521 100644
--- a/src/couch_views/test/couch_views_map_test.erl
+++ b/src/couch_views/test/couch_views_map_test.erl
@@ -300,7 +300,6 @@ should_map_duplicate_keys() ->
         {row, [{id, <<"2">>}, {key, <<"2">>}, {value, 2}]},
         {row, [{id, <<"2">>}, {key, <<"2">>}, {value, 3}]}
     ]},
-    ?debugFmt("EXPE ~p ~n", [Expect]),
     ?assertEqual(Expect, Result).
 
 
diff --git a/test/elixir/test/map_test.exs b/test/elixir/test/map_test.exs
index b7a809d..7c443ab 100644
--- a/test/elixir/test/map_test.exs
+++ b/test/elixir/test/map_test.exs
@@ -19,7 +19,7 @@ defmodule ViewMapTest do
             "two"
           end
 
-        doc = %{
+        %{
           :_id => "doc-id-#{i}",
           :value => i,
           :some => "field",
@@ -78,17 +78,6 @@ defmodule ViewMapTest do
     resp = Couch.post("/#{db_name}/_bulk_docs", body: body)
     Enum.each(resp.body, &assert(&1["ok"]))
 
-    #        ddoc = %{
-    #            :_id => "_design/map",
-    #            views: %{
-    #                some: %{map: map_fun1},
-    #                map_some: %{map: map_fun2}
-    #            }
-    #        }
-    #        resp = Couch.put("/#{db_name}/#{ddoc._id}", body: ddoc)
-    #        IO.inspect resp
-    #        assert resp.status_code == 201
-
     {:ok, [db_name: db_name]}
   end
 


[couchdb] 11/25: Fix encoding layer for different key types

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

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

commit 6678306266a676bb4ebb57e3eaad32f403f0782e
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Thu Jul 18 12:27:13 2019 -0500

    Fix encoding layer for different key types
    
    We need to be able to selective use get_sort_key or not based on whether
    we're encoding the key for the key or value. Which makes no sense unless
    you remember we're storing two FDB rows for every view row. One is
    `(KeyWithSortKeys, OriginalKey)` and the second is `(KeyWithSortKeys,
    Value)`. There's a `Type` that I've elided that distinguishes between
    the two.
---
 src/couch_views/src/couch_views_encoding.erl | 85 ++++++++++++++--------------
 1 file changed, 41 insertions(+), 44 deletions(-)

diff --git a/src/couch_views/src/couch_views_encoding.erl b/src/couch_views/src/couch_views_encoding.erl
index 3af6d7f..9d3e3fc 100644
--- a/src/couch_views/src/couch_views_encoding.erl
+++ b/src/couch_views/src/couch_views_encoding.erl
@@ -15,65 +15,65 @@
 
 -export([
     encode/1,
+    encode/2,
     decode/1
 ]).
 
 
--define(NULL, 16#00).
--define(FALSE, 16#26).
--define(TRUE, 16#27).
--define(NUMBER, 16#40).
--define(STRING, 16#41).
--define(LIST, 16#42).
--define(OBJECT, 16#43).
+-define(NULL, 0).
+-define(FALSE, 1).
+-define(TRUE, 2).
+-define(NUMBER, 3).
+-define(STRING, 4).
+-define(LIST, 5).
+-define(OBJECT, 6).
 
 
 encode(X) ->
-    Encoded = encode_int(X),
-    erlfdb_tuple:pack(Encoded).
+    encode_int(X, value).
 
 
-decode(EncodedVal) ->
-    Val = erlfdb_tuple:unpack(EncodedVal),
-    decode_int(Val).
+encode(X, Type) when Type == key; Type == value ->
+    erlfdb_tuple:pack(encode_int(X, value))
 
 
-encode_int(X) when is_atom(X) -> encode_atom(X);
-encode_int(X) when is_number(X) -> encode_number(X);
-encode_int(X) when is_binary(X) -> encode_binary(X);
-encode_int(X) when is_list(X) -> encode_list(X);
-encode_int(X) when is_tuple(X) -> encode_object(X).
+decode(Encoded) ->
+    Val = erlfdb_tuple:unpack(Encoded),
+    decode_int(Val).
 
 
-encode_atom(null) ->
+encode_int(null, _Type) ->
     {?NULL};
 
-encode_atom(false) ->
+encode_int(false, _Type) ->
     {?FALSE};
 
-encode_atom(true) ->
-    {?TRUE}.
-
-
-encode_number(Val) ->
-    {?NUMBER, float(Val)}.
+encode_int(true, _Type) ->
+    {?TRUE};
 
+encode_int(Num, key) when is_number(Num) ->
+    {?NUMBER, float(Num)};
 
-encode_binary(Val) ->
-    % TODO add sort strings
-    {?STRING, Val}.
+encode_int(Num, value) when is_number(Num) ->
+    {?NUMBER, Num};
 
+encode_int(Bin, key) when is_binary(Bin) ->
+    {?STRING, couch_util:get_sort_key(Bin)};
 
-encode_list(List) ->
-    EncodedItems = lists:map(fun encode_int/1, List),
-    {?LIST, list_to_tuple(EncodedItems)}.
+encode_int(Bin, value) when is_bianry(Bin) ->
+    {?STRING, Bin};
 
+encode_int(List, Type) when is_list(List) ->
+    Encoded = lists:map(fun(Item) ->
+        encode_int(Item, Type)
+    end, List),
+    {?LIST, list_to_tuple(Encoded)};
 
-encode_object({Props}) ->
-    EncodedProps = lists:map(fun({K, V}) -> 
-        EncodedK = encode_int(K),
-        EncodedV = encode_int(V),
-        {EncodedK, EncodedV}
+encode_int({Props}, Type) when is_list(Props) ->
+    Encoded = lists:map(fun({K, V}) ->
+        EK = encode_int(K, Type),
+        EV = encode_int(V, Type),
+        {EK, EV}
     end, Props),
     {?OBJECT, list_to_tuple(EncodedProps)}.
 
@@ -87,20 +87,17 @@ decode_int({?FALSE}) ->
 decode_int({?TRUE}) ->
     true;
 
-decode_int({?STRING, String}) ->
-    String;
+decode_int({?STRING, Bin}) ->
+    Bin;
 
-decode_int({?NUMBER, Number}) ->
-    case Number - trunc(Number) of
-        0 -> trunc(Number); % convert to integer
-        _ -> Number
-    end;
+decode_int({?NUMBER, Num}) ->
+    Num;
 
 decode_int({?LIST, List}) ->
     lists:map(fun decode_int/1, tuple_to_list(List));
 
 decode_int({?OBJECT, Object}) ->
-    Props = lists:map(fun({EncodedK, EncodedV}) ->
+    Props = lists:map(fun({EK, EV}) ->
         K = decode_int(EncodedK),
         V = decode_int(EncodedV),
         {K, V}


[couchdb] 03/25: Fix formatting of couch_views.app.src

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

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

commit 1c3184f37d4479e8cd7c20c98f6f674f0ef9ef9e
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 16 12:11:32 2019 -0500

    Fix formatting of couch_views.app.src
---
 src/couch_views/src/couch_views.app.src | 38 ++++++++++++++++-----------------
 1 file changed, 19 insertions(+), 19 deletions(-)

diff --git a/src/couch_views/src/couch_views.app.src b/src/couch_views/src/couch_views.app.src
index 9e1bbe7..667ede8 100644
--- a/src/couch_views/src/couch_views.app.src
+++ b/src/couch_views/src/couch_views.app.src
@@ -10,22 +10,22 @@
 % License for the specific language governing permissions and limitations under
 % the License.
 
-{application, couch_views,
- [{description, "CouchDB Views on FDB"},
-  {vsn, git},
-  {mod, {couch_views_app, []}},
-  {registered, [
-    couch_views_sup,
-    couch_views_worker_server
-  ]},
-  {applications, [
-    kernel,
-    stdlib,
-    erlfdb,
-    couch_log,
-    config,
-    couch_stats,
-    fabric,
-    couch_jobs
-   ]}
- ]}.
+{application, couch_views, [
+    {description, "CouchDB Views on FDB"},
+    {vsn, git},
+    {mod, {couch_views_app, []}},
+    {registered, [
+        couch_views_sup,
+        couch_views_worker_server
+    ]},
+    {applications, [
+        kernel,
+        stdlib,
+        erlfdb,
+        couch_log,
+        config,
+        couch_stats,
+        fabric,
+        couch_jobs
+    ]}
+]}.


[couchdb] 15/25: Fix compiler errors

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

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

commit 4dbf745b913b0879d8ca5c02cd9e72a2a0c08ca6
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Thu Jul 18 15:05:00 2019 -0500

    Fix compiler errors
---
 src/couch_views/src/couch_views.erl          | 18 +++---
 src/couch_views/src/couch_views_encoding.erl | 10 +--
 src/couch_views/src/couch_views_fdb.erl      | 81 ++++++++++++-----------
 src/couch_views/src/couch_views_indexer.erl  | 97 ++++++++++++----------------
 src/couch_views/src/couch_views_jobs.erl     | 29 +++++----
 src/couch_views/src/couch_views_reader.erl   | 19 +++---
 src/couch_views/src/couch_views_server.erl   | 30 +--------
 src/couch_views/src/couch_views_util.erl     |  1 +
 8 files changed, 127 insertions(+), 158 deletions(-)

diff --git a/src/couch_views/src/couch_views.erl b/src/couch_views/src/couch_views.erl
index 7deb54d..e10675b 100644
--- a/src/couch_views/src/couch_views.erl
+++ b/src/couch_views/src/couch_views.erl
@@ -31,13 +31,12 @@ query(Db, DDoc, ViewName, Callback, Acc0, QueryArgs0) ->
     {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
 
     #mrst{
-        views = Views,
-        language = Lang
+        views = Views
     } = Mrst,
 
+    View = get_view(ViewName, Views),
     QueryArgs1 = couch_mrview_util:set_view_type(QueryArgs0, View, Views),
     QueryArgs2 = couch_mrview_util:validate_args(QueryArgs1),
-    VInfo = couch_mrview_util:extract_view(Lang, QueryArgs2, View, Views),
     case is_reduce_view(QueryArgs2) of
         true -> throw({not_implemented});
         false -> ok
@@ -72,15 +71,18 @@ maybe_update_view(Db, Mrst, _Args) ->
     end),
 
     if WaitSeq == ready -> ok; true ->
-        couch_views_jobs:build_view(Db, Mrst, DbSeq)
+        couch_views_jobs:build_view(Db, Mrst, WaitSeq)
     end.
 
 
+get_view(ViewName, Views) ->
+    {value, View} = lists:search(fun(View) ->
+        lists:member(ViewName, View#mrview.map_names)
+    end, Views),
+    View.
+
+
 is_reduce_view(#mrargs{view_type = ViewType}) ->
     ViewType =:= red;
 is_reduce_view({Reduce, _, _}) ->
     Reduce =:= red.
-
-
-remove_ununsed_values(Args) ->
-    maps:filter(fun (_, V) -> V /= undefined end, Args).
diff --git a/src/couch_views/src/couch_views_encoding.erl b/src/couch_views/src/couch_views_encoding.erl
index 9d3e3fc..9f76ea6 100644
--- a/src/couch_views/src/couch_views_encoding.erl
+++ b/src/couch_views/src/couch_views_encoding.erl
@@ -34,7 +34,7 @@ encode(X) ->
 
 
 encode(X, Type) when Type == key; Type == value ->
-    erlfdb_tuple:pack(encode_int(X, value))
+    erlfdb_tuple:pack(encode_int(X, value)).
 
 
 decode(Encoded) ->
@@ -60,7 +60,7 @@ encode_int(Num, value) when is_number(Num) ->
 encode_int(Bin, key) when is_binary(Bin) ->
     {?STRING, couch_util:get_sort_key(Bin)};
 
-encode_int(Bin, value) when is_bianry(Bin) ->
+encode_int(Bin, value) when is_binary(Bin) ->
     {?STRING, Bin};
 
 encode_int(List, Type) when is_list(List) ->
@@ -75,7 +75,7 @@ encode_int({Props}, Type) when is_list(Props) ->
         EV = encode_int(V, Type),
         {EK, EV}
     end, Props),
-    {?OBJECT, list_to_tuple(EncodedProps)}.
+    {?OBJECT, list_to_tuple(Encoded)}.
 
 
 decode_int({?NULL}) ->
@@ -98,8 +98,8 @@ decode_int({?LIST, List}) ->
 
 decode_int({?OBJECT, Object}) ->
     Props = lists:map(fun({EK, EV}) ->
-        K = decode_int(EncodedK),
-        V = decode_int(EncodedV),
+        K = decode_int(EK),
+        V = decode_int(EV),
         {K, V}
     end, tuple_to_list(Object)),
     {Props}.
diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
index 57ed5f1..dc1840d 100644
--- a/src/couch_views/src/couch_views_fdb.erl
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -16,7 +16,7 @@
     get_update_seq/2,
     set_update_seq/3,
 
-    fold_map_idx/5,
+    fold_map_idx/6,
 
     write_doc/4
 ]).
@@ -27,6 +27,7 @@
 -define(VALUE, 2).
 
 
+-include_lib("couch_mrview/include/couch_mrview.hrl").
 -include_lib("fabric/src/fabric2.hrl").
 -include("couch_views.hrl").
 
@@ -40,25 +41,22 @@ get_update_seq(TxDb, #mrst{sig = Sig}) ->
         db_prefix := DbPrefix
     } = TxDb,
 
-    Key = get_seq_key(Sig, DbPrefix),
-    case erlfdb:wait(erlfdb:get(Tx, Key)) of
+    case erlfdb:wait(erlfdb:get(Tx, seq_key(DbPrefix, Sig))) of
         not_found -> <<>>;
         UpdateSeq -> UpdateSeq
     end.
 
 
-set_view_seq(TxDb, Sig, Seq) ->
+set_update_seq(TxDb, Sig, Seq) ->
     #{
-        tx := Tx
-        db_prefix := DbPrefix,
+        tx := Tx,
+        db_prefix := DbPrefix
     } = TxDb,
-    SeqKey = get_seq_key(Sig, DbPrefix),
-    ok = erlfdb:set(Tx, SeqKey, Seq).
+    ok = erlfdb:set(Tx, seq_key(DbPrefix, Sig), Seq).
 
 
 fold_map_idx(TxDb, Sig, ViewId, Options, Callback, Acc0) ->
     #{
-        tx := Tx,
         db_prefix := DbPrefix
     } = TxDb,
 
@@ -73,11 +71,11 @@ fold_map_idx(TxDb, Sig, ViewId, Options, Callback, Acc0) ->
                 docid => undefined,
                 dupe_id => undefined,
                 callback => Callback,
-                acc => Acc0,
+                acc => Acc0
             },
-            {fun fold_fwd/2, FwdAcc}
+            {fun fold_fwd/2, FwdAcc};
         rev ->
-            RevAcc #{
+            RevAcc = #{
                 prefix => MapIdxPrefix,
                 next => value,
                 value => undefined,
@@ -93,24 +91,20 @@ fold_map_idx(TxDb, Sig, ViewId, Options, Callback, Acc0) ->
     fabric2_db:fold_range(TxDb, MapIdxPrefix, Fun, Acc, Options).
 
 
-write_doc(TxDb, Sig, #{deleted := true} = Doc, ViewIds) ->
+write_doc(TxDb, Sig, #{deleted := true} = Doc, _ViewIds) ->
     #{
         id := DocId
     } = Doc,
 
     ExistingViewKeys = get_view_keys(TxDb, Sig, DocId),
 
-    sclear_id_idx(TxDb, Sig, DocId),
+    clear_id_idx(TxDb, Sig, DocId),
     lists:foreach(fun({ViewId, ViewKeys}) ->
-        clear_map_idx(TxDb, Sig, ViewId, ViewKeys)
+        clear_map_idx(TxDb, Sig, ViewId, DocId, ViewKeys)
     end, ExistingViewKeys);
 
 write_doc(TxDb, Sig, Doc, ViewIds) ->
     #{
-        db_prefix := DbPrefix
-    } = TxDb,
-
-    #{
         id := DocId,
         results := Results
     } = Doc,
@@ -122,7 +116,7 @@ write_doc(TxDb, Sig, Doc, ViewIds) ->
     lists:foreach(fun({ViewId, NewRows}) ->
         ExistingKeys = fabric2_util:get_value(ViewId, ExistingViewKeys, []),
         update_id_idx(TxDb, Sig, ViewId, DocId, NewRows),
-        update_map_idx(TxDb, Sig, ViewId, DocId, ExitingKeys, NewRows)
+        update_map_idx(TxDb, Sig, ViewId, DocId, ExistingKeys, NewRows)
     end, lists:zip(ViewIds, Results)).
 
 
@@ -131,7 +125,7 @@ fold_fwd({RowKey, EncodedOriginalKey}, #{next := key} = Acc) ->
         prefix := Prefix
     } = Acc,
 
-    {{SortKey, DocId}, _DupeId, ?VIEW_ROW_KEY} =
+    {{SortKey, DocId}, DupeId, ?VIEW_ROW_KEY} =
             erlfdb_tuple:unpack(RowKey, Prefix),
     Acc#{
         next := val,
@@ -147,21 +141,21 @@ fold_fwd({RowKey, EncodedValue}, #{next := val} = Acc) ->
         key := Key,
         sort_key := SortKey,
         docid := DocId,
-        dupe_id := DocId,
-        callback := UserCallback
+        dupe_id := DupeId,
+        callback := UserCallback,
         acc := UserAcc0
     } = Acc,
 
     % We're asserting there that this row is paired
     % correctly with the previous row by relying on
     % a badmatch if any of these values don't match.
-    {{SortKey, DocId}, DupeId, ?VIEW_ROW_VAL} =
+    {{SortKey, DocId}, DupeId, ?VIEW_ROW_VALUE} =
             erlfdb_tuple:unpack(RowKey, Prefix),
 
     Value = couch_views_encoding:decode(EncodedValue),
-    NewAcc = UserCallback(DocId, Key, Value, UserAcc0),
+    UserAcc1 = UserCallback(DocId, Key, Value, UserAcc0),
 
-    #{
+    Acc#{
         next := key,
         key := undefined,
         sort_key := undefined,
@@ -176,7 +170,7 @@ fold_rev({RowKey, EncodedValue}, #{next := value} = Acc) ->
         prefix := Prefix
     } = Acc,
 
-    {{SortKey, DocId}, _DupeId, ?VIEW_ROW_VAL} =
+    {{SortKey, DocId}, DupeId, ?VIEW_ROW_VALUE} =
             erlfdb_tuple:unpack(RowKey, Prefix),
     Acc#{
         next := key,
@@ -192,8 +186,8 @@ fold_rev({RowKey, EncodedOriginalKey}, #{next := key} = Acc) ->
         value := Value,
         sort_key := SortKey,
         docid := DocId,
-        dupe_id := DocId,
-        callback := UserCallback
+        dupe_id := DupeId,
+        callback := UserCallback,
         acc := UserAcc0
     } = Acc,
 
@@ -204,9 +198,9 @@ fold_rev({RowKey, EncodedOriginalKey}, #{next := key} = Acc) ->
             erlfdb_tuple:unpack(RowKey, Prefix),
 
     Key = couch_views_encoding:decode(EncodedOriginalKey),
-    NewAcc = UserCallback(DocId, Key, Value, UserAcc0),
+    UserAcc1 = UserCallback(DocId, Key, Value, UserAcc0),
 
-    #{
+    Acc#{
         next := val,
         value := undefined,
         sort_key := undefined,
@@ -222,10 +216,10 @@ clear_id_idx(TxDb, Sig, DocId) ->
     } = TxDb,
 
     {Start, End} = id_idx_range(DbPrefix, Sig, DocId),
-    ok = erlfdb:clear_range(Start, End).
+    ok = erlfdb:clear_range(Tx, Start, End).
 
 
-clear_map_idx(TxDb, Sig, ViewId, ViewKeys) ->
+clear_map_idx(TxDb, Sig, ViewId, DocId, ViewKeys) ->
     #{
         tx := Tx,
         db_prefix := DbPrefix
@@ -269,7 +263,7 @@ update_map_idx(TxDb, Sig, ViewId, DocId, ExistingKeys, NewRows) ->
 
     lists:foreach(fun({DupeId, Key1, Key2, Val}) ->
         KK = map_idx_key(MapIdxPrefix, {Key1, DocId}, DupeId, ?VIEW_ROW_KEY),
-        VK = map_idx_key(MapIdxPrefix, {Key1, DocId}, DupeId, ?VIEW_ROW_VAL),
+        VK = map_idx_key(MapIdxPrefix, {Key1, DocId}, DupeId, ?VIEW_ROW_VALUE),
         ok = erlfdn:store(Tx, KK, Key2),
         ok = erlfdb:store(Tx, VK, Val)
     end, KVsToAdd).
@@ -280,15 +274,20 @@ get_view_keys(TxDb, Sig, DocId) ->
         tx := Tx,
         db_prefix := DbPrefix
     } = TxDb,
-    {Start, End} = id_idx_range(DbPrefix, Sig, DocId)
+    {Start, End} = id_idx_range(DbPrefix, Sig, DocId),
     lists:map(fun({K, V}) ->
         {?DB_VIEWS, Sig, ?VIEW_ID_RANGE, DocId, ViewId} =
                 erlfdb_tuple:unpack(K, DbPrefix),
-        ViewKeys = couch_views_encoding:decode(V)
+        ViewKeys = couch_views_encoding:decode(V),
         {ViewId, ViewKeys}
     end, erlfdb:get_range(Tx, Start, End, [])).
 
 
+seq_key(DbPrefix, Sig) ->
+    Key = {?DB_VIEWS, Sig, ?VIEW_UPDATE_SEQ},
+    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).
@@ -300,12 +299,12 @@ id_idx_range(DbPrefix, Sig, DocId) ->
 
 
 map_idx_prefix(DbPrefix, Sig, ViewId) ->
-    Key = {?DB_VIES, Sig, ?VIEW_MAP_RANGE, ViewId},
-    erlfdb_tuple:pack(Key).
+    Key = {?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, ViewId},
+    erlfdb_tuple:pack(Key, DbPrefix).
 
 
-map_idx_key(MapIdxPrefix, MapKey, DocId, DupeId, Type)
-    Key = {MapKey, DocId, DupeId, Type},
+map_idx_key(MapIdxPrefix, MapKey, DupeId, Type) ->
+    Key = {MapKey, DupeId, Type},
     erldb_tuple:encode(Key, MapIdxPrefix).
 
 
@@ -320,7 +319,7 @@ process_rows(Rows) ->
         EK1 = couch_views_encoding:encode(K, key),
         EK2 = couch_views_encoding:encode(K, value),
         EV = couch_views_encoding:encode(V, value),
-        {EKK, EKV, EV}
+        {EK1, EK2, EV}
     end, Rows),
 
     Grouped = lists:foldl(fun({K1, K2, V}, Acc) ->
diff --git a/src/couch_views/src/couch_views_indexer.erl b/src/couch_views/src/couch_views_indexer.erl
index 1a84116..91072a1 100644
--- a/src/couch_views/src/couch_views_indexer.erl
+++ b/src/couch_views/src/couch_views_indexer.erl
@@ -21,10 +21,10 @@
     init/0
 ]).
 
--include_lib("couch_views/include/couch_views.hrl").
 -include_lib("couch/include/couch_db.hrl").
+-include_lib("couch_mrview/include/couch_mrview.hrl").
 -include_lib("fabric/src/fabric2.hrl").
--include_lib("eunit/include/eunit.hrl").
+-include("couch_views.hrl").
 
 % TODO:
 %  * Handle timeouts of transaction and other errors
@@ -57,6 +57,8 @@ init() ->
         db_seq => undefined,
         view_seq => undefined,
         last_seq => undefined,
+        job => Job,
+        job_data => Data,
         count => 0,
         limit => num_changes(),
         doc_acc => [],
@@ -67,7 +69,7 @@ init() ->
 
 
 update(#{} = Db, Mrst0, State0) ->
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
+    {Mrst2, State3} = fabric2_fdb:transactional(Db, fun(TxDb) ->
         % In the first iteration of update we need
         % to populate our db and view sequences
         State1 = case State0 of
@@ -75,7 +77,7 @@ update(#{} = Db, Mrst0, State0) ->
                 State0#{
                     tx_db := TxDb,
                     db_seq := fabric2_db:get_update_seq(TxDb),
-                    view_seq := couch_views_fdb:get_update_seq(TxDb, Mrst)
+                    view_seq := couch_views_fdb:get_update_seq(TxDb, Mrst0)
                 };
             _ ->
                 State0#{
@@ -93,53 +95,29 @@ update(#{} = Db, Mrst0, State0) ->
         } = State2,
 
         {Mrst1, MappedDocs} = map_docs(Mrst0, DocAcc),
-        write_docs(Db, Mrst1, MappedResults, State2),
+        write_docs(Db, Mrst1, MappedDocs, State2),
 
         case Count < Limit of
             true ->
-                report_progress(State2, finished);
+                report_progress(State2, finished),
+                {Mrst1, finished};
             false ->
                 report_progress(State2, update),
-                State3 = maps:merge(FinalState, #{
-                    count => 0,
-                    doc_acc => [],
-                    db_seq => LastSeq,
-                    last_seq => 0,
-                    mrst => Mrst1
-                }),
-
-    end).
-
-
-update_int(#{} = Db, State) ->
-
-
-    #{
-        count := Count,
-        limit := Limit,
-        doc_acc := DocAcc,
-        last_seq := LastSeq,
-        callback := Cb,
-        callback_args := CallbackArgs,
-        mrst := Mrst
-    } = FinalState,
-
-    {MappedDocs, Mrst1} = map_docs(Mrst, DocAcc),
-    write_docs(Db, Mrst1, MappedDocs, FinalState),
-
-    case Count < Limit of
-        true ->
-            Cb(undefined, finished, CallbackArgs, Db, Mrst, LastSeq);
-        false ->
-            NextState = maps:merge(FinalState, #{
-                limit => Limit,
-                count => 0,
-                doc_acc => [],
-                since_seq => LastSeq,
-                last_seq => 0,
-                mrst => Mrst1
-            }),
-            update_int(Db, NextState)
+                {Mrst1, State2#{
+                    tx_db := undefined,
+                    count := 0,
+                    doc_acc := [],
+                    view_seq := LastSeq,
+                    last_seq := undefined
+                }}
+        end
+    end),
+
+    case State3 of
+        finished ->
+            couch_query_servers:stop_doc_map(Mrst2#mrst.qserver);
+        _ ->
+            update(Db, Mrst2, State3)
     end.
 
 
@@ -172,7 +150,6 @@ process_changes(Change, Acc) ->
 
     Acc1 = case {Id, IncludeDesign} of
         {<<"_design/", _/binary>>, false} ->
-            % {ok, Doc} = fabric2_db:open_doc(Db, Id),
             maps:merge(Acc, #{
                 count => Count + 1,
                 last_seq => LastSeq
@@ -186,11 +163,11 @@ process_changes(Change, Acc) ->
             end,
 
             Change1 = maps:put(doc, Doc, Change),
-            maps:merge(Acc, #{
-                doc_acc => DocAcc ++ [Change1],
-                count => Count + 1,
-                last_seq => LastSeq
-            })
+            Acc#{
+                doc_acc := DocAcc ++ [Change1],
+                count := Count + 1,
+                last_seq := LastSeq
+            }
     end,
     {ok, Acc1}.
 
@@ -247,10 +224,22 @@ start_query_server(#mrst{} = Mrst) ->
 
 
 report_progress(State, UpdateType) ->
+    #{
+        tx_db := TxDb,
+        job := Job,
+        job_data := JobData,
+        last_seq := LastSeq
+    } = State,
+
+    NewData = JobData#{view_seq => LastSeq},
+
     case UpdateType of
         update ->
-            couch_views_jobs:update(Tx, Job, Db, Mrst, LastSeq);
+            couch_jobs:update(TxDb, Job, NewData);
         finished ->
-            couch_views_jobs:finish(Tx, Job, Db, Mrst, LastSeq)
+            couch_jobs:finish(TxDb, Job, NewData)
     end.
 
+
+num_changes() ->
+    config:get_integer("couch_views", "change_limit", 100).
diff --git a/src/couch_views/src/couch_views_jobs.erl b/src/couch_views/src/couch_views_jobs.erl
index 9714b29..9e299af 100644
--- a/src/couch_views/src/couch_views_jobs.erl
+++ b/src/couch_views/src/couch_views_jobs.erl
@@ -19,6 +19,7 @@
 ]).
 
 
+-include_lib("couch_mrview/include/couch_mrview.hrl").
 -include("couch_views.hrl").
 
 
@@ -26,18 +27,18 @@ set_timeout() ->
     couch_jobs:set_type_timeout(?INDEX_JOB_TYPE, 6 * 1000).
 
 
-build_view(Db, Mrst, UpdateSeq) ->
-    {ok, JobId} = build_view_async(Db, Mrst),
+build_view(TxDb, Mrst, UpdateSeq) ->
+    {ok, JobId} = build_view_async(TxDb, Mrst),
     case wait_for_job(JobId, UpdateSeq) of
         ok -> ok;
-        retry -> build_view(Db, Mrst, UpdateSeq)
+        retry -> build_view(TxDb, Mrst, UpdateSeq)
     end.
 
 
-build_view_async(Db, Mrst) ->
-    JobId = create_job_id(TxDb, Mrst),
-    JobData = create_job_data(TxDb, Mrst),
-    ok = couch_jobs:add(undefined, ?INDEX_JOB_TYPE, JobId, JobData).
+build_view_async(TxDb, Mrst) ->
+    JobId = job_id(TxDb, Mrst),
+    JobData = job_data(TxDb, Mrst),
+    ok = couch_jobs:add(undefined, ?INDEX_JOB_TYPE, JobId, JobData),
     {ok, JobId}.
 
 
@@ -45,7 +46,7 @@ build_view_async(Db, Mrst) ->
 wait_for_job(JobId, UpdateSeq) ->
     case couch_jobs:subscribe(?INDEX_JOB_TYPE, JobId) of
         {ok, Subscription, _State, _Data} ->
-            wait_for_job(JobId, Subscription, UpdateSeq)
+            wait_for_job(JobId, Subscription, UpdateSeq);
         {ok, finished, Data} ->
             case Data of
                 #{view_seq := ViewSeq} when ViewSeq >= UpdateSeq ->
@@ -57,7 +58,7 @@ wait_for_job(JobId, UpdateSeq) ->
 
 
 wait_for_job(JobId, Subscription, UpdateSeq) ->
-    case wait(Subscription, infinity) of
+    case wait(Subscription) of
         {error, Error} ->
             erlang:error(Error);
         {finished, #{view_seq := ViewSeq}} when ViewSeq >= UpdateSeq ->
@@ -72,14 +73,14 @@ wait_for_job(JobId, Subscription, UpdateSeq) ->
     end.
 
 
-get_id(#{name := DbName}, #mrst{sig = Sig}) ->
-    create_job_id(DbName, Sig);
+job_id(#{name := DbName}, #mrst{sig = Sig}) ->
+    job_id(DbName, Sig);
 
-get_id(DbName, Sig) ->
+job_id(DbName, Sig) ->
     <<DbName/binary, Sig/binary>>.
 
 
-create_job_data(Db, Mrst) ->
+job_data(Db, Mrst) ->
     #mrst{
         idx_name = DDocId,
         sig = Sig
@@ -93,7 +94,7 @@ create_job_data(Db, Mrst) ->
 
 
 wait(Subscription) ->
-    case couch_jobs:wait(JobSubscription, infinity) of
+    case couch_jobs:wait(Subscription, infinity) of
         {?INDEX_JOB_TYPE, _JobId, JobState, JobData} -> {JobState, JobData};
         timeout -> {error, timeout}
     end.
diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index 56b23f2..8d2bf5a 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -19,6 +19,7 @@
 
 -include("couch_views.hrl").
 -include_lib("couch/include/couch_db.hrl").
+-include_lib("couch_mrview/include/couch_mrview.hrl").
 -include_lib("fabric/src/fabric2.hrl").
 
 
@@ -39,7 +40,7 @@ read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
 
         fabric2_fdb:transactional(Db, fun(TxDb) ->
             Acc0 = #{
-                db => TxDb
+                db => TxDb,
                 skip => Args#mrargs.skip,
                 mrargs => Args,
                 callback => UserCallback,
@@ -59,7 +60,7 @@ read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
                 acc := UserAcc2
             } = Acc1,
 
-            maybe_stop(Callback(complete, UserAcc2)
+            maybe_stop(UserCallback(complete, UserAcc2))
         end)
     catch throw:{done, Out} ->
         {ok, Out}
@@ -71,10 +72,10 @@ handle_row(_DocId, _Key, _Value, #{skip := Skip} = Acc) when Skip > 0 ->
 
 handle_row(DocId, Key, Value, Acc) ->
     #{
-        db := TxDb
+        db := TxDb,
         mrargs := Args,
         callback := UserCallback,
-        acc := UserAcc
+        acc := UserAcc0
     } = Acc,
 
     BaseRow = [
@@ -83,21 +84,21 @@ handle_row(DocId, Key, Value, Acc) ->
         {value, Value}
     ],
 
-    Row = BaseRow ++ if not IncludeDocs -> []; true ->
+    Row = BaseRow ++ if not Args#mrargs.include_docs -> []; true ->
         DocOpts0 = Args#mrargs.doc_options,
-        DocOpts1 = OpenOpts0 ++ case Args#mrargs.conflicts of
+        DocOpts1 = DocOpts0 ++ case Args#mrargs.conflicts of
             true -> [conflicts];
             false -> []
         end,
-        DocObj = case fabric2_db:open_doc(Db, DocId, DocOpts1) of
+        DocObj = case fabric2_db:open_doc(TxDb, DocId, DocOpts1) of
             {ok, Doc} -> couch_doc:to_json_obj(Doc, DocOpts1);
             {not_found, _} -> null
         end,
         [{doc, DocObj}]
     end,
 
-    UserAcc1 = maybe_stop(Callback({row, Row1}, UserAcc0)),
-    Acc#{acc := UserAcc1}
+    UserAcc1 = maybe_stop(UserCallback({row, Row}, UserAcc0)),
+    Acc#{acc := UserAcc1}.
 
 
 get_view_id(ViewName, Views) ->
diff --git a/src/couch_views/src/couch_views_server.erl b/src/couch_views/src/couch_views_server.erl
index 8ec2425..0417a9b 100644
--- a/src/couch_views/src/couch_views_server.erl
+++ b/src/couch_views/src/couch_views_server.erl
@@ -41,7 +41,7 @@ start_link() ->
 init(_) ->
     process_flag(trap_exit, true),
     couch_views_jobs:set_timeout(),
-    State0 = #{
+    State = #{
         workers => [],
         num_workers => num_workers()
     },
@@ -70,7 +70,7 @@ handle_info({'EXIT', Pid, Reason}, State) ->
         NewWorkers ->
             if Reason == normal -> ok; true ->
                 LogMsg = "~p : indexer process ~p exited with ~p",
-                couch_log:error(LogMsg, [?MODULE, Pid, Reason]),
+                couch_log:error(LogMsg, [?MODULE, Pid, Reason])
             end,
             {noreply, spawn_workers(State#{workers := NewWorkers})}
     end;
@@ -90,36 +90,12 @@ spawn_workers(State) ->
     } = State,
     case length(Workers) < NumWorkers of
         true ->
-            Pid = spawn_worker(),
+            Pid = couch_views_indexer:spawn_link(),
             spawn_workers(State#{workers := [Pid | Workers]});
         false ->
             State
     end.
 
 
-spawn_worker() ->
-    couch_views_indexer:spawn_link().
-
-
-blocking_acceptor(Parent) ->
-    case couch_views_jobs:accept() of
-        not_found ->
-            blocking_acceptor(Parent);
-        {ok, Job, JobData} ->
-            gen_server:cast(Parent, {job, Job, JobData})
-    end.
-
-
-check_finished_process(#{acceptor_pid := Pid} = State, Pid) ->
-    State1 = State#{acceptor_pid := undefined},
-    spawn_acceptor(State1);
-
-check_finished_process(State, Pid) ->
-    #{workers := Workers} = State,
-    Workers1 = maps:remove(Pid, Workers),
-    State#{workers := Workers1}.
-
-
-
 num_workers() ->
     config:get_integer("couch_views", "max_workers", ?MAX_WORKERS).
diff --git a/src/couch_views/src/couch_views_util.erl b/src/couch_views/src/couch_views_util.erl
index d7ed29f..b88cfcd 100644
--- a/src/couch_views/src/couch_views_util.erl
+++ b/src/couch_views/src/couch_views_util.erl
@@ -19,6 +19,7 @@
 
 
 -include_lib("couch/include/couch_db.hrl").
+-include_lib("couch_mrview/include/couch_mrview.hrl").
 -include("couch_views.hrl").
 
 


[couchdb] 04/25: Expose the is_replicator_db and is_user_db logic

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

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

commit bb329abb5cf793b9ef4504b9032a566503f55f75
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 16 14:26:11 2019 -0500

    Expose the is_replicator_db and is_user_db logic
    
    This exposes a single place where we can check for whether a given
    database or database name is a replicator or users database.
---
 src/fabric/src/fabric2_db.erl   | 37 +++++++++++++++++++++++++++----------
 src/fabric/src/fabric2_util.erl |  6 ++++--
 2 files changed, 31 insertions(+), 12 deletions(-)

diff --git a/src/fabric/src/fabric2_db.erl b/src/fabric/src/fabric2_db.erl
index 3c3b7d3..a89b098 100644
--- a/src/fabric/src/fabric2_db.erl
+++ b/src/fabric/src/fabric2_db.erl
@@ -55,6 +55,8 @@
     is_partitioned/1,
     is_system_db/1,
     is_system_db_name/1,
+    is_replicator_db/1,
+    is_users_db/1,
 
     set_revs_limit/2,
     %% set_purge_infos_limit/2,
@@ -379,6 +381,29 @@ is_system_db_name(DbName) when is_binary(DbName) ->
     end.
 
 
+is_replicator_db(#{name := DbName}) ->
+    is_replicator_db(DbName);
+
+is_replicator_db(DbName) when is_binary(DbName) ->
+    fabric2_util:dbname_ends_with(Db, <<"_replicator">>).
+
+
+is_users_db(#{name := DbName}) ->
+    is_users_db(DbName);
+
+is_users_db(DbName) when is_binary(DbName) ->
+    AuthenticationDb = config:get("chttpd_auth", "authentication_db"),
+    CfgUsersSuffix = config:get("couchdb", "users_db_suffix", "_users"),
+
+    IsAuthCache = if AuthenticationDb == undefined -> false; true ->
+        DbName == ?l2b(AuthenticationDb)
+    end,
+    IsCfgUsersDb = fabric2_util:dbname_ends_with(DbName, ?l2b(CfgUsersSuffix)),
+    IsGlobalUsersDb = fabric2_util:dbname_ends_with(DbName, <<"_users">>),
+
+    IsAuthCache orelse IsCfgUsersDb orelse IsGlobalUsersDb.
+
+
 set_revs_limit(#{} = Db, RevsLimit) ->
     check_is_admin(Db),
     RevsLimBin = ?uint2bin(RevsLimit),
@@ -734,16 +759,8 @@ fold_changes(Db, SinceSeq, UserFun, UserAcc, Options) ->
 
 
 maybe_add_sys_db_callbacks(Db) ->
-    IsReplicatorDb = fabric2_util:dbname_ends_with(Db, <<"_replicator">>),
-
-    AuthenticationDb = config:get("chttpd_auth", "authentication_db"),
-    IsAuthCache = if AuthenticationDb == undefined -> false; true ->
-        name(Db) == ?l2b(AuthenticationDb)
-    end,
-    CfgUsersSuffix = config:get("couchdb", "users_db_suffix", "_users"),
-    IsCfgUsersDb = fabric2_util:dbname_ends_with(Db, ?l2b(CfgUsersSuffix)),
-    IsGlobalUsersDb = fabric2_util:dbname_ends_with(Db, <<"_users">>),
-    IsUsersDb = IsAuthCache orelse IsCfgUsersDb orelse IsGlobalUsersDb,
+    IsReplicatorDb = is_replicator_db(Db),
+    IsUsersDb = is_users_db(Db),
 
     {BDU, ADR} = if
         IsReplicatorDb ->
diff --git a/src/fabric/src/fabric2_util.erl b/src/fabric/src/fabric2_util.erl
index 48bf7d1..1921bca 100644
--- a/src/fabric/src/fabric2_util.erl
+++ b/src/fabric/src/fabric2_util.erl
@@ -124,8 +124,10 @@ validate_json_list_of_strings(Member, Props) ->
     end.
 
 
-dbname_ends_with(#{} = Db, Suffix) when is_binary(Suffix) ->
-    DbName = fabric2_db:name(Db),
+dbname_ends_with(#{} = Db, Suffix) ->
+    dbname_ends_with(fabric2_db:name(Db), Suffix).
+
+dbname_ends_with(DbName, Suffix) when is_binary(DbName), is_binary(Suffix) ->
     Suffix == filename:basename(DbName).
 
 


[couchdb] 09/25: Include the view sig in the job

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

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

commit a4be1fb4f9bde8f04abe356578fa1005d1634369
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Wed Jul 17 10:42:16 2019 -0500

    Include the view sig in the job
    
    This saves us from having to try and parse out the sig from the job id.
    We'll need this to assert that we're building the right version of the
    view since the ddoc could be updated between the view job creation and
    when the job actually starts.
---
 src/couch_views/src/couch_views_jobs.erl | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/src/couch_views/src/couch_views_jobs.erl b/src/couch_views/src/couch_views_jobs.erl
index d9c5157..9714b29 100644
--- a/src/couch_views/src/couch_views_jobs.erl
+++ b/src/couch_views/src/couch_views_jobs.erl
@@ -81,12 +81,14 @@ get_id(DbName, Sig) ->
 
 create_job_data(Db, Mrst) ->
     #mrst{
-        idx_name = DDocId
+        idx_name = DDocId,
+        sig = Sig
     } = Mrst,
 
     #{
         db_name => fabric2_db:name(Db),
-        ddoc_id => DDocId
+        ddoc_id => DDocId,
+        sig => Sig
     }.
 
 


[couchdb] 10/25: Simplify worker vs. indexer distinction

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

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

commit 14321badcfae21f428e63024b546f671f07b9355
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Wed Jul 17 13:26:31 2019 -0500

    Simplify worker vs. indexer distinction
    
    This just turns an indexer into a job handler and removes the worker
    concept entirely. couch_views_server just starts `max_workers` indexer
    process that each wait for a job to process. Once processing is finished
    the worker exits and couch_views_server spawns a new indexer to replace
    it.
---
 src/couch_views/src/couch_views_indexer.erl | 270 ++++++++++++++--------------
 src/couch_views/src/couch_views_worker.erl  |  44 -----
 2 files changed, 132 insertions(+), 182 deletions(-)

diff --git a/src/couch_views/src/couch_views_indexer.erl b/src/couch_views/src/couch_views_indexer.erl
index e9f0b41..1a84116 100644
--- a/src/couch_views/src/couch_views_indexer.erl
+++ b/src/couch_views/src/couch_views_indexer.erl
@@ -13,55 +13,106 @@
 -module(couch_views_indexer).
 
 -export([
-    update/2,
-    update/4,
-
-    % For tests
-    map_docs/2,
-    write_doc/4
+    spawn_link/0
 ]).
 
 
--include("couch_views.hrl").
+-export([
+    init/0
+]).
+
+-include_lib("couch_views/include/couch_views.hrl").
 -include_lib("couch/include/couch_db.hrl").
 -include_lib("fabric/src/fabric2.hrl").
 -include_lib("eunit/include/eunit.hrl").
 
-% TODO: 
+% TODO:
 %  * Handle timeouts of transaction and other errors
 
-update(Db, Mrst) ->
-    Noop = fun (_) -> ok end,
-    update(Db, Mrst, Noop, []).
-
-
-update(#{} = Db, Mrst, ProgressCallback, ProgressArgs)
-        when is_function(ProgressCallback, 6) ->
-    try
-        Seq = couch_views_fdb:get_update_seq(Db, Mrst),
-        State = #{
-            since_seq => Seq,
-            count => 0,
-            limit => config:get_integer("couch_views", "change_limit", 100),
-            doc_acc => [],
-            last_seq => Seq,
-            callback => ProgressCallback,
-            callback_args => ProgressArgs,
-            mrst => Mrst
-        },
-        update_int(Db, State)
-    catch error:database_does_not_exist ->
-        #{db_prefix := DbPrefix} = Db,
-        couch_log:notice("couch_views_indexer stopped"
-        "- ~p database does not exist", [DbPrefix])
-    end.
+
+spawn_link() ->
+    proc_lib:spawn_link(?MODULE, init, []).
+
+
+init() ->
+    {ok, Job, Data} = couch_jobs:accept(?INDEX_JOB_TYPE, #{}),
+
+    #{
+        <<"db_name">> := DbName,
+        <<"ddoc_id">> := DDocId,
+        <<"sig">> := Sig
+    } = Data,
+
+    {ok, Db} = fabric2_db:open(DbName, []),
+    {ok, DDoc} = fabric2_db:open_doc(Db, DDocId),
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+
+    if Mrst#mrst.sig == Sig -> ok; true ->
+        couch_jobs:finish(Job, Data#{error => mismatched_signature}),
+        exit(normal)
+    end,
+
+    State = #{
+        tx_db => undefined,
+        db_seq => undefined,
+        view_seq => undefined,
+        last_seq => undefined,
+        count => 0,
+        limit => num_changes(),
+        doc_acc => [],
+        design_opts => Mrst#mrst.design_opts
+    },
+
+    update(Db, Mrst, State).
+
+
+update(#{} = Db, Mrst0, State0) ->
+    fabric2_fdb:transactional(Db, fun(TxDb) ->
+        % In the first iteration of update we need
+        % to populate our db and view sequences
+        State1 = case State0 of
+            #{db_seq := undefined} ->
+                State0#{
+                    tx_db := TxDb,
+                    db_seq := fabric2_db:get_update_seq(TxDb),
+                    view_seq := couch_views_fdb:get_update_seq(TxDb, Mrst)
+                };
+            _ ->
+                State0#{
+                    tx_db := TxDb
+                }
+        end,
+
+        {ok, State2} = fold_changes(State1),
+
+        #{
+            count := Count,
+            limit := Limit,
+            doc_acc := DocAcc,
+            last_seq := LastSeq
+        } = State2,
+
+        {Mrst1, MappedDocs} = map_docs(Mrst0, DocAcc),
+        write_docs(Db, Mrst1, MappedResults, State2),
+
+        case Count < Limit of
+            true ->
+                report_progress(State2, finished);
+            false ->
+                report_progress(State2, update),
+                State3 = maps:merge(FinalState, #{
+                    count => 0,
+                    doc_acc => [],
+                    db_seq => LastSeq,
+                    last_seq => 0,
+                    mrst => Mrst1
+                }),
+
+    end).
 
 
 update_int(#{} = Db, State) ->
-    {ok, FinalState} = fabric2_fdb:transactional(Db, fun(TxDb) ->
-        State1 = maps:put(tx_db, TxDb, State),
-        fold_changes(State1)
-    end),
+
 
     #{
         count := Count,
@@ -73,8 +124,8 @@ update_int(#{} = Db, State) ->
         mrst := Mrst
     } = FinalState,
 
-    {MappedResults, Mrst1} = map_docs(Mrst, DocAcc),
-    write_docs(Db, Mrst1, MappedResults, FinalState),
+    {MappedDocs, Mrst1} = map_docs(Mrst, DocAcc),
+    write_docs(Db, Mrst1, MappedDocs, FinalState),
 
     case Count < Limit of
         true ->
@@ -94,13 +145,13 @@ update_int(#{} = Db, State) ->
 
 fold_changes(State) ->
     #{
-        since_seq := SinceSeq,
+        view_seq := SinceSeq,
         limit := Limit,
         tx_db := TxDb
     } = State,
 
-    fabric2_db:fold_changes(TxDb, SinceSeq,
-        fun process_changes/2, State, [{limit, Limit}]).
+    Fun = fun process_changes/2,
+    fabric2_db:fold_changes(TxDb, SinceSeq, Fun, State, [{limit, Limit}]).
 
 
 process_changes(Change, Acc) ->
@@ -108,7 +159,7 @@ process_changes(Change, Acc) ->
         doc_acc := DocAcc,
         count := Count,
         tx_db := TxDb,
-        mrst := Mrst
+        design_opts := DesignOpts
     } = Acc,
 
     #{
@@ -117,8 +168,7 @@ process_changes(Change, Acc) ->
         deleted := Deleted
     } = Change,
 
-    IncludeDesign = lists:keymember(<<"include_design">>, 1,
-        Mrst#mrst.design_opts),
+    IncludeDesign = lists:keymember(<<"include_design">>, 1, DesignOpts),
 
     Acc1 = case {Id, IncludeDesign} of
         {<<"_design/", _/binary>>, false} ->
@@ -126,16 +176,13 @@ process_changes(Change, Acc) ->
             maps:merge(Acc, #{
                 count => Count + 1,
                 last_seq => LastSeq
-                });
+            });
         _ ->
-
             % Making a note here that we should make fetching all the docs
             % a parallel fdb operation
-            Doc = if Deleted -> []; true ->
-                case fabric2_db:open_doc(TxDb, Id) of
-                    {ok, Doc0} -> Doc0;
-                    {not_found, _} -> []
-                end
+            {ok, Doc} = case Deleted of
+                true -> {ok, []};
+                false -> fabric2_db:open_doc(TxDb, Id)
             end,
 
             Change1 = maps:put(doc, Doc, Change),
@@ -150,113 +197,60 @@ process_changes(Change, Acc) ->
 
 map_docs(Mrst, Docs) ->
     % Run all the non deleted docs through the view engine and
-    Mrst1 = get_query_server(Mrst),
+    Mrst1 = start_query_server(Mrst),
     QServer = Mrst1#mrst.qserver,
-
     MapFun = fun
         (#{deleted := true} = Change) ->
             maps:put(results, [], Change);
-
         (Change) ->
             #{doc := Doc} = Change,
             couch_stats:increment_counter([couchdb, mrview, map_doc]),
             {ok, RawResults} = couch_query_servers:map_doc_raw(QServer, Doc),
             JsonResults = couch_query_servers:raw_to_ejson(RawResults),
-            ListResults = [[list_to_tuple(Res) || Res <- FunRs]
-                || FunRs <- JsonResults],
+            ListResults = lists:map(fun(ViewResults) ->
+                [list_to_tuple(Res) || Res <- ViewResults]
+            end, JsonResults),
             maps:put(results, ListResults, Change)
     end,
-    MappedResults = lists:map(MapFun, Docs),
-    {MappedResults, Mrst1}.
+    {Mrst1, lists:map(MapFun, Docs)}.
 
 
-start_query_server(#mrst{} = Mrst) ->
+write_docs(TxDb, Mrst, Docs, State) ->
     #mrst{
-        language=Language,
-        lib=Lib,
-        views=Views
+        views = Views,
+        sig = Sig
     } = Mrst,
-    Defs = [View#mrview.def || View <- Views],
-    {ok, QServer} = couch_query_servers:start_doc_map(Language, Defs, Lib),
-    Mrst#mrst{qserver=QServer}.
 
+    #{
+        last_seq := LastSeq
+    } = State,
 
-get_query_server(#mrst{} = Mrst) ->
-    case Mrst#mrst.qserver of
-        nil -> start_query_server(Mrst);
-        _ -> Mrst
-    end.
+    ViewIds = [View#mrview.id_num || View <- Views],
+
+    lists:foreach(fun(Doc) ->
+        couch_views_fdb:write_doc(TxDb, Sig, ViewIds, Doc)
+    end, Docs),
 
+    couch_views_fdb:update_view_seq(TxDb, Sig, LastSeq).
 
-write_docs(Db, Mrst, Docs, State) ->
+
+start_query_server(#mrst{} = Mrst) ->
     #mrst{
-        views = Views,
-        sig = Sig
+        language = Language,
+        lib = Lib,
+        views = Views
     } = Mrst,
+    Defs = [View#mrview.def || View <- Views],
+    {ok, QServer} = couch_query_servers:start_doc_map(Language, Defs, Lib),
+    Mrst#mrst{qserver = QServer}.
 
-    #{
-        callback := Cb,
-        callback_args := CallbackArgs
-    } = State,
 
-    IdxNames = lists:map(fun (View) ->
-        View#mrview.id_num
-    end, Views),
-
-    lists:foreach(fun (Doc) ->
-        #{sequence := Seq} = Doc,
-        fabric2_fdb:transactional(Db, fun(TxDb) ->
-            couch_views_fdb:update_view_seq(TxDb, Sig, Seq),
-            Cb(TxDb, update, CallbackArgs, Db, Mrst, Seq),
-            write_doc(TxDb, Sig, Doc, IdxNames)
-        end)
-    end, Docs).
-
-
-write_doc(TxDb, Sig, #{deleted := true} = Doc, ViewIds) ->
-    #{id := DocId} = Doc,
-    lists:foreach(fun (IdxName) ->
-        maybe_clear_id_and_map_index(TxDb, Sig, DocId, IdxName)
-    end, ViewIds);
-
-write_doc(TxDb, Sig, Doc, ViewIds) ->
-    #{id := DocId, results := Results} = Doc,
-    lists:foreach(fun
-        ({IdxName, []}) ->
-            maybe_clear_id_and_map_index(TxDb, Sig, DocId, IdxName);
-        ({IdxName, IdxResults}) ->
-            lists:foldl(fun (IdxResult, DocIdsCleared) ->
-                {IdxKey, _} = IdxResult,
-                OldIdxKey = couch_views_fdb:get_id_index(TxDb, Sig,
-                    DocId, IdxName),
-                IsAlreadyCleared = lists:member(DocId, DocIdsCleared),
-                case OldIdxKey == not_found orelse IsAlreadyCleared == true of
-                    true ->
-                        couch_views_fdb:set_id_index(TxDb, Sig, IdxName,
-                            DocId, IdxKey),
-                        couch_views_fdb:set_map_index_results(TxDb, Sig,
-                            IdxName, DocId, IdxResults);
-                    false ->
-                        couch_views_fdb:clear_id_index(TxDb, Sig,
-                            DocId, IdxName),
-                        couch_views_fdb:clear_map_index(TxDb, Sig, IdxName,
-                            DocId, OldIdxKey),
-                        couch_views_fdb:set_id_index(TxDb, Sig, DocId,
-                            IdxName, IdxKey),
-                        couch_views_fdb:set_map_index_results(TxDb, Sig,
-                            IdxName, DocId, IdxResults)
-                end,
-                [DocId | DocIdsCleared]
-            end, [], IdxResults)
-    end, lists:zip(ViewIds, Results)).
-
-
-maybe_clear_id_and_map_index(TxDb, Sig, DocId, IdxName) ->
-    OldIdxKey = couch_views_fdb:get_id_index(TxDb, Sig,
-        DocId, IdxName),
-    if OldIdxKey == not_found -> ok; true ->
-        couch_views_fdb:clear_id_index(TxDb, Sig,
-            DocId, IdxName),
-        couch_views_fdb:clear_map_index(TxDb, Sig, IdxName,
-            DocId, OldIdxKey)
+
+report_progress(State, UpdateType) ->
+    case UpdateType of
+        update ->
+            couch_views_jobs:update(Tx, Job, Db, Mrst, LastSeq);
+        finished ->
+            couch_views_jobs:finish(Tx, Job, Db, Mrst, LastSeq)
     end.
+
diff --git a/src/couch_views/src/couch_views_worker.erl b/src/couch_views/src/couch_views_worker.erl
deleted file mode 100644
index fa641d5..0000000
--- a/src/couch_views/src/couch_views_worker.erl
+++ /dev/null
@@ -1,44 +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.
-
--module(couch_views_worker).
-
--export([
-    start/2,
-    job_progress/6
-]).
-
-
-start(Job, JobData) ->
-    {ok, Db, Mrst} = get_indexing_info(JobData),
-    % maybe we should spawn here
-    couch_views_indexer:update(Db, Mrst, fun job_progress/6, Job).
-
-
-job_progress(Tx, Progress, Job, Db, Mrst, LastSeq) ->
-    case Progress of
-        update ->
-            couch_views_jobs:update(Tx, Job, Db, Mrst, LastSeq);
-        finished ->
-            couch_views_jobs:finish(Tx, Job, Db, Mrst, LastSeq)
-    end.
-
-
-get_indexing_info(JobData) ->
-    #{
-        <<"db_name">> := DbName,
-        <<"ddoc_id">> := DDocId
-    } = JobData,
-    {ok, Db} = fabric2_db:open(DbName, []),
-    {ok, DDoc} = fabric2_db:open_doc(Db, DDocId),
-    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
-    {ok, Db, Mrst}.


[couchdb] 05/25: Don't redefine records with the same name

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

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

commit bec2da3a39147c2c22b1ef640c92648ded4fe9e5
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 16 14:42:07 2019 -0500

    Don't redefine records with the same name
    
    This could lead to lots of errors where we're passing these records
    between the new and old versions of code.
---
 src/couch_views/include/couch_views.hrl | 70 ---------------------------------
 1 file changed, 70 deletions(-)

diff --git a/src/couch_views/include/couch_views.hrl b/src/couch_views/include/couch_views.hrl
index f5a9c8b..99a62b0 100644
--- a/src/couch_views/include/couch_views.hrl
+++ b/src/couch_views/include/couch_views.hrl
@@ -22,73 +22,3 @@
 
 % jobs api
 -define(INDEX_JOB_TYPE, <<"views">>).
-
-
--record(mrst, {
-    sig=nil,
-    fd=nil,
-    db_name,
-    idx_name,
-    language,
-    design_opts=[],
-    seq_indexed=false,
-    keyseq_indexed=false,
-    partitioned=false,
-    lib,
-    views,
-    % update_seq=0,
-    % purge_seq=0,
-    % first_build,
-    % partial_resp_pid,
-    % doc_acc,
-    % doc_queue,
-    % write_queue,
-    qserver=nil
-}).
-
-
--record(mrview, {
-    id_num,
-    % update_seq=0,
-    % purge_seq=0,
-    map_names=[],
-    reduce_funs=[],
-    def,
-    seq_indexed=false,
-    keyseq_indexed=false,
-    options=[]
-}).
-
-
--define(MAX_VIEW_LIMIT, 16#10000000).
-
-
--record(mrargs, {
-    view_type,
-    % reduce,
-
-    % preflight_fun,
-
-    start_key,
-    start_key_docid,
-    end_key,
-    end_key_docid,
-    keys,
-
-    direction = fwd,
-    limit = ?MAX_VIEW_LIMIT,
-    skip = 0,
-    % group_level = 0,
-    % group = undefined,
-    stable = false,
-    update = true,
-    multi_get = false,
-    inclusive_end = true,
-    include_docs = false,
-    doc_options = [],
-    update_seq=false,
-    conflicts,
-    % callback,
-    sorted = true
-    % extra = []
-}).


[couchdb] 08/25: Simplify couch_views_worker_server

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

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

commit 5c0b671302d778d870236ce2d6261b8dbf86dad4
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Wed Jul 17 10:41:20 2019 -0500

    Simplify couch_views_worker_server
    
    This just follows the standard pattern of keeping a pool of workers
    alive that will accept jobs individually. This avoids all of the oddness
    in passing jobs around after they've been accepted.
    
    I've also renamed it couch_views_server so that the name is a bit less
    wordy.
---
 ...ws_worker_server.erl => couch_views_server.erl} | 69 +++++++++++-----------
 1 file changed, 36 insertions(+), 33 deletions(-)

diff --git a/src/couch_views/src/couch_views_worker_server.erl b/src/couch_views/src/couch_views_server.erl
similarity index 60%
rename from src/couch_views/src/couch_views_worker_server.erl
rename to src/couch_views/src/couch_views_server.erl
index 13bd9aa..8ec2425 100644
--- a/src/couch_views/src/couch_views_worker_server.erl
+++ b/src/couch_views/src/couch_views_server.erl
@@ -10,7 +10,7 @@
 % License for the specific language governing permissions and limitations under
 % the License.
 
--module(couch_views_worker_server).
+-module(couch_views_server).
 
 
 -behaviour(gen_server).
@@ -39,13 +39,13 @@ start_link() ->
 
 
 init(_) ->
+    process_flag(trap_exit, true),
     couch_views_jobs:set_timeout(),
     State0 = #{
-        workers => #{},
-        acceptor_pid => undefined
+        workers => [],
+        num_workers => num_workers()
     },
-    State = spawn_acceptor(State0),
-    {ok, State}.
+    {ok, spawn_workers(State)}.
 
 
 terminate(_, _St) ->
@@ -56,53 +56,51 @@ handle_call(Msg, _From, St) ->
     {stop, {bad_call, Msg}, {bad_call, Msg}, St}.
 
 
-handle_cast({job, Job, JobData}, State) ->
-    State1 = start_worker(State, Job, JobData),
-    State2 = spawn_acceptor(State1),
-    {noreply, State2};
-
 handle_cast(Msg, St) ->
     {stop, {bad_cast, Msg}, St}.
 
 
-handle_info({'DOWN', _Ref, process, Pid, Reason}, State) ->
-    LogMsg = "~p : process ~p exited with ~p",
-    couch_log:error(LogMsg, [?MODULE, Pid, Reason]),
-    State1 = check_finished_process(State, Pid),
-    {noreply, State1};
+handle_info({'EXIT', Pid, Reason}, State) ->
+    #{workers := Workers} = State,
+    case Workers -- [Pid] of
+        Workers ->
+            LogMsg = "~p : unknown process ~p exited with ~p",
+            couch_log:error(LogMsg, [?MODULE, Pid, Reason]),
+            {stop, {unknown_pid_exit, Pid}, State};
+        NewWorkers ->
+            if Reason == normal -> ok; true ->
+                LogMsg = "~p : indexer process ~p exited with ~p",
+                couch_log:error(LogMsg, [?MODULE, Pid, Reason]),
+            end,
+            {noreply, spawn_workers(State#{workers := NewWorkers})}
+    end;
 
 handle_info(Msg, St) ->
-    couch_log:notice("~s ignoring info ~w", [?MODULE, Msg]),
-    {noreply, St}.
+    {stop, {bad_info, Msg}, St}.
 
 
 code_change(_OldVsn, St, _Extra) ->
     {ok, St}.
 
 
-start_worker(State, Job, JobData) ->
-    #{workers := Workers} = State,
-    {Pid, _Ref} = spawn_monitor(fun () -> couch_views_worker:start(Job, JobData) end),
-    Workers1 = Workers#{Pid => true},
-    State#{workers := Workers1}.
-
-
-spawn_acceptor(State) ->
+spawn_workers(State) ->
     #{
         workers := Workers,
-        acceptor_pid := Pid
+        num_workers := NumWorkers
     } = State,
-    MaxWorkers = config:get_integer("couch_views", "max_workers", ?MAX_WORKERS),
-    case maps:size(Workers) >= MaxWorkers of
-        false when not is_pid(Pid) ->
-            Parent = self(),
-            {Pid1, _Ref} = spawn_monitor(fun() -> blocking_acceptor(Parent) end),
-            State#{acceptor_pid := Pid1};
-        _ ->
+    case length(Workers) < NumWorkers of
+        true ->
+            Pid = spawn_worker(),
+            spawn_workers(State#{workers := [Pid | Workers]});
+        false ->
             State
     end.
 
 
+spawn_worker() ->
+    couch_views_indexer:spawn_link().
+
+
 blocking_acceptor(Parent) ->
     case couch_views_jobs:accept() of
         not_found ->
@@ -120,3 +118,8 @@ check_finished_process(State, Pid) ->
     #{workers := Workers} = State,
     Workers1 = maps:remove(Pid, Workers),
     State#{workers := Workers1}.
+
+
+
+num_workers() ->
+    config:get_integer("couch_views", "max_workers", ?MAX_WORKERS).


[couchdb] 20/25: Add Garren's map_test.exs

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

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

commit 6119d9875bc32887bc141cb855956e10f760cbce
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 23 11:52:18 2019 -0500

    Add Garren's map_test.exs
---
 test/elixir/test/map_test.exs | 121 +++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 119 insertions(+), 2 deletions(-)

diff --git a/test/elixir/test/map_test.exs b/test/elixir/test/map_test.exs
index 7c443ab..0a19760 100644
--- a/test/elixir/test/map_test.exs
+++ b/test/elixir/test/map_test.exs
@@ -9,6 +9,11 @@ defmodule ViewMapTest do
     Enum.map(rows, fn row -> row["id"] end)
   end
 
+  def get_keys(resp) do
+    %{:body => %{"rows" => rows}} = resp
+    Enum.map(rows, fn row -> row["key"] end)
+  end
+
   defp create_map_docs(db_name) do
     docs =
       for i <- 1..10 do
@@ -27,7 +32,7 @@ defmodule ViewMapTest do
         }
       end
 
-    resp = Couch.post("/#{db_name}/_bulk_docs", body: %{:docs => docs})
+    resp = Couch.post("/#{db_name}/_bulk_docs", body: %{:docs => docs, :w => 3})
     assert resp.status_code == 201
   end
 
@@ -58,13 +63,23 @@ defmodule ViewMapTest do
       }
     """
 
+    map_fun3 = """
+      function(doc) {
+        if (doc.group) {
+            emit(doc.group, 1);
+        }
+      }
+    """
+
     body = %{
+      :w => 3,
       :docs => [
         %{
           _id: "_design/map",
           views: %{
             some: %{map: map_fun1},
-            map_some: %{map: map_fun2}
+            map_some: %{map: map_fun2},
+            map_group: %{map: map_fun3}
           }
         },
         %{
@@ -197,6 +212,108 @@ defmodule ViewMapTest do
     assert ids == ["_design/include_ddocs", "_design/map", "doc-id-1"]
   end
 
+  test "can use key in query string", context do
+    db_name = context[:db_name]
+
+    url = "/#{db_name}/_design/map/_view/map_group"
+    resp = Couch.get(url, query: %{limit: 3, key: "\"one\""})
+    assert resp.status_code == 200
+    ids = get_ids(resp)
+    assert ids == ["doc-id-3", "doc-id-6", "doc-id-9"]
+
+    resp =
+      Couch.get(url,
+        query: %{
+          limit: 3,
+          key: "\"one\"",
+          descending: true
+        }
+      )
+
+    assert resp.status_code == 200
+    ids = get_ids(resp)
+    assert ids == ["doc-id-9", "doc-id-6", "doc-id-3"]
+  end
+
+  test "can use keys in query string", context do
+    db_name = context[:db_name]
+
+    url = "/#{db_name}/_design/map/_view/some"
+    resp = Couch.post(url, body: %{keys: [6, 3, 9]})
+    assert resp.status_code == 200
+    ids = get_ids(resp)
+
+    # should ignore descending = true
+    resp = Couch.post(url, body: %{keys: [6, 3, 9], descending: true})
+    assert resp.status_code == 200
+    ids = get_ids(resp)
+    assert ids == ["doc-id-6", "doc-id-3", "doc-id-9"]
+  end
+
+  test "inclusive = false", context do
+    db_name = context[:db_name]
+
+    docs = [
+      %{key: "key1"},
+      %{key: "key2"},
+      %{key: "key3"},
+      %{key: "key4"},
+      %{key: "key4"},
+      %{key: "key5"},
+      %{
+        _id: "_design/inclusive",
+        views: %{
+          by_key: %{
+            map: """
+                function (doc) {
+                    if (doc.key) {
+                        emit(doc.key, doc);
+                    }
+                }
+            """
+          }
+        }
+      }
+    ]
+
+    resp = Couch.post("/#{db_name}/_bulk_docs", body: %{:docs => docs, :w => 3})
+    assert resp.status_code == 201
+    url = "/#{db_name}/_design/inclusive/_view/by_key"
+
+    query = %{
+      endkey: "\"key4\"",
+      inclusive_end: false
+    }
+
+    resp = Couch.get(url, query: query)
+    assert resp.status_code == 200
+    keys = get_keys(resp)
+    assert keys == ["key1", "key2", "key3"]
+
+    query = %{
+      startkey: "\"key3\"",
+      endkey: "\"key4\"",
+      inclusive_end: false
+    }
+
+    resp = Couch.get(url, query: query)
+    assert resp.status_code == 200
+    keys = get_keys(resp)
+    assert keys == ["key3"]
+
+    query = %{
+      startkey: "\"key4\"",
+      endkey: "\"key1\"",
+      inclusive_end: false,
+      descending: true
+    }
+
+    resp = Couch.get(url, query: query)
+    assert resp.status_code == 200
+    keys = get_keys(resp)
+    assert keys == ["key4", "key4", "key3", "key2"]
+  end
+
   def update_doc_value(db_name, id, value) do
     resp = Couch.get("/#{db_name}/#{id}")
     doc = convert(resp.body)


[couchdb] 25/25: Minor tweaks to remove reduce function tests

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

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

commit 741a11582433a66449a4f2834d148efe5f0ff558
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 23 15:12:54 2019 -0500

    Minor tweaks to remove reduce function tests
---
 test/elixir/test/basics_test.exs | 25 +++++++++++++++++++------
 1 file changed, 19 insertions(+), 6 deletions(-)

diff --git a/test/elixir/test/basics_test.exs b/test/elixir/test/basics_test.exs
index c28c78c..ea5c125 100644
--- a/test/elixir/test/basics_test.exs
+++ b/test/elixir/test/basics_test.exs
@@ -156,6 +156,7 @@ defmodule BasicsTest do
 
     # Initial view query test
     resp = Couch.get("/#{db_name}/_design/foo/_view/baz")
+    Logger.debug("#{inspect resp}")
     assert resp.body["total_rows"] == 1
     assert hd(resp.body["rows"])["value"] == 16
 
@@ -178,21 +179,33 @@ defmodule BasicsTest do
 
     assert Couch.get("/#{db_name}").body["doc_count"] == 8
 
+    # Disabling until we figure out reduce functions
+    # # Test reduce function
+    # resp = Couch.get("/#{db_name}/_design/bar/_view/baz")
+    # assert hd(resp.body["rows"])["value"] == 33
+
     # Test reduce function
-    resp = Couch.get("/#{db_name}/_design/bar/_view/baz")
-    assert hd(resp.body["rows"])["value"] == 33
+    resp = Couch.get("/#{db_name}/_design/bar/_view/baz", query: %{:reduce => false})
+    assert resp.body["total_rows"] == 3
 
     # Delete doc and test for updated view results
     doc0 = Couch.get("/#{db_name}/0").body
     assert Couch.delete("/#{db_name}/0?rev=#{doc0["_rev"]}").body["ok"]
 
-    retry_until(fn ->
-      Couch.get("/#{db_name}/_design/foo/_view/baz").body["total_rows"] == 2
-    end)
+    # Disabling until we figure out reduce functions
+    # retry_until(fn ->
+    #  Couch.get("/#{db_name}/_design/foo/_view/baz").body["total_rows"] == 2
+    # end)
+
+    resp = Couch.get("/#{db_name}/_design/bar/_view/baz", query: %{:reduce => false})
+    assert resp.body["total_rows"] == 2
 
     assert Couch.get("/#{db_name}").body["doc_count"] == 7
     assert Couch.get("/#{db_name}/0").status_code == 404
-    refute Couch.get("/#{db_name}/0?rev=#{doc0["_rev"]}").status_code == 404
+
+    # No longer true. Old revisions are not stored after
+    # an update.
+    # refute Couch.get("/#{db_name}/0?rev=#{doc0["_rev"]}").status_code == 404
   end
 
   @tag :with_db


[couchdb] 24/25: Add total row count support

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

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

commit 81521e1e621dd5ca17a27d9c7d3893e9eb4224e7
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 23 15:12:31 2019 -0500

    Add total row count support
---
 src/couch_views/include/couch_views.hrl           | 16 ++++----
 src/couch_views/src/couch_views_fdb.erl           | 50 ++++++++++++++++++++---
 src/couch_views/src/couch_views_reader.erl        |  9 ++--
 src/couch_views/test/couch_views_indexer_test.erl |  4 +-
 4 files changed, 59 insertions(+), 20 deletions(-)

diff --git a/src/couch_views/include/couch_views.hrl b/src/couch_views/include/couch_views.hrl
index 4fcc57e..525f62f 100644
--- a/src/couch_views/include/couch_views.hrl
+++ b/src/couch_views/include/couch_views.hrl
@@ -11,15 +11,15 @@
 % the License.
 
 % indexing
--define(VIEW_UPDATE_SEQ, 1).
--define(VIEW_ID_RANGE,   2).
--define(VIEW_MAP_RANGE,  3).
--define(VIEW_BUILDS,     4).
--define(VIEW_STATUS,     5).
--define(VIEW_WATCH,      6).
+-define(VIEW_UPDATE_SEQ, 0).
+-define(VIEW_ID_INFO, 1).
+-define(VIEW_ID_RANGE, 2).
+-define(VIEW_MAP_RANGE, 3).
 
--define(VIEW_ROW_KEY,    0).
--define(VIEW_ROW_VALUE,  1).
+-define(VIEW_ROW_COUNT, 0).
+
+-define(VIEW_ROW_KEY, 0).
+-define(VIEW_ROW_VALUE, 1).
 
 % jobs api
 -define(INDEX_JOB_TYPE, <<"views">>).
diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
index 09a9802..a0e4bd1 100644
--- a/src/couch_views/src/couch_views_fdb.erl
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -16,6 +16,8 @@
     get_update_seq/2,
     set_update_seq/3,
 
+    get_row_count/3,
+
     fold_map_idx/6,
 
     write_doc/4
@@ -59,6 +61,18 @@ set_update_seq(TxDb, Sig, Seq) ->
     ok = erlfdb:set(Tx, seq_key(DbPrefix, Sig), Seq).
 
 
+get_row_count(TxDb, #mrst{sig = Sig}, ViewId) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = TxDb,
+
+    case erlfdb:wait(erlfdb:get(Tx, row_count_key(DbPrefix, Sig, ViewId))) of
+        not_found -> 0; % Can this happen?
+        CountBin -> ?bin2uint(CountBin)
+    end.
+
+
 fold_map_idx(TxDb, Sig, ViewId, Options, Callback, Acc0) ->
     #{
         db_prefix := DbPrefix
@@ -107,8 +121,9 @@ write_doc(TxDb, Sig, _ViewIds, #{deleted := true} = Doc) ->
     ExistingViewKeys = get_view_keys(TxDb, Sig, DocId),
 
     clear_id_idx(TxDb, Sig, DocId),
-    lists:foreach(fun({ViewId, ViewKeys}) ->
-        clear_map_idx(TxDb, Sig, ViewId, DocId, ViewKeys)
+    lists:foreach(fun({ViewId, TotalKeys, UniqueKeys}) ->
+        clear_map_idx(TxDb, Sig, ViewId, DocId, UniqueKeys),
+        update_row_count(TxDb, Sig, ViewId, -TotalKeys)
     end, ExistingViewKeys);
 
 write_doc(TxDb, Sig, ViewIds, Doc) ->
@@ -122,8 +137,17 @@ write_doc(TxDb, Sig, ViewIds, Doc) ->
     clear_id_idx(TxDb, Sig, DocId),
 
     lists:foreach(fun({ViewId, NewRows}) ->
-        ExistingKeys = fabric2_util:get_value(ViewId, ExistingViewKeys, []),
         update_id_idx(TxDb, Sig, ViewId, DocId, NewRows),
+
+        ExistingKeys = case lists:keyfind(ViewId, 1, ExistingViewKeys) of
+            {ViewId, TotalRows, EKeys} ->
+                Change = length(NewRows) - TotalRows,
+                update_row_count(TxDb, Sig, ViewId, Change),
+                EKeys;
+            false ->
+                update_row_count(TxDb, Sig, ViewId, length(NewRows)),
+                []
+        end,
         update_map_idx(TxDb, Sig, ViewId, DocId, ExistingKeys, NewRows)
     end, lists:zip(ViewIds, Results)).
 
@@ -251,7 +275,7 @@ update_id_idx(TxDb, Sig, ViewId, DocId, NewRows) ->
     couch_log:error("Updating ID index: ~p ~p ~p ~p", [ViewId, DocId, NewRows, Unique]),
 
     Key = id_idx_key(DbPrefix, Sig, DocId, ViewId),
-    Val = couch_views_encoding:encode(Unique),
+    Val = couch_views_encoding:encode([length(NewRows), Unique]),
     ok = erlfdb:set(Tx, Key, Val).
 
 
@@ -289,16 +313,30 @@ get_view_keys(TxDb, Sig, DocId) ->
     lists:map(fun({K, V}) ->
         {?DB_VIEWS, Sig, ?VIEW_ID_RANGE, DocId, ViewId} =
                 erlfdb_tuple:unpack(K, DbPrefix),
-        ViewKeys = couch_views_encoding:decode(V),
-        {ViewId, ViewKeys}
+        [TotalKeys, UniqueKeys] = couch_views_encoding:decode(V),
+        {ViewId, TotalKeys, UniqueKeys}
     end, erlfdb:get_range(Tx, Start, End, [])).
 
 
+update_row_count(TxDb, Sig, ViewId, Increment) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = TxDb,
+    Key = row_count_key(DbPrefix, Sig, ViewId),
+    erlfdb:add(Tx, Key, Increment).
+
+
 seq_key(DbPrefix, Sig) ->
     Key = {?DB_VIEWS, Sig, ?VIEW_UPDATE_SEQ},
     erlfdb_tuple:pack(Key, DbPrefix).
 
 
+row_count_key(DbPrefix, Sig, ViewId) ->
+    Key = {?DB_VIEWS, Sig, ?VIEW_ID_INFO, ViewId, ?VIEW_ROW_COUNT},
+    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/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index ce5097ba..8bbe1fc 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -34,11 +34,12 @@ read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
     Fun = fun handle_row/4,
 
     try
-        % Need to add total_rows support
-        Meta = {meta, [{total_rows, null}, {offset, null}]},
-        UserAcc1 = maybe_stop(UserCallback(Meta, UserAcc0)),
-
         fabric2_fdb:transactional(Db, fun(TxDb) ->
+            TotalRows = couch_views_fdb:get_row_count(TxDb, Mrst, ViewId),
+
+            Meta = {meta, [{total, TotalRows}, {offset, null}]},
+            UserAcc1 = maybe_stop(UserCallback(Meta, UserAcc0)),
+
             Acc0 = #{
                 db => TxDb,
                 skip => Args#mrargs.skip,
diff --git a/src/couch_views/test/couch_views_indexer_test.erl b/src/couch_views/test/couch_views_indexer_test.erl
index e6dfdc4..fa0d99e 100644
--- a/src/couch_views/test/couch_views_indexer_test.erl
+++ b/src/couch_views/test/couch_views_indexer_test.erl
@@ -157,7 +157,7 @@ updated_docs_are_reindexed(Db) ->
     DbName = fabric2_db:name(Db),
     {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
     Sig = Mrst#mrst.sig,
-    Expect = [{0, [1]}, {1, []}],
+    Expect = [{0, 1, [1]}, {1, 0, []}],
     fabric2_fdb:transactional(Db, fun(TxDb) ->
         ?assertEqual(
                 Expect,
@@ -214,7 +214,7 @@ updated_docs_without_changes_are_reindexed(Db) ->
     DbName = fabric2_db:name(Db),
     {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
     Sig = Mrst#mrst.sig,
-    Expect = [{0, [0]}, {1, []}],
+    Expect = [{0, 1, [0]}, {1, 0, []}],
     fabric2_fdb:transactional(Db, fun(TxDb) ->
         ?assertEqual(
                 Expect,


[couchdb] 21/25: Update chttpd for new API

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

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

commit d6758dbfdbe4a4a4443c4e41a28a398cfbea5c7c
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 23 14:24:59 2019 -0500

    Update chttpd for new API
---
 src/chttpd/src/chttpd_view.erl | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/src/chttpd/src/chttpd_view.erl b/src/chttpd/src/chttpd_view.erl
index 849d870..6765cca 100644
--- a/src/chttpd/src/chttpd_view.erl
+++ b/src/chttpd/src/chttpd_view.erl
@@ -43,10 +43,9 @@ multi_query_view(Req, Db, DDoc, ViewName, Queries) ->
 design_doc_view(Req, Db, DDoc, ViewName, Keys) ->
     Args = couch_mrview_http:parse_params(Req, Keys),
     Max = chttpd:chunked_response_buffer_size(),
+    Fun = fun view_cb/2,
     VAcc = #vacc{db=Db, req=Req, threshold=Max},
-    Options = [{user_ctx, Req#httpd.user_ctx}],
-    {ok, Resp} = fabric2_view:query(Db, Options, DDoc, ViewName,
-            fun view_cb/2, VAcc, Args),
+    {ok, Resp} = couch_views:query(Db, DDoc, ViewName, Fun, VAcc, Args),
     {ok, Resp#vacc.resp}.
 
 


[couchdb] 13/25: Move fdb logic to couch_views_fdb

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

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

commit 5d6068e004f1976191c9327a97d9b33c452e64f3
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Thu Jul 18 13:58:57 2019 -0500

    Move fdb logic to couch_views_fdb
---
 src/couch_views/src/couch_views.erl        |  45 +-----
 src/couch_views/src/couch_views_fdb.erl    | 147 +++++++++++++++--
 src/couch_views/src/couch_views_reader.erl | 250 ++++++++++++-----------------
 3 files changed, 245 insertions(+), 197 deletions(-)

diff --git a/src/couch_views/src/couch_views.erl b/src/couch_views/src/couch_views.erl
index 65af1bf..7deb54d 100644
--- a/src/couch_views/src/couch_views.erl
+++ b/src/couch_views/src/couch_views.erl
@@ -43,23 +43,22 @@ query(Db, DDoc, ViewName, Callback, Acc0, QueryArgs0) ->
         false -> ok
     end,
 
-    Args = mrargs_to_map(QueryArgs2),
-    ok = maybe_update_view(Db, Mrst, Args),
+    ok = maybe_update_view(Db, Mrst, QueryArgs2),
 
     try
-        couch_views_reader:read(Db, Mrst, ViewName, Callback, Acc0, Args)
+        couch_views_reader:read(Db, Mrst, ViewName, Callback, Acc0, QueryArgs2)
     after
-        UpdateAfter = maps:get(update, Args) == lazy,
+        UpdateAfter = QueryArgs2#mrargs.update == lazy,
         if UpdateAfter == false -> ok; true ->
             couch_views_jobs:build_view_async(Db, Mrst)
         end
     end.
 
 
-maybe_update_view(_Db, _Mrst, #{update := false}) ->
+maybe_update_view(_Db, _Mrst, #mrargs{update = false}) ->
     ok;
 
-maybe_update_view(_Db, _Mrst, #{update := lazy}) ->
+maybe_update_view(_Db, _Mrst, #mrargs{update = laze}) ->
     ok;
 
 maybe_update_view(Db, Mrst, _Args) ->
@@ -83,39 +82,5 @@ is_reduce_view({Reduce, _, _}) ->
     Reduce =:= red.
 
 
-mrargs_to_map(#mrargs{} = Args) ->
-    process_args(#{
-        start_key => Args#mrargs.start_key,
-        start_key_docid => Args#mrargs.start_key_docid,
-        end_key => Args#mrargs.end_key,
-        end_key_docid => Args#mrargs.end_key_docid,
-        keys => Args#mrargs.keys,
-        direction => Args#mrargs.direction,
-        limit => Args#mrargs.limit,
-        skip => Args#mrargs.skip,
-        update => Args#mrargs.update,
-        multi_get => Args#mrargs.multi_get,
-        inclusive_end => Args#mrargs.inclusive_end,
-        include_docs => Args#mrargs.include_docs,
-        doc_options => Args#mrargs.doc_options,
-        update_seq => Args#mrargs.update_seq,
-        conflicts => Args#mrargs.conflicts,
-        sorted => Args#mrargs.sorted
-    }).
-
-
-process_args(#{} = Args) ->
-    Args1 = remove_ununsed_values(Args),
-    Defaults = #{
-            direction => fwd,
-            inclusive_end => true,
-            update => true,
-            skip => 0,
-            limit => ?MAX_VIEW_LIMIT
-        },
-
-    maps:merge(Defaults, Args1).
-
-
 remove_ununsed_values(Args) ->
     maps:filter(fun (_, V) -> V /= undefined end, Args).
diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
index f47f1b1..57ed5f1 100644
--- a/src/couch_views/src/couch_views_fdb.erl
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -16,7 +16,9 @@
     get_update_seq/2,
     set_update_seq/3,
 
-    write_rows/4
+    fold_map_idx/5,
+
+    write_doc/4
 ]).
 
 
@@ -54,18 +56,54 @@ set_view_seq(TxDb, Sig, Seq) ->
     ok = erlfdb:set(Tx, SeqKey, Seq).
 
 
+fold_map_idx(TxDb, Sig, ViewId, Options, Callback, Acc0) ->
+    #{
+        tx := Tx,
+        db_prefix := DbPrefix
+    } = TxDb,
+
+    MapIdxPrefix = map_idx_prefix(DbPrefix, Sig, ViewId),
+    {Fun, Acc} = case fabric2_util:get_value(dir, Options, fwd) of
+        fwd ->
+            FwdAcc = #{
+                prefix => MapIdxPrefix,
+                next => key,
+                key => undefined,
+                sort_key => undefined,
+                docid => undefined,
+                dupe_id => undefined,
+                callback => Callback,
+                acc => Acc0,
+            },
+            {fun fold_fwd/2, FwdAcc}
+        rev ->
+            RevAcc #{
+                prefix => MapIdxPrefix,
+                next => value,
+                value => undefined,
+                sort_key => undefined,
+                docid => undefined,
+                dupe_id => undefined,
+                callback => Callback,
+                acc => Acc0
+            },
+            {fun fold_rev/2, RevAcc}
+    end,
+
+    fabric2_db:fold_range(TxDb, MapIdxPrefix, Fun, Acc, Options).
+
+
 write_doc(TxDb, Sig, #{deleted := true} = Doc, ViewIds) ->
     #{
         id := DocId
     } = Doc,
 
-    ViewKeys = get_view_keys(TxDb, Sig, DocId),
+    ExistingViewKeys = get_view_keys(TxDb, Sig, DocId),
 
-    clear_id_idx(TxDb, Sig, DocId),
+    sclear_id_idx(TxDb, Sig, DocId),
     lists:foreach(fun({ViewId, ViewKeys}) ->
         clear_map_idx(TxDb, Sig, ViewId, ViewKeys)
-    end, ViewKeys).
-
+    end, ExistingViewKeys);
 
 write_doc(TxDb, Sig, Doc, ViewIds) ->
     #{
@@ -79,7 +117,7 @@ write_doc(TxDb, Sig, Doc, ViewIds) ->
 
     ExistingViewKeys = get_view_keys(TxDb, Sig, DocId),
 
-    ok = clear_id_idx(TxDb, Sig, DocId),
+    clear_id_idx(TxDb, Sig, DocId),
 
     lists:foreach(fun({ViewId, NewRows}) ->
         ExistingKeys = fabric2_util:get_value(ViewId, ExistingViewKeys, []),
@@ -88,6 +126,95 @@ write_doc(TxDb, Sig, Doc, ViewIds) ->
     end, lists:zip(ViewIds, Results)).
 
 
+fold_fwd({RowKey, EncodedOriginalKey}, #{next := key} = Acc) ->
+    #{
+        prefix := Prefix
+    } = Acc,
+
+    {{SortKey, DocId}, _DupeId, ?VIEW_ROW_KEY} =
+            erlfdb_tuple:unpack(RowKey, Prefix),
+    Acc#{
+        next := val,
+        key := couch_views_encoding:decode(EncodedOriginalKey),
+        sort_key := SortKey,
+        docid := DocId,
+        dupe_id := DupeId
+    };
+
+fold_fwd({RowKey, EncodedValue}, #{next := val} = Acc) ->
+    #{
+        prefix := Prefix,
+        key := Key,
+        sort_key := SortKey,
+        docid := DocId,
+        dupe_id := DocId,
+        callback := UserCallback
+        acc := UserAcc0
+    } = Acc,
+
+    % We're asserting there that this row is paired
+    % correctly with the previous row by relying on
+    % a badmatch if any of these values don't match.
+    {{SortKey, DocId}, DupeId, ?VIEW_ROW_VAL} =
+            erlfdb_tuple:unpack(RowKey, Prefix),
+
+    Value = couch_views_encoding:decode(EncodedValue),
+    NewAcc = UserCallback(DocId, Key, Value, UserAcc0),
+
+    #{
+        next := key,
+        key := undefined,
+        sort_key := undefined,
+        docid := undefined,
+        dupe_id := undefined,
+        acc := UserAcc1
+    }.
+
+
+fold_rev({RowKey, EncodedValue}, #{next := value} = Acc) ->
+    #{
+        prefix := Prefix
+    } = Acc,
+
+    {{SortKey, DocId}, _DupeId, ?VIEW_ROW_VAL} =
+            erlfdb_tuple:unpack(RowKey, Prefix),
+    Acc#{
+        next := key,
+        value := couch_views_encoding:decode(EncodedValue),
+        sort_key := SortKey,
+        docid := DocId,
+        dupe_id := DupeId
+    };
+
+fold_rev({RowKey, EncodedOriginalKey}, #{next := key} = Acc) ->
+    #{
+        prefix := Prefix,
+        value := Value,
+        sort_key := SortKey,
+        docid := DocId,
+        dupe_id := DocId,
+        callback := UserCallback
+        acc := UserAcc0
+    } = Acc,
+
+    % We're asserting there that this row is paired
+    % correctly with the previous row by relying on
+    % a badmatch if any of these values don't match.
+    {{SortKey, DocId}, DupeId, ?VIEW_ROW_KEY} =
+            erlfdb_tuple:unpack(RowKey, Prefix),
+
+    Key = couch_views_encoding:decode(EncodedOriginalKey),
+    NewAcc = UserCallback(DocId, Key, Value, UserAcc0),
+
+    #{
+        next := val,
+        value := undefined,
+        sort_key := undefined,
+        docid := undefined,
+        dupe_id := undefined,
+        acc := UserAcc1
+    }.
+
 clear_id_idx(TxDb, Sig, DocId) ->
     #{
         tx := Tx,
@@ -141,10 +268,10 @@ update_map_idx(TxDb, Sig, ViewId, DocId, ExistingKeys, NewRows) ->
     MapIdxPrefix = map_idx_prefix(DbPrefix, Sig, ViewId),
 
     lists:foreach(fun({DupeId, Key1, Key2, Val}) ->
-        KeyKey = map_idx_key(MapIdxPrefix, Key1, DocId, DupeId, ?VIEW_ROW_KEY),
-        ValKey = map_idx_key(MapIdxPrefix, Key1, DocId, DupeId, ?VIEW_ROW_VAL),
-        ok = erlfdn:store(Tx, KeyKey, Key2),
-        ok = erlfdb:store(Tx, ValKey, Val)
+        KK = map_idx_key(MapIdxPrefix, {Key1, DocId}, DupeId, ?VIEW_ROW_KEY),
+        VK = map_idx_key(MapIdxPrefix, {Key1, DocId}, DupeId, ?VIEW_ROW_VAL),
+        ok = erlfdn:store(Tx, KK, Key2),
+        ok = erlfdb:store(Tx, VK, Val)
     end, KVsToAdd).
 
 
diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index f4e768a..56b23f2 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -22,174 +22,130 @@
 -include_lib("fabric/src/fabric2.hrl").
 
 
-read(Db, DDoc, ViewName, Callback, Acc0, Args) ->
-    #{name := DbName} = Db,
-
-    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
     #mrst{
         sig = Sig,
         views = Views
     } = Mrst,
 
-    IdxName = get_idx_name(ViewName, Views),
-    State0 = #{
-        acc => Acc0,
-        skip => maps:get(skip, Args, 0),
-        include_docs => maps:get(include_docs, Args, false),
-        db => Db
-    },
+    ViewId = get_view_id(ViewName, Views),
+    Opts = mrargs_to_fdb_options(Args),
+    Fun = fun handle_row/4,
+
+    try
+        % Need to add total_rows support
+        Meta = {meta, [{total_rows, null}, {offset, null}]},
+        UserAcc1 = maybe_stop(UserCallback(Meta, UserAcc0)),
+
+        fabric2_fdb:transactional(Db, fun(TxDb) ->
+            Acc0 = #{
+                db => TxDb
+                skip => Args#mrargs.skip,
+                mrargs => Args,
+                callback => UserCallback,
+                acc => UserAcc1
+            },
+
+            Acc1 = couch_views_fdb:fold_map_idx(
+                    TxDb,
+                    Sig,
+                    ViewId,
+                    Opts,
+                    Fun,
+                    Acc0
+                ),
+
+            #{
+                acc := UserAcc2
+            } = Acc1,
+
+            maybe_stop(Callback(complete, UserAcc2)
+        end)
+    catch throw:{done, Out} ->
+        {ok, Out}
+    end.
 
-    DefaultOpts = [{streaming_mode, want_all}],
-    {Start, End, QueryOpts} = convert_args_to_fdb(Db, Sig, IdxName, Args,
-        DefaultOpts),
-    Opts = QueryOpts ++ DefaultOpts,
 
-    fabric2_fdb:transactional(Db, fun(TxDb) ->
-        Future = couch_views_fdb:get_map_range(TxDb, Start, End, Opts),
+handle_row(_DocId, _Key, _Value, #{skip := Skip} = Acc) when Skip > 0 ->
+    {ok, Acc#{skip := Skip - 1}};
 
-        UnPack = get_unpack_fun(TxDb, Opts, Callback),
-        State1 = lists:foldl(UnPack, State0, erlfdb:wait(Future)),
+handle_row(DocId, Key, Value, Acc) ->
+    #{
+        db := TxDb
+        mrargs := Args,
+        callback := UserCallback,
+        acc := UserAcc
+    } = Acc,
+
+    BaseRow = [
+        {id, DocId},
+        {key, Key},
+        {value, Value}
+    ],
+
+    Row = BaseRow ++ if not IncludeDocs -> []; true ->
+        DocOpts0 = Args#mrargs.doc_options,
+        DocOpts1 = OpenOpts0 ++ case Args#mrargs.conflicts of
+            true -> [conflicts];
+            false -> []
+        end,
+        DocObj = case fabric2_db:open_doc(Db, DocId, DocOpts1) of
+            {ok, Doc} -> couch_doc:to_json_obj(Doc, DocOpts1);
+            {not_found, _} -> null
+        end,
+        [{doc, DocObj}]
+    end,
 
-        #{acc := Acc1} = State1,
-        Callback(complete, Acc1)
-    end).
+    UserAcc1 = maybe_stop(Callback({row, Row1}, UserAcc0)),
+    Acc#{acc := UserAcc1}
 
 
-get_idx_name(ViewName, Views) ->
-    {value, View} = lists:search(fun (View) ->
+get_view_id(ViewName, Views) ->
+    {value, View} = lists:search(fun(View) ->
         lists:member(ViewName, View#mrview.map_names)
     end, Views),
     View#mrview.id_num.
 
 
-convert_args_to_fdb(Db, Sig, IdxName, Args, Opts) ->
-    #{
-        direction := Direction
-    } = Args,
-
-    {Start1, End1} = get_range_keys(Db, Sig, IdxName, Args),
-
-    Opts1 = case maps:is_key(limit, Args) of
-        false ->
-            Opts;
-        true ->
-            Skip = maps:get(skip, Args, 0),
-            Limit = maps:get(limit, Args),
-            % Limit is multiplied by two because there are two rows per key
-            % value.
-            % Skip is added because that is done in the fold so we need
-            % to fetch the number of documents
-            % along with the docs we would skip.
-            % Limit = (Doc limit + Skip) * Num of Rows per Map KV
-            [{limit, (Limit + Skip) * 2} | Opts]
-    end,
-
-    Opts2 = case Direction of
-        fwd ->
-            Opts1;
-        rev ->
-            [{reverse, true} | Opts1]
-    end,
-    {Start1, End1, Opts2}.
-
-
-get_range_keys(Db, Sig, IdxName, Args) ->
-    #{
-        inclusive_end := InclusiveEnd,
-        direction := Direction
+mrargs_to_fdb_options(Args) ->
+    #mrargs{
+        start_key = StartKey,
+        start_key_docid = StartKeyDocId,
+        end_key = EndKey,
+        end_key_docid = EndKeyDocId,
+        direction = Direction,
+        limit = Limit,
+        inclusive_end = InclusiveEnd
     } = Args,
 
-    {MapStartKey, MapEndKey} = case Direction of
-        fwd -> {start_key, end_key};
-        rev -> {end_key, start_key}
+    StartKeyOpts = case {StartKey, StartKeyDocId} of
+        {undefined, _} ->
+            [];
+        {StartKey, undefined} ->
+            [{start_key, {StartKey}}];
+        {_, _} ->
+            [{start_key, {StartKey, StartKeyDocId}}]
     end,
 
-    {Start0, End0} = couch_views_fdb:get_map_range_keys(Db, Sig, IdxName),
-
-    Start1 = case maps:is_key(MapStartKey, Args) of
-        false ->
-            Start0;
-        true ->
-            StartKey = maps:get(MapStartKey, Args),
-            Start = couch_views_fdb:get_map_index_key(Db, Sig, IdxName,
-                StartKey),
-            erlfdb_key:first_greater_or_equal(Start)
-    end,
-
-    End1 = case maps:is_key(MapEndKey, Args) of
-        false ->
-            End0;
-        true ->
-            EndKey = maps:get(MapEndKey, Args),
-            EndBin = couch_views_fdb:get_map_index_key(Db, Sig, IdxName,
-                EndKey),
-            EndBin1 = case InclusiveEnd of
-                true -> <<EndBin/binary, 16#FF>>;
-                false -> EndBin
-            end,
-            erlfdb_key:first_greater_than(EndBin1)
-    end,
-    {Start1, End1}.
-
-
-get_unpack_fun(TxDb, Opts, Callback) ->
-    UnPackFwd = fun({K, V}, State) ->
-        case couch_views_fdb:unpack_map_row(TxDb, K, V) of
-            {key, _Id, RowKey} ->
-                State#{current_key => RowKey};
-            {value, Id, RowValue} ->
-                #{
-                    current_key := RowKey
-                } = State,
-                process_map_row(Id, RowKey, RowValue, State, Callback)
-        end
+    EndKeyOpts = case {EndKey, EndKeyDocId} of
+        {undefined, _} ->
+            [];
+        {EndKey, undefined} when InclusiveEnd ->
+            [{end_key, {EndKey}}];
+        {EndKey, undefined} ->
+            [{end_key_gt, {EndKey}}];
+        {EndKey, EndKeyDocId} when InclusiveEnd ->
+            [{end_key, {EndKey, EndKeyDocId}}];
+        {EndKey, EndKeyDocId} ->
+            [{end_key_gt, {EndKey, EndKeyDocId}}]
     end,
 
-    UnPackRev = fun({K, V}, State) ->
-        case couch_views_fdb:unpack_map_row(TxDb, K, V) of
-            {key, Id, RowKey} ->
-                #{
-                    current_value := RowValue
-                } = State,
-                process_map_row(Id, RowKey, RowValue, State, Callback);
-            {value, _Id, RowValue} ->
-                State#{current_value => RowValue}
-        end
-    end,
-
-    case lists:keyfind(reverse, 1, Opts) of
-        {reverse, true} -> UnPackRev;
-        _ -> UnPackFwd
-    end.
+    [
+        {dir, Direction},
+        {limit, Limit * 2},
+        {streaming_mode, want_all}
+    ] ++ StartKeyOpts ++ EndKeyOpts.
 
 
-process_map_row(Id, RowKey, RowValue, State, Callback) ->
-    #{
-        acc := Acc,
-        skip := Skip,
-        db := Db
-    } = State,
-
-    case Skip > 0 of
-        true ->
-            State#{skip := Skip -1};
-        false ->
-            Row = [{id, Id}, {key, RowKey}, {value, RowValue}],
-
-            IncludeDoc = maps:get(include_docs, State, false),
-            Row1 = maybe_include_doc(Db, Id, Row, IncludeDoc),
-
-            {ok, AccNext} = Callback({row, Row1}, Acc),
-            State#{acc := AccNext}
-    end.
-
-
-maybe_include_doc(_Db, _Id, Row, false) ->
-    Row;
-
-maybe_include_doc(Db, Id, Row, true) ->
-    Doc1 = case fabric2_db:open_doc(Db, Id) of
-        {ok, Doc} -> couch_doc:to_json_obj(Doc, []);
-        {not_found, _} -> []
-    end,
-    Row ++ [{doc, Doc1}].
+maybe_stop({ok, Acc}) -> Acc;
+maybe_stop({stop, Acc}) -> throw({done, Acc}).


[couchdb] 17/25: Fix code and tests to pass eunit

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

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

commit 55214bb1ca7742b1b7bd50d032b9c311fd29c675
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Mon Jul 22 16:23:44 2019 -0500

    Fix code and tests to pass eunit
    
    Earlier code was just what I had gotten for moving logic around and
    simplfiying a bit or two. Now things compile and most tests pass. I've
    still got two tests to fix for map query arguments.
---
 src/couch_views/src/couch_views.erl                |  37 +-
 src/couch_views/src/couch_views_encoding.erl       |   2 +-
 src/couch_views/src/couch_views_fdb.erl            |  40 +-
 src/couch_views/src/couch_views_indexer.erl        |  37 +-
 src/couch_views/src/couch_views_jobs.erl           |  19 +-
 src/couch_views/src/couch_views_reader.erl         |  65 ++-
 src/couch_views/src/couch_views_sup.erl            |   4 +-
 src/couch_views/test/couch_views_encoding_test.erl |  19 +-
 src/couch_views/test/couch_views_indexer_test.erl  | 580 ++++++++++++++-------
 src/couch_views/test/couch_views_map_test.erl      |  56 +-
 10 files changed, 570 insertions(+), 289 deletions(-)

diff --git a/src/couch_views/src/couch_views.erl b/src/couch_views/src/couch_views.erl
index e10675b..e619a67 100644
--- a/src/couch_views/src/couch_views.erl
+++ b/src/couch_views/src/couch_views.erl
@@ -19,7 +19,7 @@
 -include("couch_mrview/include/couch_mrview.hrl").
 
 
-query(Db, DDoc, ViewName, Callback, Acc0, QueryArgs0) ->
+query(Db, DDoc, ViewName, Callback, Acc0, Args0) ->
     case fabric2_db:is_users_db(Db) of
         true ->
             fabric2_users_db:after_doc_read(DDoc, Db);
@@ -34,20 +34,20 @@ query(Db, DDoc, ViewName, Callback, Acc0, QueryArgs0) ->
         views = Views
     } = Mrst,
 
-    View = get_view(ViewName, Views),
-    QueryArgs1 = couch_mrview_util:set_view_type(QueryArgs0, View, Views),
-    QueryArgs2 = couch_mrview_util:validate_args(QueryArgs1),
-    case is_reduce_view(QueryArgs2) of
+    Args1 = to_mrargs(Args0),
+    Args2 = couch_mrview_util:set_view_type(Args1, ViewName, Views),
+    Args3 = couch_mrview_util:validate_args(Args2),
+    case is_reduce_view(Args3) of
         true -> throw({not_implemented});
         false -> ok
     end,
 
-    ok = maybe_update_view(Db, Mrst, QueryArgs2),
+    ok = maybe_update_view(Db, Mrst, Args3),
 
     try
-        couch_views_reader:read(Db, Mrst, ViewName, Callback, Acc0, QueryArgs2)
+        couch_views_reader:read(Db, Mrst, ViewName, Callback, Acc0, Args3)
     after
-        UpdateAfter = QueryArgs2#mrargs.update == lazy,
+        UpdateAfter = Args3#mrargs.update == lazy,
         if UpdateAfter == false -> ok; true ->
             couch_views_jobs:build_view_async(Db, Mrst)
         end
@@ -75,14 +75,21 @@ maybe_update_view(Db, Mrst, _Args) ->
     end.
 
 
-get_view(ViewName, Views) ->
-    {value, View} = lists:search(fun(View) ->
-        lists:member(ViewName, View#mrview.map_names)
-    end, Views),
-    View.
-
-
 is_reduce_view(#mrargs{view_type = ViewType}) ->
     ViewType =:= red;
 is_reduce_view({Reduce, _, _}) ->
     Reduce =:= red.
+
+
+to_mrargs(#mrargs{} = Args) ->
+    Args;
+
+to_mrargs(#{} = Args) ->
+    Fields = record_info(fields, mrargs),
+    Indexes = lists:seq(2, record_info(size, mrargs)),
+    LU = lists:zip(Fields, Indexes),
+
+    maps:fold(fun(Key, Value, Acc) ->
+        Index = fabric2_util:get_value(couch_util:to_existing_atom(Key), LU),
+        setelement(Index, Acc, Value)
+    end, #mrargs{}, Args).
diff --git a/src/couch_views/src/couch_views_encoding.erl b/src/couch_views/src/couch_views_encoding.erl
index 9f76ea6..f80ddfe 100644
--- a/src/couch_views/src/couch_views_encoding.erl
+++ b/src/couch_views/src/couch_views_encoding.erl
@@ -30,7 +30,7 @@
 
 
 encode(X) ->
-    encode_int(X, value).
+    encode(X, value).
 
 
 encode(X, Type) when Type == key; Type == value ->
diff --git a/src/couch_views/src/couch_views_fdb.erl b/src/couch_views/src/couch_views_fdb.erl
index dc1840d..16da0fc 100644
--- a/src/couch_views/src/couch_views_fdb.erl
+++ b/src/couch_views/src/couch_views_fdb.erl
@@ -21,6 +21,10 @@
     write_doc/4
 ]).
 
+-ifdef(TEST).
+-compile(export_all).
+-compile(nowarn_export_all).
+-endif.
 
 -define(LIST_VALUE, 0).
 -define(JSON_VALUE, 1).
@@ -88,10 +92,14 @@ fold_map_idx(TxDb, Sig, ViewId, Options, Callback, Acc0) ->
             {fun fold_rev/2, RevAcc}
     end,
 
-    fabric2_db:fold_range(TxDb, MapIdxPrefix, Fun, Acc, Options).
+    #{
+        acc := Acc1
+    } = fabric2_fdb:fold_range(TxDb, MapIdxPrefix, Fun, Acc, Options),
+
+    Acc1.
 
 
-write_doc(TxDb, Sig, #{deleted := true} = Doc, _ViewIds) ->
+write_doc(TxDb, Sig, _ViewIds, #{deleted := true} = Doc) ->
     #{
         id := DocId
     } = Doc,
@@ -103,7 +111,7 @@ write_doc(TxDb, Sig, #{deleted := true} = Doc, _ViewIds) ->
         clear_map_idx(TxDb, Sig, ViewId, DocId, ViewKeys)
     end, ExistingViewKeys);
 
-write_doc(TxDb, Sig, Doc, ViewIds) ->
+write_doc(TxDb, Sig, ViewIds, Doc) ->
     #{
         id := DocId,
         results := Results
@@ -201,7 +209,7 @@ fold_rev({RowKey, EncodedOriginalKey}, #{next := key} = Acc) ->
     UserAcc1 = UserCallback(DocId, Key, Value, UserAcc0),
 
     Acc#{
-        next := val,
+        next := value,
         value := undefined,
         sort_key := undefined,
         docid := undefined,
@@ -209,6 +217,7 @@ fold_rev({RowKey, EncodedOriginalKey}, #{next := key} = Acc) ->
         acc := UserAcc1
     }.
 
+
 clear_id_idx(TxDb, Sig, DocId) ->
     #{
         tx := Tx,
@@ -239,7 +248,9 @@ update_id_idx(TxDb, Sig, ViewId, DocId, NewRows) ->
 
     Unique = lists:usort([K || {K, _V} <- NewRows]),
 
-    Key = id_idx_key(DbPrefix, Sig, ViewId, DocId),
+    couch_log:error("Updating ID index: ~p ~p ~p ~p", [ViewId, DocId, NewRows, Unique]),
+
+    Key = id_idx_key(DbPrefix, Sig, DocId, ViewId),
     Val = couch_views_encoding:encode(Unique),
     ok = erlfdb:set(Tx, Key, Val).
 
@@ -264,8 +275,8 @@ update_map_idx(TxDb, Sig, ViewId, DocId, ExistingKeys, NewRows) ->
     lists:foreach(fun({DupeId, Key1, Key2, Val}) ->
         KK = map_idx_key(MapIdxPrefix, {Key1, DocId}, DupeId, ?VIEW_ROW_KEY),
         VK = map_idx_key(MapIdxPrefix, {Key1, DocId}, DupeId, ?VIEW_ROW_VALUE),
-        ok = erlfdn:store(Tx, KK, Key2),
-        ok = erlfdb:store(Tx, VK, Val)
+        ok = erlfdb:set(Tx, KK, Key2),
+        ok = erlfdb:set(Tx, VK, Val)
     end, KVsToAdd).
 
 
@@ -305,12 +316,12 @@ map_idx_prefix(DbPrefix, Sig, ViewId) ->
 
 map_idx_key(MapIdxPrefix, MapKey, DupeId, Type) ->
     Key = {MapKey, DupeId, Type},
-    erldb_tuple:encode(Key, MapIdxPrefix).
+    erlfdb_tuple:pack(Key, MapIdxPrefix).
 
 
 map_idx_range(DbPrefix, Sig, ViewId, MapKey, DocId) ->
     Encoded = couch_views_encoding:encode(MapKey, key),
-    Key = {?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, ViewId, Encoded, DocId},
+    Key = {?DB_VIEWS, Sig, ?VIEW_MAP_RANGE, ViewId, {Encoded, DocId}},
     erlfdb_tuple:range(Key, DbPrefix).
 
 
@@ -326,10 +337,9 @@ process_rows(Rows) ->
         dict:append(K1, {K2, V}, Acc)
     end, dict:new(), Encoded),
 
-    {_, Labeled} = dict:fold(fun(K1, Vals) ->
-        lists:foldl(fun({K2, V}, {Count, Acc}) ->
+    dict:fold(fun(K1, Vals, DAcc) ->
+        {_, Labeled} = lists:foldl(fun({K2, V}, {Count, Acc}) ->
             {Count + 1, [{Count, K1, K2, V} | Acc]}
-        end, {0, []}, Vals)
-    end, [], Grouped),
-
-    Labeled.
+        end, {0, []}, Vals),
+        Labeled ++ DAcc
+    end, [], Grouped).
diff --git a/src/couch_views/src/couch_views_indexer.erl b/src/couch_views/src/couch_views_indexer.erl
index 91072a1..decec42 100644
--- a/src/couch_views/src/couch_views_indexer.erl
+++ b/src/couch_views/src/couch_views_indexer.erl
@@ -36,19 +36,24 @@ spawn_link() ->
 
 init() ->
     {ok, Job, Data} = couch_jobs:accept(?INDEX_JOB_TYPE, #{}),
+    couch_log:error("XKCD: GOT JOB: ~p~n", [Data]),
 
     #{
         <<"db_name">> := DbName,
         <<"ddoc_id">> := DDocId,
-        <<"sig">> := Sig
+        <<"sig">> := JobSig
     } = Data,
 
     {ok, Db} = fabric2_db:open(DbName, []),
     {ok, DDoc} = fabric2_db:open_doc(Db, DDocId),
     {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+    HexSig = fabric2_util:to_hex(Mrst#mrst.sig),
 
-    if Mrst#mrst.sig == Sig -> ok; true ->
-        couch_jobs:finish(Job, Data#{error => mismatched_signature}),
+    if  HexSig == JobSig -> ok; true ->
+        couch_jobs:finish(undefined, Job, Data#{
+            error => sig_changed,
+            reason => <<"Design document was modified">>
+        }),
         exit(normal)
     end,
 
@@ -74,10 +79,12 @@ update(#{} = Db, Mrst0, State0) ->
         % to populate our db and view sequences
         State1 = case State0 of
             #{db_seq := undefined} ->
+                ViewSeq = couch_views_fdb:get_update_seq(TxDb, Mrst0),
                 State0#{
                     tx_db := TxDb,
                     db_seq := fabric2_db:get_update_seq(TxDb),
-                    view_seq := couch_views_fdb:get_update_seq(TxDb, Mrst0)
+                    view_seq := ViewSeq,
+                    last_seq := ViewSeq
                 };
             _ ->
                 State0#{
@@ -95,7 +102,7 @@ update(#{} = Db, Mrst0, State0) ->
         } = State2,
 
         {Mrst1, MappedDocs} = map_docs(Mrst0, DocAcc),
-        write_docs(Db, Mrst1, MappedDocs, State2),
+        write_docs(TxDb, Mrst1, MappedDocs, State2),
 
         case Count < Limit of
             true ->
@@ -107,8 +114,7 @@ update(#{} = Db, Mrst0, State0) ->
                     tx_db := undefined,
                     count := 0,
                     doc_acc := [],
-                    view_seq := LastSeq,
-                    last_seq := undefined
+                    view_seq := LastSeq
                 }}
         end
     end),
@@ -208,7 +214,7 @@ write_docs(TxDb, Mrst, Docs, State) ->
         couch_views_fdb:write_doc(TxDb, Sig, ViewIds, Doc)
     end, Docs),
 
-    couch_views_fdb:update_view_seq(TxDb, Sig, LastSeq).
+    couch_views_fdb:set_update_seq(TxDb, Sig, LastSeq).
 
 
 start_query_server(#mrst{} = Mrst) ->
@@ -231,7 +237,20 @@ report_progress(State, UpdateType) ->
         last_seq := LastSeq
     } = State,
 
-    NewData = JobData#{view_seq => LastSeq},
+    #{
+        <<"db_name">> := DbName,
+        <<"ddoc_id">> := DDocId,
+        <<"sig">> := Sig
+    } = JobData,
+
+    % Reconstruct from scratch to remove any
+    % possible existing error state.
+    NewData = #{
+        <<"db_name">> => DbName,
+        <<"ddoc_id">> => DDocId,
+        <<"sig">> => Sig,
+        <<"view_seq">> => LastSeq
+    },
 
     case UpdateType of
         update ->
diff --git a/src/couch_views/src/couch_views_jobs.erl b/src/couch_views/src/couch_views_jobs.erl
index 9e299af..85264c6 100644
--- a/src/couch_views/src/couch_views_jobs.erl
+++ b/src/couch_views/src/couch_views_jobs.erl
@@ -49,7 +49,7 @@ wait_for_job(JobId, UpdateSeq) ->
             wait_for_job(JobId, Subscription, UpdateSeq);
         {ok, finished, Data} ->
             case Data of
-                #{view_seq := ViewSeq} when ViewSeq >= UpdateSeq ->
+                #{<<"view_sig">> := ViewSeq} when ViewSeq >= UpdateSeq ->
                     ok;
                 _ ->
                     retry
@@ -61,11 +61,13 @@ wait_for_job(JobId, Subscription, UpdateSeq) ->
     case wait(Subscription) of
         {error, Error} ->
             erlang:error(Error);
-        {finished, #{view_seq := ViewSeq}} when ViewSeq >= UpdateSeq ->
+        {finished, #{<<"error">> := Error, <<"reason">> := Reason}} ->
+            erlang:error({binary_to_atom(Error, latin1), Reason});
+        {finished, #{<<"view_seq">> := ViewSeq}} when ViewSeq >= UpdateSeq ->
             ok;
         {finished, _} ->
             wait_for_job(JobId, UpdateSeq);
-        {_State, #{view_seq := ViewSeq}} when ViewSeq >= UpdateSeq ->
+        {_State, #{<<"view_seq">> := ViewSeq}} when ViewSeq >= UpdateSeq ->
             couch_jobs:unsubscribe(Subscription),
             ok;
         {_, _} ->
@@ -77,7 +79,8 @@ job_id(#{name := DbName}, #mrst{sig = Sig}) ->
     job_id(DbName, Sig);
 
 job_id(DbName, Sig) ->
-    <<DbName/binary, Sig/binary>>.
+    HexSig = fabric2_util:to_hex(Sig),
+    <<DbName/binary, "-", HexSig/binary>>.
 
 
 job_data(Db, Mrst) ->
@@ -89,12 +92,14 @@ job_data(Db, Mrst) ->
     #{
         db_name => fabric2_db:name(Db),
         ddoc_id => DDocId,
-        sig => Sig
+        sig => fabric2_util:to_hex(Sig)
     }.
 
 
 wait(Subscription) ->
     case couch_jobs:wait(Subscription, infinity) of
-        {?INDEX_JOB_TYPE, _JobId, JobState, JobData} -> {JobState, JobData};
-        timeout -> {error, timeout}
+        {?INDEX_JOB_TYPE, _JobId, JobState, JobData} ->
+            {JobState, JobData};
+        timeout ->
+            {error, timeout}
     end.
diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index 8d2bf5a..41a37eb 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -25,11 +25,12 @@
 
 read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
     #mrst{
+        language = Lang,
         sig = Sig,
         views = Views
     } = Mrst,
 
-    ViewId = get_view_id(ViewName, Views),
+    ViewId = get_view_id(Lang, Args, ViewName, Views),
     Opts = mrargs_to_fdb_options(Args),
     Fun = fun handle_row/4,
 
@@ -59,8 +60,7 @@ read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
             #{
                 acc := UserAcc2
             } = Acc1,
-
-            maybe_stop(UserCallback(complete, UserAcc2))
+            {ok, maybe_stop(UserCallback(complete, UserAcc2))}
         end)
     catch throw:{done, Out} ->
         {ok, Out}
@@ -68,7 +68,7 @@ read(Db, Mrst, ViewName, UserCallback, UserAcc0, Args) ->
 
 
 handle_row(_DocId, _Key, _Value, #{skip := Skip} = Acc) when Skip > 0 ->
-    {ok, Acc#{skip := Skip - 1}};
+    Acc#{skip := Skip - 1};
 
 handle_row(DocId, Key, Value, Acc) ->
     #{
@@ -88,7 +88,7 @@ handle_row(DocId, Key, Value, Acc) ->
         DocOpts0 = Args#mrargs.doc_options,
         DocOpts1 = DocOpts0 ++ case Args#mrargs.conflicts of
             true -> [conflicts];
-            false -> []
+            _ -> []
         end,
         DocObj = case fabric2_db:open_doc(TxDb, DocId, DocOpts1) of
             {ok, Doc} -> couch_doc:to_json_obj(Doc, DocOpts1);
@@ -101,49 +101,60 @@ handle_row(DocId, Key, Value, Acc) ->
     Acc#{acc := UserAcc1}.
 
 
-get_view_id(ViewName, Views) ->
-    {value, View} = lists:search(fun(View) ->
-        lists:member(ViewName, View#mrview.map_names)
-    end, Views),
-    View#mrview.id_num.
+get_view_id(Lang, Args, ViewName, Views) ->
+    case couch_mrview_util:extract_view(Lang, Args, ViewName, Views) of
+        {map, View, _Args} -> View#mrview.id_num;
+        {red, {_Idx, _Lang, View}} -> View#mrview.id_num
+    end.
 
 
 mrargs_to_fdb_options(Args) ->
     #mrargs{
-        start_key = StartKey,
+        start_key = StartKey0,
         start_key_docid = StartKeyDocId,
-        end_key = EndKey,
+        end_key = EndKey0,
         end_key_docid = EndKeyDocId,
         direction = Direction,
         limit = Limit,
+        skip = Skip,
         inclusive_end = InclusiveEnd
     } = Args,
 
-    StartKeyOpts = case {StartKey, StartKeyDocId} of
+    StartKey1 = if StartKey0 == undefined -> undefined; true ->
+        couch_views_encoding:encode(StartKey0)
+    end,
+
+    StartKeyOpts = case {StartKey1, StartKeyDocId} of
         {undefined, _} ->
             [];
-        {StartKey, undefined} ->
-            [{start_key, {StartKey}}];
-        {_, _} ->
-            [{start_key, {StartKey, StartKeyDocId}}]
+        {StartKey1, StartKeyDocId} ->
+            [{start_key, {StartKey1, StartKeyDocId}}]
+    end,
+
+    EndKey1 = if EndKey0 == undefined -> undefined; true ->
+        couch_views_encoding:encode(EndKey0)
     end,
 
-    EndKeyOpts = case {EndKey, EndKeyDocId} of
+    EndKeyOpts = case {EndKey1, EndKeyDocId} of
         {undefined, _} ->
             [];
-        {EndKey, undefined} when InclusiveEnd ->
-            [{end_key, {EndKey}}];
-        {EndKey, undefined} ->
-            [{end_key_gt, {EndKey}}];
-        {EndKey, EndKeyDocId} when InclusiveEnd ->
-            [{end_key, {EndKey, EndKeyDocId}}];
-        {EndKey, EndKeyDocId} ->
-            [{end_key_gt, {EndKey, EndKeyDocId}}]
+        {EndKey1, <<255>>} when not InclusiveEnd ->
+            % When inclusive_end=false we need to
+            % elide the default end_key_docid so as
+            % to not sort past the docids with the
+            % given end key.
+            [{end_key_gt, {EndKey1}}];
+        {EndKey1, EndKeyDocId} when not InclusiveEnd ->
+            [{end_key_gt, {EndKey1, EndKeyDocId}}];
+        {EndKey1, EndKeyDocId} when InclusiveEnd ->
+            [{end_key, {EndKey1, EndKeyDocId}}];
+        {EndKey1, EndKeyDocId} when InclusiveEnd ->
+            [{end_key_gt, {EndKey1, EndKeyDocId}}]
     end,
 
     [
         {dir, Direction},
-        {limit, Limit * 2},
+        {limit, Limit * 2 + Skip * 2},
         {streaming_mode, want_all}
     ] ++ StartKeyOpts ++ EndKeyOpts.
 
diff --git a/src/couch_views/src/couch_views_sup.erl b/src/couch_views/src/couch_views_sup.erl
index da7d796..7650fdf 100644
--- a/src/couch_views/src/couch_views_sup.erl
+++ b/src/couch_views/src/couch_views_sup.erl
@@ -39,8 +39,8 @@ init([]) ->
     },
     Children = [
         #{
-            id => couch_views_worker_server,
-            start => {couch_views_worker_server, start_link, []}
+            id => couch_views_server,
+            start => {couch_views_server, start_link, []}
         }
     ],
     {ok, {Flags, Children}}.
diff --git a/src/couch_views/test/couch_views_encoding_test.erl b/src/couch_views/test/couch_views_encoding_test.erl
index a73cb42..9282265 100644
--- a/src/couch_views/test/couch_views_encoding_test.erl
+++ b/src/couch_views/test/couch_views_encoding_test.erl
@@ -35,12 +35,7 @@ correct_ordering_test() ->
         false,
         true,
 
-        %  Then numbers
-        % 1,
-        % 2,
-        % 3.0,
-        % 4,
-
+        % Then numbers
         1.0,
         2.0,
         3.0,
@@ -63,11 +58,11 @@ correct_ordering_test() ->
         % Member order does matter for collation
         {[{<<"b">>, 2.0}, {<<"a">>, 1.0}]},
         {[{<<"b">>, 2.0}, {<<"c">>, 2.0}]}
-
     ],
 
-    BinList = lists:map(fun couch_views_encoding:encode/1, Ordered),
-    SortedBinList = lists:sort(BinList),
-    DecodedBinList = lists:map(fun couch_views_encoding:decode/1,
-        SortedBinList),
-    ?assertEqual(Ordered, DecodedBinList).
+    BinList = [couch_views_encoding:encode(O, key) || O <- Ordered],
+    Random = [{rand:uniform(), Bin} || Bin <- BinList],
+    {_, Unsorted} = lists:unzip(lists:sort(Random)),
+    Sorted = lists:sort(Unsorted),
+    Decoded = [couch_views_encoding:decode(O) || O <- Sorted],
+    ?assertEqual(Ordered, Decoded).
diff --git a/src/couch_views/test/couch_views_indexer_test.erl b/src/couch_views/test/couch_views_indexer_test.erl
index 2d192a6..e6dfdc4 100644
--- a/src/couch_views/test/couch_views_indexer_test.erl
+++ b/src/couch_views/test/couch_views_indexer_test.erl
@@ -12,232 +12,396 @@
 
 -module(couch_views_indexer_test).
 
--include_lib("couch/include/couch_eunit.hrl").
 -include_lib("eunit/include/eunit.hrl").
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("couch/include/couch_eunit.hrl").
 -include_lib("couch_mrview/include/couch_mrview.hrl").
 
 
--define(TDEF(A), {atom_to_list(A), fun A/0}).
-
-setup() ->
-    test_util:start_couch([fabric]).
-
-
-teardown(State) ->
-    test_util:stop_couch(State).
-
-
-foreach_setup() ->
-    ok.
+-define(I_HEART_EUNIT(Tests), [{with, [T]} || T <- Tests]).
 
 
-foreach_teardown(_) ->
-    meck:unload().
-
-
-index_server_test_() ->
+indexer_test_() ->
     {
-        "Test Couch Views indexer",
+        "Test view indexing",
         {
             setup,
             fun setup/0,
-            fun teardown/1,
+            fun cleanup/1,
             {
                 foreach,
-                fun foreach_setup/0, fun foreach_teardown/1,
-                [
-                    ?TDEF(map_docs_no_results_for_deleted),
-                    ?TDEF(map_docs_returns_sorted_results),
-                    ?TDEF(write_doc_clears_for_deleted_doc),
-                    ?TDEF(write_doc_adds_for_new_doc),
-                    ?TDEF(write_doc_clears_and_sets_for_update),
-                    ?TDEF(write_doc_clears_for_no_new_update),
-                    ?TDEF(write_doc_clears_and_updates_duplicates)
-                ]
+                fun foreach_setup/0,
+                fun foreach_teardown/1,
+                ?I_HEART_EUNIT([
+                    fun indexed_empty_db/1,
+                    fun indexed_single_doc/1,
+                    fun updated_docs_are_reindexed/1,
+                    fun updated_docs_without_changes_are_reindexed/1,
+                    fun deleted_docs_not_indexed/1,
+                    fun deleted_docs_are_unindexed/1,
+                    fun multipe_docs_with_same_key/1,
+                    fun multipe_keys_from_same_doc/1,
+                    fun multipe_identical_keys_from_same_doc/1
+                ])
             }
-
         }
     }.
 
 
-map_docs_no_results_for_deleted() ->
-    DbName = ?tempdb,
+setup() ->
+    Ctx = test_util:start_couch([
+            fabric,
+            couch_jobs,
+            couch_views
+        ]),
+    Ctx.
+
+
+cleanup(Ctx) ->
+    test_util:stop_couch(Ctx).
 
+
+foreach_setup() ->
+    {ok, Db} = fabric2_db:create(?tempdb(), [{user_ctx, ?ADMIN_USER}]),
+    Db.
+
+
+foreach_teardown(Db) ->
+    ok = fabric2_db:delete(fabric2_db:name(Db), []).
+
+
+indexed_empty_db(Db) ->
     DDoc = create_ddoc(),
-    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
 
-    Doc = #{
-        id => <<"id">>,
-        sequence => <<1111>>,
-        rev_id => <<"1-123">>,
-        deleted => true
-    },
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, Out} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
 
-    meck:expect(couch_query_servers, start_doc_map, fun(_, _, _) ->
-        {ok, fake}
-    end),
+    ?assertEqual([], Out).
 
-    {Results, _} = couch_views_indexer:map_docs(Mrst, [Doc]),
 
-    [#{results := DocResult}] = Results,
-    ?assertEqual([], DocResult).
+indexed_single_doc(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0),
 
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, _} = fabric2_db:update_doc(Db, Doc1, []),
 
-map_docs_returns_sorted_results() ->
-    DbName = ?tempdb,
-    Doc = #{
-        id => <<"id">>,
-        sequence => <<1111>>,
-        rev_id => <<"1-123">>,
-        doc => doc(1)
-    },
+    {ok, Out} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
 
-    CompleteResult = [[{1, 1}], []],
+    ?assertEqual([{row, [
+            {id, <<"0">>},
+            {key, 0},
+            {value, 0}
+        ]}], Out).
 
+
+updated_docs_are_reindexed(Db) ->
     DDoc = create_ddoc(),
+    Doc1 = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, {Pos, Rev}} = fabric2_db:update_doc(Db, Doc1, []),
+
+    {ok, Out1} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([{row, [
+            {id, <<"0">>},
+            {key, 0},
+            {value, 0}
+        ]}], Out1),
+
+    Doc2 = Doc1#doc{
+        revs = {Pos, [Rev]},
+        body = {[{<<"val">>, 1}]}
+    },
+    {ok, _} = fabric2_db:update_doc(Db, Doc2, []),
+
+    {ok, Out2} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([{row, [
+            {id, <<"0">>},
+            {key, 1},
+            {value, 1}
+        ]}], Out2),
+
+    % Check that our id index is updated properly
+    % as well.
+    DbName = fabric2_db:name(Db),
     {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+    Sig = Mrst#mrst.sig,
+    Expect = [{0, [1]}, {1, []}],
+    fabric2_fdb:transactional(Db, fun(TxDb) ->
+        ?assertEqual(
+                Expect,
+                couch_views_fdb:get_view_keys(TxDb, Sig, <<"0">>)
+            )
+    end).
 
 
-    {Results, _} = couch_views_indexer:map_docs(Mrst, [Doc]),
-    [#{results := DocResult}] = Results,
-    ?assertEqual(CompleteResult, DocResult).
-
-
-write_doc_clears_for_deleted_doc() ->
-    TxDb = #{},
-    Sig = <<123>>,
-    Doc = #{deleted => true, id => 1},
-    ViewIds = [1],
-    OldIdxKey = old_key,
-
-    meck:expect(couch_views_fdb, get_id_index, 4, old_key),
-    meck:expect(couch_views_fdb, clear_id_index, 4, ok),
-    meck:expect(couch_views_fdb, clear_map_index, 5, ok),
-
-    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
-    ?assert(meck:called(couch_views_fdb, get_id_index, [TxDb, Sig, 1, 1])),
-    ?assert(meck:called(couch_views_fdb, clear_id_index, [TxDb, Sig, 1, 1])),
-    ?assert(meck:called(couch_views_fdb, clear_map_index,
-        [TxDb, Sig, 1, 1, OldIdxKey])),
-    ?assertEqual(length(meck:history(couch_views_fdb)), 3).
-
-
-write_doc_adds_for_new_doc() ->
-    TxDb = #{},
-    Sig = <<123>>,
-    Key = <<"key">>,
-    Value = 1,
-    Results = [{Key, Value}],
-    Doc = #{
-        deleted => false,
-        id => 1,
-        results => [Results]
-    },
-    ViewIds = [1],
-
-    meck:expect(couch_views_fdb, get_id_index, 4, not_found),
-    meck:expect(couch_views_fdb, set_id_index, 5, ok),
-    meck:expect(couch_views_fdb, set_map_index_results, 5, ok),
-
-    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
-    ?assert(meck:called(couch_views_fdb, get_id_index, [TxDb, Sig, 1, 1])),
-    ?assert(meck:called(couch_views_fdb, set_id_index,
-        [TxDb, Sig, 1, 1, Key])),
-    ?assert(meck:called(couch_views_fdb, set_map_index_results,
-        [TxDb, Sig, 1, 1, Results])),
-    ?assertEqual(length(meck:history(couch_views_fdb)), 3).
-
-
-write_doc_clears_and_sets_for_update() ->
-    TxDb = #{},
-    Sig = <<123>>,
-    Key = <<"key">>,
-    Value = 1,
-    Results = [{Key, Value}],
-    Doc = #{
-        deleted => false,
-        id => 1,
-        results => [Results]
+updated_docs_without_changes_are_reindexed(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, {Pos, Rev}} = fabric2_db:update_doc(Db, Doc1, []),
+
+    {ok, Out1} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([{row, [
+            {id, <<"0">>},
+            {key, 0},
+            {value, 0}
+        ]}], Out1),
+
+    Doc2 = Doc1#doc{
+        revs = {Pos, [Rev]},
+        body = {[{<<"val">>, 0}]}
     },
-    ViewIds = [1],
-    OldKey = oldkey,
-
-    meck:expect(couch_views_fdb, get_id_index, 4, OldKey),
-    meck:expect(couch_views_fdb, clear_id_index, 4, ok),
-    meck:expect(couch_views_fdb, clear_map_index, 5, ok),
-    meck:expect(couch_views_fdb, set_id_index, 5, ok),
-    meck:expect(couch_views_fdb, set_map_index_results, 5, ok),
-
-    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
-    ?assert(meck:called(couch_views_fdb, get_id_index, [TxDb, Sig, 1, 1])),
-    ?assert(meck:called(couch_views_fdb, clear_id_index, [TxDb, Sig, 1, 1])),
-    ?assert(meck:called(couch_views_fdb, clear_map_index,
-        [TxDb, Sig, 1, 1, OldKey])),
-    ?assert(meck:called(couch_views_fdb, set_id_index,
-        [TxDb, Sig, 1, 1, Key])),
-    ?assert(meck:called(couch_views_fdb, set_map_index_results,
-        [TxDb, Sig, 1, 1, Results])),
-    ?assertEqual(length(meck:history(couch_views_fdb)), 5).
-
-
-write_doc_clears_for_no_new_update() ->
-    TxDb = #{},
-    Sig = <<123>>,
-    Results = [],
-    Doc = #{
-        deleted => false,
-        id => 1,
-        results => [Results]
+    {ok, _} = fabric2_db:update_doc(Db, Doc2, []),
+
+    {ok, Out2} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([{row, [
+            {id, <<"0">>},
+            {key, 0},
+            {value, 0}
+        ]}], Out2),
+
+    % Check fdb directly to make sure we've also
+    % removed the id idx keys properly.
+    DbName = fabric2_db:name(Db),
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+    Sig = Mrst#mrst.sig,
+    Expect = [{0, [0]}, {1, []}],
+    fabric2_fdb:transactional(Db, fun(TxDb) ->
+        ?assertEqual(
+                Expect,
+                couch_views_fdb:get_view_keys(TxDb, Sig, <<"0">>)
+            )
+    end).
+
+
+deleted_docs_not_indexed(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, {Pos, Rev}} = fabric2_db:update_doc(Db, Doc1, []),
+    Doc2 = Doc1#doc{
+        revs = {Pos, [Rev]},
+        deleted = true,
+        body = {[{<<"val">>, 1}]}
     },
-    ViewIds = [1],
-    OldKey = oldkey,
-
-    meck:expect(couch_views_fdb, get_id_index, 4, OldKey),
-    meck:expect(couch_views_fdb, clear_id_index, 4, ok),
-    meck:expect(couch_views_fdb, clear_map_index, 5, ok),
-
-    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
-    ?assert(meck:called(couch_views_fdb, get_id_index, [TxDb, Sig, 1, 1])),
-    ?assert(meck:called(couch_views_fdb, clear_id_index, [TxDb, Sig, 1, 1])),
-    ?assert(meck:called(couch_views_fdb, clear_map_index,
-        [TxDb, Sig, 1, 1, OldKey])),
-    ?assertEqual(length(meck:history(couch_views_fdb)), 3).
-
-
-write_doc_clears_and_updates_duplicates() ->
-    TxDb = #{},
-    Sig = <<123>>,
-    Key = <<"key">>,
-    Results = [{Key, 1}, {Key, 2}],
-    Doc = #{
-        deleted => false,
-        id => 1,
-        results => [Results]
+    {ok, _} = fabric2_db:update_doc(Db, Doc2, []),
+
+    {ok, Out} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([], Out).
+
+
+deleted_docs_are_unindexed(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, {Pos, Rev}} = fabric2_db:update_doc(Db, Doc1, []),
+
+    {ok, Out1} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([{row, [
+            {id, <<"0">>},
+            {key, 0},
+            {value, 0}
+        ]}], Out1),
+
+    Doc2 = Doc1#doc{
+        revs = {Pos, [Rev]},
+        deleted = true,
+        body = {[{<<"val">>, 1}]}
     },
-    ViewIds = [1],
-    OldKey = oldkey,
-
-    meck:expect(couch_views_fdb, get_id_index, 4, OldKey),
-    meck:expect(couch_views_fdb, clear_id_index, 4, ok),
-    meck:expect(couch_views_fdb, clear_map_index, 5, ok),
-    meck:expect(couch_views_fdb, set_id_index, 5, ok),
-    meck:expect(couch_views_fdb, set_map_index_results, 5, ok),
-
-    couch_views_indexer:write_doc(TxDb, Sig, Doc, ViewIds),
-    ?assertEqual(meck:num_calls(couch_views_fdb, get_id_index,
-        [TxDb, Sig, 1, 1]), 2),
-    ?assertEqual(meck:num_calls(couch_views_fdb, clear_id_index,
-        [TxDb, Sig, 1, 1]), 1),
-    ?assertEqual(meck:num_calls(couch_views_fdb, set_id_index,
-        [TxDb, Sig, 1, 1, Key]), 2),
-    ?assertEqual(meck:num_calls(couch_views_fdb, clear_map_index,
-        [TxDb, Sig, 1, 1, OldKey]), 1),
-    ?assertEqual(meck:num_calls(couch_views_fdb, set_map_index_results,
-        [TxDb, Sig, 1, 1, Results]), 2),
-    ?assertEqual(length(meck:history(couch_views_fdb)), 8).
+    {ok, _} = fabric2_db:update_doc(Db, Doc2, []),
+
+    {ok, Out2} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([], Out2),
+
+    % Check fdb directly to make sure we've also
+    % removed the id idx keys properly.
+    DbName = fabric2_db:name(Db),
+    {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
+    Sig = Mrst#mrst.sig,
+    fabric2_fdb:transactional(Db, fun(TxDb) ->
+        ?assertEqual([], couch_views_fdb:get_view_keys(TxDb, Sig, <<"0">>))
+    end).
+
+
+multipe_docs_with_same_key(Db) ->
+    DDoc = create_ddoc(),
+    Doc1 = doc(0, 1),
+    Doc2 = doc(1, 1),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, _} = fabric2_db:update_docs(Db, [Doc1, Doc2], []),
+
+    {ok, Out} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([
+            {row, [
+                {id, <<"0">>},
+                {key, 1},
+                {value, 1}
+            ]},
+            {row, [
+                {id, <<"1">>},
+                {key, 1},
+                {value, 1}
+            ]}
+        ], Out).
+
+
+multipe_keys_from_same_doc(Db) ->
+    DDoc = create_ddoc(multi_emit_different),
+    Doc = doc(0, 1),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, _} = fabric2_db:update_doc(Db, Doc, []),
+
+    {ok, Out} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([
+            {row, [
+                {id, <<"0">>},
+                {key, 1},
+                {value, 1}
+            ]},
+            {row, [
+                {id, <<"0">>},
+                {key, <<"0">>},
+                {value, <<"0">>}
+            ]}
+        ], Out).
+
+
+multipe_identical_keys_from_same_doc(Db) ->
+    DDoc = create_ddoc(multi_emit_same),
+    Doc = doc(0, 1),
+
+    {ok, _} = fabric2_db:update_doc(Db, DDoc, []),
+    {ok, _} = fabric2_db:update_doc(Db, Doc, []),
+
+    {ok, Out} = couch_views:query(
+            Db,
+            DDoc,
+            <<"map_fun1">>,
+            fun fold_fun/2,
+            [],
+            #mrargs{}
+        ),
+
+    ?assertEqual([
+            {row, [
+                {id, <<"0">>},
+                {key, 1},
+                {value, 2}
+            ]},
+            {row, [
+                {id, <<"0">>},
+                {key, 1},
+                {value, 1}
+            ]}
+        ], Out).
+
+
+fold_fun({meta, _Meta}, Acc) ->
+    {ok, Acc};
+fold_fun({row, _} = Row, Acc) ->
+    {ok, [Row | Acc]};
+fold_fun(complete, Acc) ->
+    {ok, lists:reverse(Acc)}.
 
 
 create_ddoc() ->
+    create_ddoc(simple).
+
+
+create_ddoc(simple) ->
     couch_doc:from_json_obj({[
         {<<"_id">>, <<"_design/bar">>},
         {<<"views">>, {[
@@ -248,11 +412,47 @@ create_ddoc() ->
                 {<<"map">>, <<"function(doc) {}">>}
             ]}}
         ]}}
+    ]});
+
+create_ddoc(multi_emit_different) ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, <<"_design/bar">>},
+        {<<"views">>, {[
+            {<<"map_fun1">>, {[
+                {<<"map">>, <<"function(doc) { "
+                    "emit(doc._id, doc._id); "
+                    "emit(doc.val, doc.val); "
+                "}">>}
+            ]}},
+            {<<"map_fun2">>, {[
+                {<<"map">>, <<"function(doc) {}">>}
+            ]}}
+        ]}}
+    ]});
+
+create_ddoc(multi_emit_same) ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, <<"_design/bar">>},
+        {<<"views">>, {[
+            {<<"map_fun1">>, {[
+                {<<"map">>, <<"function(doc) { "
+                    "emit(doc.val, doc.val * 2); "
+                    "emit(doc.val, doc.val); "
+                "}">>}
+            ]}},
+            {<<"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">>, Id}
+        {<<"val">>, Val}
     ]}).
diff --git a/src/couch_views/test/couch_views_map_test.erl b/src/couch_views/test/couch_views_map_test.erl
index e7be521..ab3000e 100644
--- a/src/couch_views/test/couch_views_map_test.erl
+++ b/src/couch_views/test/couch_views_map_test.erl
@@ -76,7 +76,7 @@ should_map() ->
 
 
 should_map_with_startkey() ->
-    Result = run_query(<<"baz">>, #{start_key => 4}),
+    Result = run_query(<<"baz">>, #{start_key => 4}, true),
     Expect = {ok, [
         {row, [{id, <<"4">>}, {key, 4}, {value, 4}]},
         {row, [{id, <<"5">>}, {key, 5}, {value, 5}]},
@@ -233,7 +233,7 @@ should_map_with_startkey_with_key_array() ->
 
 
 should_map_with_startkey_and_endkey_with_key_array() ->
-    Rows = [
+    Rows1 = [
         {row, [{id, <<"4">>}, {key, [<<"4">>, 4]}, {value, 4}]},
         {row, [{id, <<"5">>}, {key, [<<"5">>, 5]}, {value, 5}]},
         {row, [{id, <<"6">>}, {key, [<<"6">>, 6]}, {value, 6}]},
@@ -241,12 +241,21 @@ should_map_with_startkey_and_endkey_with_key_array() ->
         {row, [{id, <<"8">>}, {key, [<<"8">>, 8]}, {value, 8}]}
     ],
 
+    Rows2 = [
+        {row, [{id, <<"4">>}, {key, [<<"4">>, 4]}, {value, 4}]},
+        {row, [{id, <<"5">>}, {key, [<<"5">>, 5]}, {value, 5}]},
+        {row, [{id, <<"6">>}, {key, [<<"6">>, 6]}, {value, 6}]},
+        {row, [{id, <<"7">>}, {key, [<<"7">>, 7]}, {value, 7}]},
+        {row, [{id, <<"8">>}, {key, [<<"8">>, 8]}, {value, 8}]},
+        {row, [{id, <<"9">>}, {key, [<<"9">>, 9]}, {value, 9}]}
+    ],
+
     Result = run_query(<<"boom">>, #{
         start_key => [<<"4">>],
         end_key => [<<"8">>, []]
     }),
 
-    ?assertEqual({ok, Rows}, Result),
+    ?assertEqual({ok, Rows1}, Result),
 
     ResultRev = run_query(<<"boom">>, #{
         start_key => [<<"8">>, []],
@@ -254,7 +263,7 @@ should_map_with_startkey_and_endkey_with_key_array() ->
         direction => rev
     }),
 
-    ?assertEqual({ok, lists:reverse(Rows)}, ResultRev),
+    ?assertEqual({ok, lists:reverse(Rows1)}, ResultRev),
 
     ResultRev2 = run_query(<<"boom">>, #{
         start_key => [<<"9">>, 9],
@@ -263,7 +272,21 @@ should_map_with_startkey_and_endkey_with_key_array() ->
         inclusive_end => false
     }),
 
-    ?assertEqual({ok, lists:reverse(Rows)}, ResultRev2).
+    % Here, [<<"4">>] is less than [<<"4">>, 4] so we
+    % expect rows 9-4
+    ?assertEqual({ok, lists:reverse(Rows2)}, ResultRev2),
+
+    ResultRev2 = run_query(<<"boom">>, #{
+        start_key => [<<"9">>, 9],
+        end_key => [<<"4">>, 4],
+        direction => rev,
+        inclusive_end => false
+    }),
+
+    % Here, specifying [<<"4">>, 4] as the key will prevent
+    % us from including that row which leaves rows 9-5
+    ?assertEqual({ok, lists:reverse(lists:nthtail(1, Rows2))}, ResultRev2).
+
 
 
 should_map_empty_views() ->
@@ -330,7 +353,7 @@ should_map_update_is_false() ->
         start_key => 8
     },
 
-    Result1 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+    Result1 = couch_views:query(Db, DDoc, Idx, fun default_cb/2,
         [], Args1),
     ?assertEqual(Expect, Result1),
 
@@ -342,11 +365,11 @@ should_map_update_is_false() ->
         update => false
     },
 
-    Result2 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+    Result2 = couch_views:query(Db, DDoc, Idx, fun default_cb/2,
         [], Args2),
     ?assertEqual(Expect, Result2),
 
-    Result3 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+    Result3 = couch_views:query(Db, DDoc, Idx, fun default_cb/2,
         [], Args1),
     ?assertEqual(Expect1, Result3).
 
@@ -373,7 +396,7 @@ should_map_update_is_lazy() ->
         update => lazy
     },
 
-    Result1 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+    Result1 = couch_views:query(Db, DDoc, Idx, fun default_cb/2,
         [], Args1),
     ?assertEqual({ok, []}, Result1),
 
@@ -386,7 +409,7 @@ should_map_update_is_lazy() ->
         update => false
     },
 
-    Result2 = couch_views:map_query(Db, DDoc, Idx, fun default_cb/2,
+    Result2 = couch_views:query(Db, DDoc, Idx, fun default_cb/2,
         [], Args2),
     ?assertEqual(Expect, Result2).
 
@@ -413,12 +436,21 @@ should_map_update_is_lazy() ->
 
 
 run_query(Idx, Args) ->
+    run_query(Idx, Args, false).
+
+
+run_query(Idx, Args, DebugCluster) ->
     DbName = ?tempdb(),
     {ok, Db} = fabric2_db:create(DbName, [{user_ctx, ?ADMIN_USER}]),
     DDoc = create_ddoc(),
     Docs = make_docs(10),
     fabric2_db:update_docs(Db, [DDoc | Docs]),
-    couch_views:map_query(Db, DDoc, Idx, fun default_cb/2, [], Args).
+    if not DebugCluster -> ok; true ->
+        %% couch_views:query(Db, DDoc, Idx, fun default_cb/2, [], #{}),
+        %% fabric2_fdb:debug_cluster()
+        ok
+    end,
+    couch_views:query(Db, DDoc, Idx, fun default_cb/2, [], Args).
 
 
 default_cb(complete, Acc) ->
@@ -427,6 +459,8 @@ default_cb({final, Info}, []) ->
     {ok, [Info]};
 default_cb({final, _}, Acc) ->
     {ok, Acc};
+default_cb({meta, _}, Acc) ->
+    {ok, Acc};
 default_cb(ok, ddoc_updated) ->
     {ok, ddoc_updated};
 default_cb(Row, Acc) ->


[couchdb] 19/25: All tests passing

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

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

commit cefa11130a7fb094795d4746a4797ac5d6cc8866
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Tue Jul 23 11:51:49 2019 -0500

    All tests passing
---
 src/couch_views/src/couch_views_reader.erl    | 4 +---
 src/couch_views/test/couch_views_map_test.erl | 5 +++--
 2 files changed, 4 insertions(+), 5 deletions(-)

diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index 2deb9f6..25144ba 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -153,9 +153,7 @@ mrargs_to_fdb_options(Args) ->
         {EndKey1, EndKeyDocId, _} when not InclusiveEnd ->
             [{end_key_gt, {EndKey1, EndKeyDocId}}];
         {EndKey1, EndKeyDocId, _} when InclusiveEnd ->
-            [{end_key, {EndKey1, EndKeyDocId}}];
-        {EndKey1, EndKeyDocId, _} when InclusiveEnd ->
-            [{end_key_gt, {EndKey1, EndKeyDocId}}]
+            [{end_key, {EndKey1, EndKeyDocId}}]
     end,
 
     [
diff --git a/src/couch_views/test/couch_views_map_test.erl b/src/couch_views/test/couch_views_map_test.erl
index c5db382..8fe9ffd 100644
--- a/src/couch_views/test/couch_views_map_test.erl
+++ b/src/couch_views/test/couch_views_map_test.erl
@@ -401,8 +401,9 @@ should_map_update_is_lazy() ->
     ?assertEqual({ok, []}, Result1),
 
     {ok, Mrst} = couch_views_util:ddoc_to_mrst(DbName, DDoc),
-    {ok, Subscription, _, _} = couch_views_jobs:subscribe(Db, Mrst),
-    couch_jobs:wait(Subscription, finished, 1000),
+    JobId = couch_views_jobs:job_id(Db, Mrst),
+    UpdateSeq = fabric2_db:get_update_seq(Db),
+    ok = couch_views_jobs:wait_for_job(JobId, UpdateSeq),
 
     Args2 = #{
         start_key => 8,