You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@couchdb.apache.org by banjiewen <gi...@git.apache.org> on 2016/03/16 06:24:52 UTC

[GitHub] couchdb-fabric pull request: Support raw view collation

GitHub user banjiewen opened a pull request:

    https://github.com/apache/couchdb-fabric/pull/43

    Support raw view collation

    COUCHDB-2900

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

    $ git pull https://github.com/banjiewen/couchdb-fabric 2900-raw-collation

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

    https://github.com/apache/couchdb-fabric/pull/43.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 #43
    
----
commit f83055642e7b08603c5c71b62ca20ef9bcd65ce3
Author: Benjamin Anderson <b...@banjiewen.net>
Date:   2016-03-16T05:12:42Z

    Add sort_fun to view result collector record
    
    There's no reference to the #mrview when results are being streamed to
    the user, so we need a place to store the relevant fun while results are
    collected.
    
    COUCHDB-2900

commit 9be3c61e27fc3478e451283e7d802b1e7800550e
Author: Benjamin Anderson <b...@banjiewen.net>
Date:   2016-03-16T05:17:41Z

    Support raw collation in reduce results
    
    Also simplify find_next_key/3 and avoid generating a new fun for every
    streamed key.
    
    COUCHDB-2900

commit e90569a4df2912c1fa1631c14f3973da51eca02f
Author: Benjamin Anderson <b...@banjiewen.net>
Date:   2016-03-16T05:21:02Z

    Support raw collation in map results
    
    COUCHDB-2900

----


---
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-fabric pull request: Support raw view collation

