You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@couchdb.apache.org by jiangphcn <gi...@git.apache.org> on 2017/02/27 13:51:05 UTC

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

GitHub user jiangphcn opened a pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67

    68276 new purge api

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/cloudant/couchdb-couch-mrview 68276_new_purge_api

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/couchdb-couch-mrview/pull/67.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #67
    
----
commit 838aaf4964341383ce707c8fa770867912a46ea0
Author: Paul J. Davis <pa...@gmail.com>
Date:   2016-02-10T22:31:45Z

    Update to use pluggable storage API

commit d685f9abcf9f75e9d4ee83d9b0848ccf086bf32e
Author: Mayya Sharipova <ma...@ca.ibm.com>
Date:   2016-08-15T13:26:48Z

    Update couch_mrview for new purge API
    
    Add test for view after purge requests
    
    BugzID: 68276

commit 80aeafdb5a5b66ecb0d0f1d268032e4be3e76525
Author: Mayya Sharipova <ma...@ca.ibm.com>
Date:   2016-11-25T23:06:56Z

    Update tests for view after purge requests
    
    couch_db:purge_docs now accepts a list of IdRevs
    this changes in the tests calls this new API
    
    BugzID: 68276

commit 5b2de8d0c8781c13b0c3f39e83179de900b18491
Author: Mayya Sharipova <ma...@ca.ibm.com>
Date:   2016-12-08T20:47:17Z

    couch_db:get_purge_seq/1 returns Seq (not {ok, Seq})
    
    BugzID: 68276

commit 9bb86dbf3dea464f6459a33743796b55ca235e76
Author: jiangphcn <ji...@cn.ibm.com>
Date:   2017-02-16T08:09:47Z

    Update couch_mrview for clustered purge improvement
     - implementation of updating local purge document
     - provide verify_index_exists/1 to check whether
       index exists or not
     - clean up local purge document in cleanup cycle
     - add new test case and refine existing test cases
    
    Bugzid: 68276

commit a7bbe7923cbfe26703bc6df67b77f5db93ffdd25
Author: jiangphcn <ji...@cn.ibm.com>
Date:   2017-02-21T07:48:44Z

    Update couch_mrview for clustered purge improvement
     - create local purge document if it doesn\u2019t exist
       when couch_mrview_index:open/2 is called
     - directly use db to clean up local purge document
    
    Bugzid: 68276

