You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@couchdb.apache.org by da...@apache.org on 2020/10/12 19:06:09 UTC

[couchdb] branch feat-disable-custom-reduce-functions created (now 9345a71)

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

davisp pushed a change to branch feat-disable-custom-reduce-functions
in repository https://gitbox.apache.org/repos/asf/couchdb.git.


      at 9345a71  Disable custom reduce functions by default

This branch includes the following new commits:

     new 9345a71  Disable custom reduce functions by default

The 1 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/01: Disable custom reduce functions by default

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

davisp pushed a commit to branch feat-disable-custom-reduce-functions
in repository https://gitbox.apache.org/repos/asf/couchdb.git

commit 9345a71c59e3c9e4b27cc37b8979a097031c6b8c
Author: Paul J. Davis <pa...@gmail.com>
AuthorDate: Mon Oct 12 14:08:01 2020 -0500

    Disable custom reduce functions by default
    
    This prevents users from creating custom reduce functions. Custom reduce
    functions are notoriously difficult to write correctly. This change
    disables them by default in the eventual 4.0 release.
---
 rel/overlay/etc/default.ini                        |   3 +
 src/chttpd/src/chttpd.erl                          |   2 +
 src/couch_views/src/couch_views_reader.erl         |  10 +-
 src/couch_views/src/couch_views_trees.erl          |   2 +-
 src/couch_views/src/couch_views_util.erl           |  34 +++-
 .../test/couch_views_custom_red_test.erl           | 194 +++++++++++++++++++++
 6 files changed, 240 insertions(+), 5 deletions(-)

diff --git a/rel/overlay/etc/default.ini b/rel/overlay/etc/default.ini
index 3a377c7..745a1c9 100644
--- a/rel/overlay/etc/default.ini
+++ b/rel/overlay/etc/default.ini
@@ -325,6 +325,9 @@ iterations = 10 ; iterations for password hashing
 
 ; Settings for view indexing
 [couch_views]
+; Enable custom reduce functions
+;custom_reduce_enabled = false
+
 ; Maximum acceptors waiting to accept view indexing jobs
 ;max_acceptors = 5
 ;
diff --git a/src/chttpd/src/chttpd.erl b/src/chttpd/src/chttpd.erl
index 1a9b19b..3100694 100644
--- a/src/chttpd/src/chttpd.erl
+++ b/src/chttpd/src/chttpd.erl
@@ -1017,6 +1017,8 @@ error_info(all_workers_died) ->
         "request due to overloading or maintenance mode.">>};
 error_info(not_implemented) ->
     {501, <<"not_implemented">>, <<"this feature is not yet implemented">>};
+error_info({disabled, Reason}) ->
+    {501, <<"disabled">>, Reason};
 error_info(timeout) ->
     {500, <<"timeout">>, <<"The request could not be processed in a reasonable"
         " amount of time.">>};
diff --git a/src/couch_views/src/couch_views_reader.erl b/src/couch_views/src/couch_views_reader.erl
index 3c58627..35ee8a0 100644
--- a/src/couch_views/src/couch_views_reader.erl
+++ b/src/couch_views/src/couch_views_reader.erl
@@ -245,11 +245,19 @@ get_map_view(Lang, Args, ViewName, Views) ->
 
 get_red_view(Lang, Args, ViewName, Views) ->
     case couch_mrview_util:extract_view(Lang, Args, ViewName, Views) of
-        {red, {Idx, Lang, View}, _} -> {Idx, Lang, View};
+        {red, {Idx, Lang, View}, _} -> check_red_enabled({Idx, Lang, View});
         _ -> throw({not_found, missing_named_view})
     end.
 
 