Posted by kocolosk <gi...@git.apache.org>.
Github user kocolosk commented on the pull request:

    https://github.com/apache/couchdb-fabric/pull/43#issuecomment-201830063
  
    This all looks correct, but ... wow, it sure would be nice to have a dict where we could override the key comparator. It's tempting to consider a version where we just use a sorted list (basically an `orddict`) instead of `dict`. The two data structures are comparable in performance for ~100 elements or less, and the fetch operation for an `orddict` is trivial:
    ```
    fetch(Key, [{K,_}|D]) when Key > K -> fetch(Key, D);
    fetch(Key, [{K,Value}|_]) when Key == K -> Value.
    ```
    Some other time perhaps.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r57513846
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -135,31 +137,100 @@ handle_message(#view_row{} = Row, {_,From}, #collector{sorted=false} = St) ->
     
     handle_message(#view_row{} = Row, {Worker, From}, State) ->
         #collector{
    -        query_args = #mrargs{direction=Dir},
    +        query_args = #mrargs{direction = Dir},
             counters = Counters0,
             rows = Rows0,
    -        keys = KeyDict
    +        keys = KeyDict0,
    +        collation = Collation
         } = State,
    -    Rows = merge_row(Dir, KeyDict, Row#view_row{worker={Worker, From}}, Rows0),
    +    {Rows, KeyDict} = merge_row(
    +        Dir,
    +        Collation,
    +        KeyDict0,
    +        Row#view_row{worker={Worker, From}},
    +        Rows0
    +    ),
         Counters1 = fabric_dict:update_counter(Worker, 1, Counters0),
    -    State1 = State#collector{rows=Rows, counters=Counters1},
    +    State1 = State#collector{rows=Rows, counters=Counters1, keys=KeyDict},
         fabric_view:maybe_send_row(State1);
     
     handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    -    lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    -    end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    -    lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    -    end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    -    lists:merge(fun(#view_row{key=A, id=IdA}, #view_row{key=B, id=IdB}) ->
    -        if A =:= B -> IdA < IdB; true ->
    -            dict:fetch(A, KeyDict) < dict:fetch(B, KeyDict)
    -        end
    -    end, [Row], Rows).
    +merge_row(Dir, Collation, undefined, Row, Rows0) ->
    +    Rows1 = lists:merge(
    +        fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    +            compare(Dir, Collation, {KeyA, IdA}, {KeyB, IdB})
    +        end,
    +        [Row],
    +        Rows0
    +    ),
    +    {Rows1, undefined};
    +merge_row(Dir, Collation, KeyDict0, Row, Rows0) ->
    +    CmpFun = case Collation of
    +        <<"raw">> ->
    +            fun (A, A) -> 0;
    +                (A, B) -> case A < B of
    +                    true -> -1;
    +                    false -> 1
    +                end
    +            end;
    +        _ ->
    +            fun couch_ejson_compare:less/2
    +    end,
    +    case maybe_update_keydict(Row#view_row.key, KeyDict0, CmpFun) of
    +        undefined ->
    +            {Rows0, KeyDict0};
    +        KeyDict1 ->
    +            Rows1 = lists:merge(
    +                fun(#view_row{key=A, id=IdA}, #view_row{key=B, id=IdB}) ->
    +                    case {Dir, CmpFun(A, B)} of
    +                        {fwd, 0} ->
    +                            IdA < IdB;
    +                        {rev, 0} ->
    +                            IdB < IdA;
    +                        {fwd, _} ->
    +                            dict:fetch(A, KeyDict1) < dict:fetch(B, KeyDict1);
    +                        {rev, _} ->
    +                            dict:fetch(B, KeyDict1) < dict:fetch(A, KeyDict1)
    +                    end
    +                end,
    +                [Row],
    +                Rows0
    +            ),
    +            {Rows1, KeyDict1}
    +    end.
    +
    +compare(_, _, A, A) -> true;
    +compare(fwd, <<"raw">>, A, B) -> A < B;
    +compare(rev, <<"raw">>, A, B) -> B < A;
    +compare(fwd, _, A, B) -> couch_ejson_compare:less_json_ids(A, B);
    +compare(rev, _, A, B) -> couch_ejson_compare:less_json_ids(B, A).
    +
    +% KeyDict captures the user-supplied ordering of keys POSTed by the user by
    +% mapping to integers (see fabric_view:keydict/1). It's possible that these keys
    +% do not compare equal (i.e., =:=, used by dict) to those returned by the view
    +% but are in fact equal under ICU. In this case (assuming the view uses ICU
    +% collation) we must update KeyDict with a mapping from the ICU-equal key to its
    +% appropriate value.
    +maybe_update_keydict(Key, KeyDict, CmpFun) ->
    +    case dict:find(Key, KeyDict) of
    +        {ok, _} ->
    +            KeyDict;
    +        error ->
    +            case key_index(Key, dict:to_list(KeyDict), CmpFun) of
    +                undefined ->
    +                    undefined;
    +                Value ->
    +                    dict:store(Key, Value, KeyDict)
    +            end
    +    end.
    +
    +key_index(_, [], _) ->
    --- End diff --
    
    It should never happen, but this is OK by me given it's already merged.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r56283527
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -149,15 +161,15 @@ handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    +merge_row(fwd, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    +        SortFun({KeyA, IdA}, {KeyB, IdB})
         end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    +merge_row(rev, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    +        SortFun({KeyB, IdB}, {KeyA, IdA})
         end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    +merge_row(_, KeyDict, Row, Rows, _) ->
    --- End diff --
    
    I didn't touch this head (other than changing the arity) but I'm not sure it's right; looks like it's been doing raw collation the whole time, and as such should be changed in this PR to `SortFun`.


---
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-fabric pull request: Support raw view collation

Posted by banjiewen <gi...@git.apache.org>.
Github user banjiewen commented on the pull request:

    https://github.com/apache/couchdb-fabric/pull/43#issuecomment-201886856
  
    > It's tempting to consider a version where we just use a sorted list (basically an orddict) instead of dict.
    
    Right - I sketched out a `gb_trees` implementation that would have turned out nicely, but given the infrequency of the slow case I figured it'd be nicer to stick with the O(1) behavior for the fast case.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r57256872
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -149,15 +161,15 @@ handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    +merge_row(fwd, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    +        SortFun({KeyA, IdA}, {KeyB, IdB})
         end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    +merge_row(rev, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    +        SortFun({KeyB, IdB}, {KeyA, IdA})
         end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    +merge_row(_, KeyDict, Row, Rows, _) ->
    --- End diff --
    
    The `IdA < IdB` comparison is OK; we always use raw collation for document ID comparison (see e.g. `couch_ejson_compare:less_json_ids/2`).
    
    The `A=:=B` one is more interesting. There are certainly keys that compare equal using ICU collation but are not equal in raw collation, for example the one @iilyak loaded as a test case in #44.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r57205987
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -149,15 +161,15 @@ handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    +merge_row(fwd, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    +        SortFun({KeyA, IdA}, {KeyB, IdB})
         end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    +merge_row(rev, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    +        SortFun({KeyB, IdB}, {KeyA, IdA})
         end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    +merge_row(_, KeyDict, Row, Rows, _) ->
    --- End diff --
    
    Actually there is a collation issue here, maybe you had it in mind but I didn't. The issue isn't the explicit comparator, it's the use of the `fetch` operation. The fetch is going to use raw collation to search for an exact key match, but we'll return keys from the view that compare equal using the collation function specified in the view.
    
    I think the fix here is to use the same collation function to lookup elements in the dictionary, and then close #44.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r57277135
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -57,8 +57,12 @@ go(DbName, DDoc, View, Args, Callback, Acc) ->
             rexi_monitor:stop(RexiMon)
         end.
     
    -go(DbName, Workers, Args, Callback, Acc0) ->
    +go(DbName, Workers, {map, View, _}, Args, Callback, Acc0) ->
    --- End diff --
    
    Agree, but this style matches the code in `fabric_view_reduce:go/7`, for better or worse.


---
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-fabric pull request: Support raw view collation

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/couchdb-fabric/pull/43


---
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-fabric pull request: Support raw view collation

Posted by banjiewen <gi...@git.apache.org>.
Github user banjiewen commented on the pull request:

    https://github.com/apache/couchdb-fabric/pull/43#issuecomment-200693010
  
    > Is it worth adding Adam's explanation of KeyDict to the commit message for e90569a for the benefit of future maintainers?
    
    This is done, more or less.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r56868113
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -57,8 +57,12 @@ go(DbName, DDoc, View, Args, Callback, Acc) ->
             rexi_monitor:stop(RexiMon)
         end.
     
    -go(DbName, Workers, Args, Callback, Acc0) ->
    +go(DbName, Workers, {map, View, _}, Args, Callback, Acc0) ->
         #mrargs{limit = Limit, skip = Skip, keys = Keys} = Args,
    +    SortFun = case couch_util:get_value(<<"collation">>, View#mrview.options) of
    +        <<"raw">> -> fun(A, A) -> true; (A, B) -> A < B end;
    +        _ -> fun couch_ejson_compare:less_json_ids/2
    +    end,
    --- End diff --
    
    Oh I see it now - it's in the calls to SortFun down on L166 and L170.


---
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-fabric pull request: Support raw view collation

Posted by kocolosk <gi...@git.apache.org>.
Github user kocolosk commented on the pull request:

    https://github.com/apache/couchdb-fabric/pull/43#issuecomment-200117536
  
    LGTM, nice work.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r56864601
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -57,8 +57,12 @@ go(DbName, DDoc, View, Args, Callback, Acc) ->
             rexi_monitor:stop(RexiMon)
         end.
     
    -go(DbName, Workers, Args, Callback, Acc0) ->
    +go(DbName, Workers, {map, View, _}, Args, Callback, Acc0) ->
    --- End diff --
    
    For some reason my eyes expected `VInfo` to follow `Args, Callback, Acc`, as with `go/7`.


---
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-fabric pull request: Support raw view collation

Posted by mikewallace1979 <gi...@git.apache.org>.
Github user mikewallace1979 commented on the pull request:

    https://github.com/apache/couchdb-fabric/pull/43#issuecomment-199950499
  
    This LGTM but (given I screwed something up last time I touched collation) it'd be handy if someone like @davisp or @kocolosk could take a look.


---
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-fabric pull request: Support raw view collation

Posted by mikewallace1979 <gi...@git.apache.org>.
Github user mikewallace1979 commented on the pull request:

    https://github.com/apache/couchdb-fabric/pull/43#issuecomment-200324098
  
    @banjiewen Is it worth adding Adam's explanation of `KeyDict` to the commit message for e90569a for the benefit of future maintainers?


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r57255762
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -149,15 +161,15 @@ handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    +merge_row(fwd, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    +        SortFun({KeyA, IdA}, {KeyB, IdB})
         end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    +merge_row(rev, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    +        SortFun({KeyB, IdB}, {KeyA, IdA})
         end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    +merge_row(_, KeyDict, Row, Rows, _) ->
    --- End diff --
    
    > The purpose of KeyDict is to capture the original order of the keys posted by the user and ensure that we respond in the same order.
    
    That makes more sense now - I'll add a comment before I'm done here - but I'm not quite convinced that the `fetch` is the whole problem. Shouldn't the `IdA < IdB` comparison on L174 be done using the configured collator, since those results are for identical keys? The same argument might apply for the `A =:-= B` comparison on L174 as well, though I don't know of any values that compare equal under ICU that don't under Erlang's `=:=`...


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r57262900
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -149,15 +161,15 @@ handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    +merge_row(fwd, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    +        SortFun({KeyA, IdA}, {KeyB, IdB})
         end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    +merge_row(rev, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    +        SortFun({KeyB, IdB}, {KeyA, IdA})
         end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    +merge_row(_, KeyDict, Row, Rows, _) ->
    --- End diff --
    
    > we always use raw collation for document ID comparison
    
    Heh, hadn't re-read that one - good point.
    
    > There are certainly keys that compare equal using ICU collation but are not equal in raw collation
    
    Ah - I hadn't fully grokked the implications of that issue. I'll put together a fix.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r56870110
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -149,15 +161,15 @@ handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    +merge_row(fwd, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    +        SortFun({KeyA, IdA}, {KeyB, IdB})
         end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    +merge_row(rev, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    +        SortFun({KeyB, IdB}, {KeyA, IdA})
         end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    +merge_row(_, KeyDict, Row, Rows, _) ->
    --- End diff --
    
    I suspect `SortFun` is the right thing too.


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r57057309
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -149,15 +161,15 @@ handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    +merge_row(fwd, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    +        SortFun({KeyA, IdA}, {KeyB, IdB})
         end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    +merge_row(rev, undefined, Row, Rows, SortFun) ->
         lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    +        SortFun({KeyB, IdB}, {KeyA, IdA})
         end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    +merge_row(_, KeyDict, Row, Rows, _) ->
    --- End diff --
    
    No I think this is right, albeit woefully documented. The purpose of `KeyDict` is to capture the original order of the keys posted by the user and ensure that we respond in the same order. The values in the KeyDict are integer positions computed from the user request per `fabric_view:keydict/1`.


---
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-fabric pull request: Support raw view collation

Posted by banjiewen <gi...@git.apache.org>.
Github user banjiewen commented on the pull request:

    https://github.com/apache/couchdb-fabric/pull/43#issuecomment-200693294
  
    This PR has been substantially rewritten in light of [this discussion](https://github.com/apache/couchdb-fabric/pull/43#discussion_r57205987). The only fundamental behavior change is in 1be5506, and the previous HEAD is [here](https://github.com/banjiewen/couchdb-fabric/tree/e90569a4df2912c1fa1631c14f3973da51eca02f). It's ready for another pass when you folks have a chance. /cc @kocolosk, @mikewallace1979, @iilyak


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r56865726
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -57,8 +57,12 @@ go(DbName, DDoc, View, Args, Callback, Acc) ->
             rexi_monitor:stop(RexiMon)
         end.
     
    -go(DbName, Workers, Args, Callback, Acc0) ->
    +go(DbName, Workers, {map, View, _}, Args, Callback, Acc0) ->
         #mrargs{limit = Limit, skip = Skip, keys = Keys} = Args,
    +    SortFun = case couch_util:get_value(<<"collation">>, View#mrview.options) of
    +        <<"raw">> -> fun(A, A) -> true; (A, B) -> A < B end;
    +        _ -> fun couch_ejson_compare:less_json_ids/2
    +    end,
    --- End diff --
    
    I can't see how direction is handled for map queries - am I missing something implicit?


---
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-fabric pull request: Support raw view collation

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

    https://github.com/apache/couchdb-fabric/pull/43#discussion_r57278616
  
    --- Diff: src/fabric_view_map.erl ---
    @@ -135,31 +137,100 @@ handle_message(#view_row{} = Row, {_,From}, #collector{sorted=false} = St) ->
     
     handle_message(#view_row{} = Row, {Worker, From}, State) ->
         #collector{
    -        query_args = #mrargs{direction=Dir},
    +        query_args = #mrargs{direction = Dir},
             counters = Counters0,
             rows = Rows0,
    -        keys = KeyDict
    +        keys = KeyDict0,
    +        collation = Collation
         } = State,
    -    Rows = merge_row(Dir, KeyDict, Row#view_row{worker={Worker, From}}, Rows0),
    +    {Rows, KeyDict} = merge_row(
    +        Dir,
    +        Collation,
    +        KeyDict0,
    +        Row#view_row{worker={Worker, From}},
    +        Rows0
    +    ),
         Counters1 = fabric_dict:update_counter(Worker, 1, Counters0),
    -    State1 = State#collector{rows=Rows, counters=Counters1},
    +    State1 = State#collector{rows=Rows, counters=Counters1, keys=KeyDict},
         fabric_view:maybe_send_row(State1);
     
     handle_message(complete, Worker, State) ->
         Counters = fabric_dict:update_counter(Worker, 1, State#collector.counters),
         fabric_view:maybe_send_row(State#collector{counters = Counters}).
     
    -merge_row(fwd, undefined, Row, Rows) ->
    -    lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyA, IdA}, {KeyB, IdB})
    -    end, [Row], Rows);
    -merge_row(rev, undefined, Row, Rows) ->
    -    lists:merge(fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    -        couch_ejson_compare:less_json_ids({KeyB, IdB}, {KeyA, IdA})
    -    end, [Row], Rows);
    -merge_row(_, KeyDict, Row, Rows) ->
    -    lists:merge(fun(#view_row{key=A, id=IdA}, #view_row{key=B, id=IdB}) ->
    -        if A =:= B -> IdA < IdB; true ->
    -            dict:fetch(A, KeyDict) < dict:fetch(B, KeyDict)
    -        end
    -    end, [Row], Rows).
    +merge_row(Dir, Collation, undefined, Row, Rows0) ->
    +    Rows1 = lists:merge(
    +        fun(#view_row{key=KeyA, id=IdA}, #view_row{key=KeyB, id=IdB}) ->
    +            compare(Dir, Collation, {KeyA, IdA}, {KeyB, IdB})
    +        end,
    +        [Row],
    +        Rows0
    +    ),
    +    {Rows1, undefined};
    +merge_row(Dir, Collation, KeyDict0, Row, Rows0) ->
    +    CmpFun = case Collation of
    +        <<"raw">> ->
    +            fun (A, A) -> 0;
    +                (A, B) -> case A < B of
    +                    true -> -1;
    +                    false -> 1
    +                end
    +            end;
    +        _ ->
    +            fun couch_ejson_compare:less/2
    +    end,
    +    case maybe_update_keydict(Row#view_row.key, KeyDict0, CmpFun) of
    +        undefined ->
    +            {Rows0, KeyDict0};
    +        KeyDict1 ->
    +            Rows1 = lists:merge(
    +                fun(#view_row{key=A, id=IdA}, #view_row{key=B, id=IdB}) ->
    +                    case {Dir, CmpFun(A, B)} of
    +                        {fwd, 0} ->
    +                            IdA < IdB;
    +                        {rev, 0} ->
    +                            IdB < IdA;
    +                        {fwd, _} ->
    +                            dict:fetch(A, KeyDict1) < dict:fetch(B, KeyDict1);
    +                        {rev, _} ->
    +                            dict:fetch(B, KeyDict1) < dict:fetch(A, KeyDict1)
    +                    end
    +                end,
    +                [Row],
    +                Rows0
    +            ),
    +            {Rows1, KeyDict1}
    +    end.
    +
    +compare(_, _, A, A) -> true;
    +compare(fwd, <<"raw">>, A, B) -> A < B;
    +compare(rev, <<"raw">>, A, B) -> B < A;
    +compare(fwd, _, A, B) -> couch_ejson_compare:less_json_ids(A, B);
    +compare(rev, _, A, B) -> couch_ejson_compare:less_json_ids(B, A).
    +
    +% KeyDict captures the user-supplied ordering of keys POSTed by the user by
    +% mapping to integers (see fabric_view:keydict/1). It's possible that these keys
    +% do not compare equal (i.e., =:=, used by dict) to those returned by the view
    +% but are in fact equal under ICU. In this case (assuming the view uses ICU
    +% collation) we must update KeyDict with a mapping from the ICU-equal key to its
    +% appropriate value.
    +maybe_update_keydict(Key, KeyDict, CmpFun) ->
    +    case dict:find(Key, KeyDict) of
    +        {ok, _} ->
    +            KeyDict;
    +        error ->
    +            case key_index(Key, dict:to_list(KeyDict), CmpFun) of
    +                undefined ->
    +                    undefined;
    +                Value ->
    +                    dict:store(Key, Value, KeyDict)
    +            end
    +    end.
    +
    +key_index(_, [], _) ->
    --- End diff --
    
    It wasn't immediately clear to me if we should _never_ find a key in the results that wasn't POSTed by the user. The existing code doesn't handle it, of course. We could remove this head if we'd rather throw than elide the row.


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