commit 46fa6b4a85de29855dd16635525a09636cd37883
Author: jiangphcn <ji...@cn.ibm.com>
Date:   2017-02-27T13:35:48Z

    update on couch_mrview for clustered purge improvement
     - use couch_db:purge_docs/2 with UUID
     - add error handling for verify_index_exists/1
    
    Bugzid: 68276

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r105926830
  
    --- Diff: test/couch_mrview_purge_docs_tests.erl ---
    @@ -0,0 +1,126 @@
    +% 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_mrview_purge_docs_tests).
    +
    +-include_lib("couch/include/couch_eunit.hrl").
    +-include_lib("couch/include/couch_db.hrl").
    +-include_lib("couch_mrview/include/couch_mrview.hrl").
    +
    +-define(TIMEOUT, 1000).
    +
    +
    +setup() ->
    +    {ok, Db} = couch_mrview_test_util:init_db(?tempdb(), map, 5),
    +    Db.
    +
    +teardown(Db) ->
    +    couch_db:close(Db),
    +    couch_server:delete(Db#db.name, [?ADMIN_CTX]),
    +    ok.
    +
    +view_purge_test_() ->
    +    {
    +        "Map views",
    +        {
    +            setup,
    +            fun test_util:start_couch/0, fun test_util:stop_couch/1,
    +            {
    +                foreach,
    +                fun setup/0, fun teardown/1,
    +                [
    +                    fun test_purge_single/1,
    +                    fun test_purge_multiple/1
    +                ]
    +            }
    +        }
    +    }.
    +
    +
    +test_purge_single(Db) ->
    +    ?_test(begin
    +        Result = run_query(Db, []),
    +        Expect = {ok, [
    +            {meta, [{total, 5}, {offset, 0}]},
    +            {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),
    +
    +        FDI = couch_db:get_full_doc_info(Db, <<"1">>),
    +        Rev = get_rev(FDI),
    +        {ok, {_, _}} = couch_db:purge_docs(Db, [{<<"UUID1">>, <<"1">>, [Rev]}]),
    +        {ok, Db2} = couch_db:reopen(Db),
    +
    +        Result2 = run_query(Db2, []),
    +        Expect2 = {ok, [
    +            {meta, [{total, 4}, {offset, 0}]},
    +            {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(Expect2, Result2),
    +
    +        ok
    +    end).
    +
    +
    +test_purge_multiple(Db) ->
    +    ?_test(begin
    +        Result = run_query(Db, []),
    +        Expect = {ok, [
    +            {meta, [{total, 5}, {offset, 0}]},
    +            {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),
    +
    +        % 1st purge request
    --- End diff --
    
    Unnecessary comment


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r103232925
  
    --- Diff: src/couch_mrview_index.erl ---
    @@ -204,3 +208,53 @@ index_file_exists(State) ->
         } = State,
         IndexFName = couch_mrview_util:index_file(DbName, Sig),
         filelib:is_file(IndexFName).
    +
    +
    +update_local_purge_doc(Db, State) ->
    +    Sig = couch_index_util:hexsig(get(signature, State)),
    +    Doc = couch_doc:from_json_obj({[
    +        {<<"_id">>, couch_mrview_util:get_local_purge_doc_id(Sig)},
    +        {<<"purge_seq">>, get(purge_seq, State)},
    +        {<<"timestamp_utc">>, list_to_binary(couch_mrview_util:utc_string())},
    +        {<<"verify_module">>, <<"couch_mrview_index">>},
    +        {<<"verify_function">>, <<"verify_index_exists">>},
    +        {<<"verify_options">>, {[
    +            {<<"dbname">>, get(db_name, State)},
    +            {<<"indexname">>, get(idx_name, State)},
    +            {<<"signature">>, Sig}
    +        ]}},
    +        {<<"type">>, <<"view">>}
    +    ]}),
    +    couch_db:update_doc(Db, Doc, []).
    +
    +
    +verify_index_exists(Options) ->
    +    DbName = couch_mrview_util:get_value_from_options(<<"dbname">>, Options),
    +    IndexName = couch_mrview_util:get_value_from_options(<<"indexname">>, Options),
    +    SigInLocal = couch_mrview_util:get_value_from_options(<<"signature">>, Options),
    +    case couch_db:open_int(DbName, []) of
    +        {ok, Db} ->
    +            try
    +                {ok, DDoc} = couch_db:open_doc(Db, IndexName, []),
    +                {ok, IdxState} = couch_mrview_util:ddoc_to_mrst(DbName, DDoc),
    +                couch_index_util:hexsig(IdxState#mrst.sig) == SigInLocal
    +            catch _:_ ->
    +                false
    +            after
    +                catch couch_db:close(Db)
    +            end;
    +        _ ->
    +            false
    +    end.
    +
    +
    --- End diff --
    
    You've got two extra blank lines here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r103233260
  
    --- Diff: src/couch_mrview_index.erl ---
    @@ -204,3 +208,53 @@ index_file_exists(State) ->
         } = State,
         IndexFName = couch_mrview_util:index_file(DbName, Sig),
         filelib:is_file(IndexFName).
    +
    +
    +update_local_purge_doc(Db, State) ->
    +    Sig = couch_index_util:hexsig(get(signature, State)),
    +    Doc = couch_doc:from_json_obj({[
    +        {<<"_id">>, couch_mrview_util:get_local_purge_doc_id(Sig)},
    +        {<<"purge_seq">>, get(purge_seq, State)},
    +        {<<"timestamp_utc">>, list_to_binary(couch_mrview_util:utc_string())},
    +        {<<"verify_module">>, <<"couch_mrview_index">>},
    +        {<<"verify_function">>, <<"verify_index_exists">>},
    +        {<<"verify_options">>, {[
    +            {<<"dbname">>, get(db_name, State)},
    +            {<<"indexname">>, get(idx_name, State)},
    +            {<<"signature">>, Sig}
    +        ]}},
    +        {<<"type">>, <<"view">>}
    --- End diff --
    
    I would use <<"mrview">> instead of just <<"view">>.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview issue #67: 68276 new purge api

Posted by jiangphcn <gi...@git.apache.org>.
Github user jiangphcn commented on the issue:

    https://github.com/apache/couchdb-couch-mrview/pull/67
  
    Thanks, Paul. I already addressed all comments above in commit 4a1d645. Please help check when you get time. Regarding `utc_string()`, we plan to ask Mayya to create one in couchdb-couch and refer to new one in other places.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r103232522
  
    --- Diff: src/couch_mrview_cleanup.erl ---
    @@ -41,7 +41,20 @@ run(Db) ->
     
         lists:foreach(fun(FN) ->
             couch_log:debug("Deleting stale view file: ~s", [FN]),
    -        couch_file:delete(RootDir, FN, [sync])
    +        couch_file:delete(RootDir, FN, [sync]),
    +        FilePathList = filename:split(FN),
    +        % delete local purge document if have
    +        [PureFN] = lists:nthtail(length(FilePathList)-1, FilePathList),
    +        PureFNExt = filename:extension(PureFN),
    +        Sig = filename:basename(PureFN, PureFNExt),
    --- End diff --
    
    You should take the last four or five lines and move them to a couch_mrview_util function called get_signature_from_filename or something similar. We should also probably assert that the signature is something close to expected, ie > 16 characters with only hexadecimal characters or something.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r105942223
  
    --- Diff: src/couch_mrview_index.erl ---
    @@ -230,20 +230,23 @@ update_local_purge_doc(Db, State) ->
     
     verify_index_exists(Options) ->
         ShardDbName = couch_mrview_util:get_value_from_options(<<"dbname">>, Options),
    -    IndexName = couch_mrview_util:get_value_from_options(<<"indexname">>, Options),
    +    DDocId = couch_mrview_util:get_value_from_options(<<"ddoc_id">>, Options),
         SigInLocal = couch_mrview_util:get_value_from_options(<<"signature">>, Options),
         case couch_db:open_int(ShardDbName, []) of
             {ok, Db} ->
                 try
    -                DbName =mem3:dbname(Db#db.name),
    -                case ddoc_cache:open(DbName, IndexName) of
    +                DbName = mem3:dbname(Db#db.name),
    +                case ddoc_cache:open(DbName, DDocId) of
                         {ok, DDoc} ->
                             {ok, IdxState} = couch_mrview_util:ddoc_to_mrst(ShardDbName, DDoc),
                             couch_index_util:hexsig(IdxState#mrst.sig) == SigInLocal;
                         _Else ->
                             false
                     end
    -            catch _:_ ->
    +            catch E:T ->
    +                Stack = erlang:get_stacktrace(),
    +                couch_log:error("Error occurs when verifying existence of index: ~p ~p",
    +                    [{E, T}, Stack]),
    --- End diff --
    
    I would change that to ... "verifying existince of ~s/~s :: ~p ~p", [ShardDbName, DDocId, {E, T}, Stack]


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r105926680
  
    --- Diff: src/couch_mrview_util.erl ---
    @@ -39,6 +41,26 @@
     -include_lib("couch_mrview/include/couch_mrview.hrl").
     
     
    +get_local_purge_doc_id(Sig) ->
    +    list_to_binary(?LOCAL_DOC_PREFIX ++ "purge-mrview-" ++ Sig).
    +
    +
    +get_value_from_options(Key, Options) ->
    +    case couch_util:get_value(Key, Options) of
    +        undefined ->
    +            Reason = binary_to_list(Key) ++ " must exist in Options.",
    +            throw({bad_request, Reason});
    +        Value -> Value
    +    end.
    +
    +
    +get_signature_from_filename(FileName) ->
    +    FilePathList = filename:split(FileName),
    +    [PureFN] = lists:nthtail(length(FilePathList)-1, FilePathList),
    --- End diff --
    
    Missing spaces around - operator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r105926127
  
    --- Diff: src/couch_mrview_index.erl ---
    @@ -204,3 +208,56 @@ index_file_exists(State) ->
         } = State,
         IndexFName = couch_mrview_util:index_file(DbName, Sig),
         filelib:is_file(IndexFName).
    +
    +
    +update_local_purge_doc(Db, State) ->
    +    Sig = couch_index_util:hexsig(get(signature, State)),
    +    Doc = couch_doc:from_json_obj({[
    +        {<<"_id">>, couch_mrview_util:get_local_purge_doc_id(Sig)},
    +        {<<"purge_seq">>, get(purge_seq, State)},
    +        {<<"timestamp_utc">>, list_to_binary(couch_util:utc_string())},
    +        {<<"verify_module">>, <<"couch_mrview_index">>},
    +        {<<"verify_function">>, <<"verify_index_exists">>},
    +        {<<"verify_options">>, {[
    +            {<<"dbname">>, get(db_name, State)},
    +            {<<"indexname">>, get(idx_name, State)},
    +            {<<"signature">>, Sig}
    +        ]}},
    +        {<<"type">>, <<"mrview">>}
    +    ]}),
    +    couch_db:update_doc(Db, Doc, []).
    +
    +
    +verify_index_exists(Options) ->
    +    ShardDbName = couch_mrview_util:get_value_from_options(<<"dbname">>, Options),
    +    IndexName = couch_mrview_util:get_value_from_options(<<"indexname">>, Options),
    +    SigInLocal = couch_mrview_util:get_value_from_options(<<"signature">>, Options),
    +    case couch_db:open_int(ShardDbName, []) of
    +        {ok, Db} ->
    +            try
    +                DbName =mem3:dbname(Db#db.name),
    --- End diff --
    
    Missing space after = operator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r103233337
  
    --- Diff: src/couch_mrview_util.erl ---
    @@ -38,6 +39,26 @@
     -include_lib("couch/include/couch_db.hrl").
     -include_lib("couch_mrview/include/couch_mrview.hrl").
     
    +utc_string() ->
    +    {{Year, Month, Day}, {Hour, Minute, Second}} =
    +        calendar:now_to_universal_time(os:timestamp()),
    +    lists:flatten(
    +        io_lib:format("~.4.0w-~.2.0w-~.2.0wT~.2.0w:~.2.0w:~.2.0wZ",
    +            [Year, Month, Day, Hour, Minute, Second])).
    +
    +
    +get_local_purge_doc_id(Sig) ->
    +    list_to_binary(?LOCAL_DOC_PREFIX ++ "purge-view-" ++ Sig).
    --- End diff --
    
    I would use purge-mrview instead of purge-view


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r105926330
  
    --- Diff: src/couch_mrview_index.erl ---
    @@ -204,3 +208,56 @@ index_file_exists(State) ->
         } = State,
         IndexFName = couch_mrview_util:index_file(DbName, Sig),
         filelib:is_file(IndexFName).
    +
    +
    +update_local_purge_doc(Db, State) ->
    +    Sig = couch_index_util:hexsig(get(signature, State)),
    +    Doc = couch_doc:from_json_obj({[
    +        {<<"_id">>, couch_mrview_util:get_local_purge_doc_id(Sig)},
    +        {<<"purge_seq">>, get(purge_seq, State)},
    +        {<<"timestamp_utc">>, list_to_binary(couch_util:utc_string())},
    +        {<<"verify_module">>, <<"couch_mrview_index">>},
    +        {<<"verify_function">>, <<"verify_index_exists">>},
    +        {<<"verify_options">>, {[
    +            {<<"dbname">>, get(db_name, State)},
    +            {<<"indexname">>, get(idx_name, State)},
    +            {<<"signature">>, Sig}
    +        ]}},
    +        {<<"type">>, <<"mrview">>}
    +    ]}),
    +    couch_db:update_doc(Db, Doc, []).
    +
    +
    +verify_index_exists(Options) ->
    +    ShardDbName = couch_mrview_util:get_value_from_options(<<"dbname">>, Options),
    +    IndexName = couch_mrview_util:get_value_from_options(<<"indexname">>, Options),
    +    SigInLocal = couch_mrview_util:get_value_from_options(<<"signature">>, Options),
    +    case couch_db:open_int(ShardDbName, []) of
    +        {ok, Db} ->
    +            try
    +                DbName =mem3:dbname(Db#db.name),
    +                case ddoc_cache:open(DbName, IndexName) of
    +                    {ok, DDoc} ->
    +                        {ok, IdxState} = couch_mrview_util:ddoc_to_mrst(ShardDbName, DDoc),
    +                        couch_index_util:hexsig(IdxState#mrst.sig) == SigInLocal;
    +                    _Else ->
    +                        false
    +                end
    +            catch _:_ ->
    +                false
    +            after
    +                catch couch_db:close(Db)
    +            end;
    +        _ ->
    +            false
    +    end.
    +
    +
    +maybe_create_local_purge_doc(Db, State) ->
    +    Sig = couch_index_util:hexsig(get(signature, State)),
    +    case couch_db:open_doc(Db, couch_mrview_util:get_local_purge_doc_id(Sig), []) of
    --- End diff --
    
    Line is longer than 80 characters.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview issue #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on the issue:

    https://github.com/apache/couchdb-couch-mrview/pull/67
  
    LGTM though we can't merge this until after PSE stuff lands of course.
    
    I just have a few minor style tweaks to note but otherwise this looks pretty good all around.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r105929893
  
    --- Diff: src/couch_mrview_index.erl ---
    @@ -204,3 +208,56 @@ index_file_exists(State) ->
         } = State,
         IndexFName = couch_mrview_util:index_file(DbName, Sig),
         filelib:is_file(IndexFName).
    +
    +
    +update_local_purge_doc(Db, State) ->
    +    Sig = couch_index_util:hexsig(get(signature, State)),
    +    Doc = couch_doc:from_json_obj({[
    +        {<<"_id">>, couch_mrview_util:get_local_purge_doc_id(Sig)},
    +        {<<"purge_seq">>, get(purge_seq, State)},
    +        {<<"timestamp_utc">>, list_to_binary(couch_util:utc_string())},
    +        {<<"verify_module">>, <<"couch_mrview_index">>},
    +        {<<"verify_function">>, <<"verify_index_exists">>},
    +        {<<"verify_options">>, {[
    +            {<<"dbname">>, get(db_name, State)},
    +            {<<"indexname">>, get(idx_name, State)},
    --- End diff --
    
    you should call this ddoc_id here since that'll be more consistent with other parts of the code base. For other indexers that don't treat the whole ddoc as a single index is where there's an added indexname.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] couchdb-couch-mrview pull request #67: 68276 new purge api

Posted by davisp <gi...@git.apache.org>.
Github user davisp commented on a diff in the pull request:

    https://github.com/apache/couchdb-couch-mrview/pull/67#discussion_r103233058
  
    --- Diff: src/couch_mrview_util.erl ---
    @@ -38,6 +39,26 @@
     -include_lib("couch/include/couch_db.hrl").
     -include_lib("couch_mrview/include/couch_mrview.hrl").
     
    +utc_string() ->
    --- End diff --
    
    Missing a blank line before utc_string/0


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---