+check_red_enabled({Idx, _Lang, View} = Resp) ->
+    case lists:nth(Idx, View#mrview.reduce_funs) of
+        {_, disabled} ->
+            throw({disabled, <<"Custom reduce functions are disabled.">>});
+        _ ->
+            Resp
+    end.
+
 expand_keys_args(#mrargs{keys = undefined} = Args) ->
     [Args];
 
diff --git a/src/couch_views/src/couch_views_trees.erl b/src/couch_views/src/couch_views_trees.erl
index b45750b..d9340ad 100644
--- a/src/couch_views/src/couch_views_trees.erl
+++ b/src/couch_views/src/couch_views_trees.erl
@@ -323,7 +323,7 @@ make_read_only_reduce_fun(Lang, View, NthRed) ->
 
 
 make_reduce_fun(Lang, #mrview{} = View) ->
-    RedFuns = [Src || {_, Src} <- View#mrview.reduce_funs],
+    RedFuns = [Src || {_, Src} <- View#mrview.reduce_funs, Src /= disabled],
     fun
         (KVs0, _ReReduce = false) ->
             KVs1 = expand_dupes(KVs0),
diff --git a/src/couch_views/src/couch_views_util.erl b/src/couch_views/src/couch_views_util.erl
index 1e3e4be..c4130f4 100644
--- a/src/couch_views/src/couch_views_util.erl
+++ b/src/couch_views/src/couch_views_util.erl
@@ -65,7 +65,8 @@ ddoc_to_mrst(DbName, #doc{id=Id, body={Fields}}) ->
     NumViews = fun({_, View}, N) ->
             {View#mrview{id_num = N}, N+1}
     end,
-    {Views, _} = lists:mapfoldl(NumViews, 0, lists:sort(dict:to_list(BySrc))),
+    {Views0, _} = lists:mapfoldl(NumViews, 0, lists:sort(dict:to_list(BySrc))),
+    Views1 = maybe_filter_custom_reduce_funs(Views0),
 
     Language = couch_util:get_value(<<"language">>, Fields, <<"javascript">>),
     Lib = couch_util:get_value(<<"lib">>, RawViews, {[]}),
@@ -74,12 +75,12 @@ ddoc_to_mrst(DbName, #doc{id=Id, body={Fields}}) ->
         db_name=DbName,
         idx_name=Id,
         lib=Lib,
-        views=Views,
+        views=Views1,
         language=Language,
         design_opts=DesignOpts,
         partitioned=Partitioned
     },
-    SigInfo = {Views, Language, DesignOpts, couch_index_util:sort_lib(Lib)},
+    SigInfo = {Views1, Language, DesignOpts, couch_index_util:sort_lib(Lib)},
     {ok, IdxState#mrst{sig=couch_hash:md5_hash(term_to_binary(SigInfo))}}.
 
 
@@ -327,6 +328,33 @@ active_tasks_info(ChangesDone, DbName, DDocId, LastSeq, DBSeq) ->
     }.
 
 
+maybe_filter_custom_reduce_funs(Views) ->
+    case config:get_boolean("couch_views", "custom_reduce_enabled", false) of
+        true ->
+            Views;
+        false ->
+            filter_custom_reduce_funs(Views)
+    end.
+
+
+filter_custom_reduce_funs(Views) ->
+    lists:map(fun(View) ->
+        #mrview{
+            reduce_funs = ReduceFuns
+        } = View,
+        {Builtin, Custom} = lists:partition(fun({_Name, RedSrc}) ->
+            case RedSrc of
+                <<"_", _/binary>> -> true;
+                <<_/binary>> -> false
+            end
+        end, ReduceFuns),
+        DisabledCustom = [{Name, disabled} || {Name, _Src} <- Custom],
+        View#mrview{
+            reduce_funs = Builtin ++ DisabledCustom
+        }
+    end, Views).
+
+
 convert_seq_to_stamp(<<"0">>) ->
     <<"0-0-0">>;
 
diff --git a/src/couch_views/test/couch_views_custom_red_test.erl b/src/couch_views/test/couch_views_custom_red_test.erl
new file mode 100644
index 0000000..a6b3817
--- /dev/null
+++ b/src/couch_views/test/couch_views_custom_red_test.erl
@@ -0,0 +1,194 @@
+% 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_custom_red_test).
+
+-include_lib("couch/include/couch_eunit.hrl").
+-include_lib("couch/include/couch_db.hrl").
+-include_lib("couch_mrview/include/couch_mrview.hrl").
+-include_lib("fabric/test/fabric2_test.hrl").
+-include("couch_views.hrl").
+
+
+-define(NUM_DOCS, 100).
+
+
+custom_reduce_disabled_test_() ->
+    {
+        "Custom Reduce Disabled",
+        {
+            setup,
+            fun setup_disabled/0,
+            fun teardown/1,
+            with([
+                ?TDEF(builtin_reductions_work),
+                ?TDEF(custom_reduces_disabled)
+            ])
+        }
+    }.
+
+
+custom_reduce_enabled_test_() ->
+    {
+        "Custom Reduce Disabled",
+        {
+            setup,
+            fun setup_enabled/0,
+            fun teardown/1,
+            with([
+                ?TDEF(builtin_reductions_work),
+                ?TDEF(custom_reduces_enabled)
+            ])
+        }
+    }.
+
+
+sigs_change_test_() ->
+    {
+        "Sigs Change Test",
+        {
+            setup,
+            fun setup_sigs_change/0,
+            fun teardown_sigs_change/1,
+            with([
+                ?TDEF(sigs_change)
+            ])
+        }
+    }.
+
+setup_disabled() ->
+    setup_common(false).
+
+
+setup_enabled() ->
+    setup_common(true).
+
+
+setup_common(Enabled) ->
+    Ctx = test_util:start_couch([
+            fabric,
+            couch_jobs,
+            couch_js,
+            couch_views
+        ]),
+    config:set_boolean("couch_views", "custom_reduce_enabled", Enabled),
+    {ok, Db} = fabric2_db:create(?tempdb(), [{user_ctx, ?ADMIN_USER}]),
+    fabric2_db:update_docs(Db, [create_ddoc()]),
+    make_docs(Db, ?NUM_DOCS),
+    run_query(Db, <<"builtin">>, #{limit => 0}),
+    {Db, Ctx}.
+
+
+teardown({Db, Ctx}) ->
+    fabric2_db:delete(fabric2_db:name(Db), [{user_ctx, ?ADMIN_USER}]),
+    test_util:stop_couch(Ctx).
+
+
+setup_sigs_change() ->
+    meck:new(config, [passthrough]),
+    meck:expect(config, get, fun(_, _, Default) -> Default end).
+
+
+teardown_sigs_change(_) ->
+    meck:unload().
+
+
+builtin_reductions_work({Db, _}) ->
+    Result = run_query(Db, <<"builtin">>, #{}),
+    Expect = {ok, [row(null, ?NUM_DOCS)]},
+    ?assertEqual(Expect, Result).
+
+
+custom_reduces_disabled({Db, _}) ->
+    ?assertThrow({disabled, _}, run_query(Db, <<"custom">>, #{})).
+
+
+custom_reduces_enabled({Db, _}) ->
+    Result = run_query(Db, <<"custom">>, #{}),
+    Expect = {ok, [row(null, <<"silly_reduce">>)]},
+    ?assertEqual(Expect, Result).
+
+
+sigs_change(_) ->
+    meck:expect(config, get_boolean, fun("couch_views", _, _) -> false end),
+    {ok, Mrst1} = couch_views_util:ddoc_to_mrst(<<"foo">>, create_ddoc()),
+    meck:expect(config, get_boolean, fun("couch_views", _, _) -> true end),
+    {ok, Mrst2} = couch_views_util:ddoc_to_mrst(<<"foo">>, create_ddoc()),
+    ?assertNotEqual(Mrst1#mrst.sig, Mrst2#mrst.sig).
+
+
+run_query(Db, Idx, Args) ->
+    DDoc = create_ddoc(),
+    run_query(Db, DDoc, Idx, Args).
+
+
+run_query(Db, DDoc, Idx, Args) ->
+    couch_views: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({meta, _}, Acc) ->
+    {ok, Acc};
+default_cb(ok, ddoc_updated) ->
+    {ok, ddoc_updated};
+default_cb(Row, Acc) ->
+    {ok, [Row | Acc]}.
+
+
+row(Key, Value) ->
+    {row, [{key, Key}, {value, Value}]}.
+
+
+create_ddoc() ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, <<"_design/bar">>},
+        {<<"views">>, {[
+            {<<"custom">>, {[
+                {<<"map">>, <<"function(doc) {emit(doc.val, doc.val);}">>},
+                {<<"reduce">>, <<
+                    "function(keys, values, rereduce) {\n"
+                    "  return \"silly_reduce\";\n"
+                    "}\n"
+                >>}
+            ]}},
+            {<<"builtin">>, {[
+                {<<"map">>, <<"function(doc) {emit(doc.val, doc.val);}">>},
+                {<<"reduce">>, <<"_count">>}
+            ]}}
+        ]}}
+    ]}).
+
+
+make_docs(Db, TotalDocs) when TotalDocs > 0 ->
+    make_docs(Db, TotalDocs, 0).
+
+
+make_docs(Db, TotalDocs, DocsMade) when TotalDocs > DocsMade ->
+    DocCount = min(TotalDocs - DocsMade, 500),
+    Docs = [doc(I + DocsMade) || I <- lists:seq(1, DocCount)],
+    fabric2_db:update_docs(Db, Docs),
+    make_docs(Db, TotalDocs, DocsMade + DocCount);
+
+make_docs(_Db, TotalDocs, DocsMade) when TotalDocs =< DocsMade ->
+    ok.
+
+
+doc(Id) ->
+    couch_doc:from_json_obj({[
+        {<<"_id">>, list_to_binary(integer_to_list(Id))},
+        {<<"val">>, Id}
+    ]}).