Dmitriy, Honestly, I don't understand your issues with a reviewing, because I've provided PR link in the JIRA ticket.
Just open it in browser and enjoy :) Nevertheless, I've added short description for API changes in the JIRA ticket. Other comments are below. 2017-01-19 20:35 GMT+03:00 Dmitriy Setrakyan <dsetrak...@apache.org>: > Alexey S, > > My comments are below. I would like to ask you again to provide all the API > changes explicitly in the Jira ticket without asking the community to > download the whole GIT repo. > > D. > > On Thu, Jan 19, 2017 at 6:27 AM, Alexei Scherbakov < > alexey.scherbak...@gmail.com> wrote: > > > 1. OK. > > > > Agree. > > > > > > 2. Agreed. In future we might split query execution between nodes, but > for > > now query is routed to random node in grid, > > > > If you are talking about REPLICATED caches, then sending query to a random > node when a user explicitly specifies the partitions is just deceiving. I > would throw an exception in case if a user specifies partitions for > REPLICATED caches. > > Exception is added. > > > > > 3. OK, let's mark getter/setter as deprecated. > > > > I still do not know the proposed new API and why we are deprecating methods > on the old API. I have asked many times to post all the API changes in the > ticket. > > Alexey S., can you please do it, so the community members can review them > without installing the project on their mobile devices? > Description is added to the JIRA ticket. > > 4. Query must be executed locally only for defined partitions. Currently > > this setting is ignored for local queries. > > > > This is again the wrong behavior. We should not "ignore" anything. Let's > throw an exception with a correct error message. > It's already working. > > > > 5. I have the same understanding. Distributed joins will ignore the > > setting. > > This is not implemented yet.. > > > > And again, this will be very confusing to users. Any chance we can throw an > exception with a proper error message here? > I hope to make it working too. But first I need a review of current PR state to understand whether I'm moving in right direction or not. > > > > > > > > 2017-01-19 15:39 GMT+03:00 Sergi Vladykin <sergi.vlady...@gmail.com>: > > > > > Agree, lets remove everything related to partition ranges. Looks like > > > unnecessary complication. > > > > > > Sergi > > > > > > 2017-01-19 10:01 GMT+03:00 Vladimir Ozerov <voze...@gridgain.com>: > > > > > > > Several side notes about API. > > > > > > > > 1) I would avoid ranges even in this form.for the sake of simplicity. > > > > Ignite do not have any notion of "partition range" in affinity API, > so > > I > > > do > > > > not understand how users are going to work on ranges unless they have > > > some > > > > very special custom affinity function, which is rather unlikely case. > > > > > > > > 2) The fact that this property is ignored in REPLICATED cache is > > > confusing. > > > > REPLICATED cache still divides partitions into primaries and backups. > > If > > > I > > > > have very large data set and want to execute some query, I would > > > definitely > > > > expect that Ignite will take advantage of distributed computing and > > > spread > > > > the load between nodes. I understand that currently SQL queries do > not > > > work > > > > this way, but this is clear disadvantage for certain scenarios, which > > we > > > > may improve in future. I would remove this paragraph from docs. > > > > > > > > 3) We already have ScanQuery.partition getter/setter. We need to make > > > sure > > > > that they are "merged" somehow. For instance, we may deprecate two > > > methods > > > > in ScanQuery class, and advise users to use Query.partitions, with > > > > clarification - only single partition is supported for ScanQuery at > the > > > > moment. > > > > > > > > 4) What should happen if "partitions" are defined and "local" flag is > > > set? > > > > > > > > As per distributed joins - how are we going to execute them when > > > partitions > > > > are set explicitly? As far as I understand, partitions should apply > > only > > > to > > > > map step and only for the cache query was created from, This way > > > > distributed join execution should effectively ignore partitions? > > > > > > > > Vladimir. > > > > > > > > > > > > On Thu, Jan 19, 2017 at 1:04 AM, Alexei Scherbakov < > > > > alexey.scherbak...@gmail.com> wrote: > > > > > > > > > I mean distributed joins. > > > > > > > > > > 2017-01-19 0:10 GMT+03:00 Alexei Scherbakov < > > > > alexey.scherbak...@gmail.com> > > > > > : > > > > > > > > > > > Guys, > > > > > > > > > > > > I've finished adding API changes and implemented proper nodes > > > routing. > > > > > > > > > > > > Currently it doesn't work with distributed queries.But I think > this > > > > > > feature should be compatible with it. > > > > > > > > > > > > Could anyone take a look at current branch state while I'm > looking > > > > deeper > > > > > > into dsitributed queries code? > > > > > > > > > > > > Issue: https://issues.apache.org/jira/browse/IGNITE-4523 > > > > > > PR: https://github.com/apache/ignite/pull/1418 > > > > > > > > > > > > > > > > > > > > > > > > 2017-01-13 15:55 GMT+03:00 Alexei Scherbakov < > > > > > alexey.scherbak...@gmail.com > > > > > > >: > > > > > > > > > > > >> OK, let's do it this way. > > > > > >> > > > > > >> > > > > > >> > > > > > >> > > > > > >> > > > > > >> 2017-01-13 13:27 GMT+03:00 Sergi Vladykin < > > sergi.vlady...@gmail.com > > > >: > > > > > >> > > > > > >>> Internally we still use int[] when we send partitions (see > > > > > >>> GridH2QueryRequest.parts). It looks like we only do more work > > with > > > > > >>> PartitionSet. > > > > > >>> > > > > > >>> I like the idea of bitset for partitions, but > > > > > >>> > > > > > >>> 1. We have to change internals first to use it, otherwise the > > > > > >>> optimization > > > > > >>> makes no sense. > > > > > >>> 2. We will need to have a method SqlQuery.setPartitions(int... > > > parts) > > > > > for > > > > > >>> usability reasons anyways. > > > > > >>> > > > > > >>> Thus I suggest for now to go the straightforward way with int[] > > and > > > > > >>> create > > > > > >>> a separate ticket describing the optimization with bitset. > > > > > >>> > > > > > >>> Sergi > > > > > >>> > > > > > >>> 2017-01-13 13:06 GMT+03:00 Alexei Scherbakov < > > > > > >>> alexey.scherbak...@gmail.com>: > > > > > >>> > > > > > >>> > PartitionSet hides internal implementation of int array. > > > > > >>> > > > > > > >>> > This allows as to efficiently represent contiguous range of > > > > > partitions > > > > > >>> and > > > > > >>> > defines clear API for ordered iteration over partitions and > > > > > containment > > > > > >>> > check. > > > > > >>> > > > > > > >>> > Even better to go with compressed bitmap, as I mentioned in > > > ticket > > > > > >>> comment. > > > > > >>> > This will allow us to minimize heap footprint for this > object. > > > > > >>> > > > > > > >>> > Moreover, it will be useful to create reusable compressed > > bitmap > > > > > >>> > implementation in Ignite and use it in other cases, on > example, > > > for > > > > > >>> > replacing H2's IntArray and Set<Integer>. > > > > > >>> > > > > > > >>> > Should I create a ticket for this ? > > > > > >>> > > > > > > >>> > . > > > > > >>> > > > > > > >>> > 2017-01-13 1:01 GMT+03:00 Dmitriy Setrakyan < > > > dsetrak...@apache.org > > > > >: > > > > > >>> > > > > > > >>> > > On Thu, Jan 12, 2017 at 6:12 AM, Sergi Vladykin < > > > > > >>> > sergi.vlady...@gmail.com> > > > > > >>> > > wrote: > > > > > >>> > > > > > > > >>> > > > I looked at the code. The PartitionSet concept looks > > > > > >>> overengineered to > > > > > >>> > > me, > > > > > >>> > > > why wouldn't we just go with int[]? > > > > > >>> > > > > > > > > >>> > > > > > > > >>> > > Agree. > > > > > >>> > > > > > > > >>> > > > > > > > >>> > > > > > > > > >>> > > > Sergi > > > > > >>> > > > > > > > > >>> > > > 2017-01-12 15:18 GMT+03:00 Alexei Scherbakov < > > > > > >>> > > alexey.scherbak...@gmail.com > > > > > >>> > > > >: > > > > > >>> > > > > > > > > >>> > > > > Done. > > > > > >>> > > > > > > > > > >>> > > > > 2017-01-11 20:39 GMT+03:00 Dmitriy Setrakyan < > > > > > >>> dsetrak...@apache.org > > > > > >>> > >: > > > > > >>> > > > > > > > > > >>> > > > > > Alexey, > > > > > >>> > > > > > > > > > > >>> > > > > > I am not sure I am seeing the API changes documented > in > > > the > > > > > >>> ticket. > > > > > >>> > > Can > > > > > >>> > > > > you > > > > > >>> > > > > > please either document them or add GIT links for the > > new > > > > > >>> classes? > > > > > >>> > > > > > > > > > > >>> > > > > > D. > > > > > >>> > > > > > > > > > > >>> > > > > > On Wed, Jan 11, 2017 at 9:29 AM, Alexei Scherbakov < > > > > > >>> > > > > > alexey.scherbak...@gmail.com> wrote: > > > > > >>> > > > > > > > > > > >>> > > > > > > Guys, > > > > > >>> > > > > > > > > > > > >>> > > > > > > I've just submitted a PR for > > > > > >>> > > > > > > https://issues.apache.org/jira/browse/IGNITE-4523. > > > > > >>> > > > > > > > > > > > >>> > > > > > > Please review API changes while waiting for TC > > results. > > > > > >>> > > > > > > > > > > > >>> > > > > > > -- > > > > > >>> > > > > > > > > > > > >>> > > > > > > Best regards, > > > > > >>> > > > > > > Alexei Scherbakov > > > > > >>> > > > > > > > > > > > >>> > > > > > > > > > > >>> > > > > > > > > > >>> > > > > > > > > > >>> > > > > > > > > > >>> > > > > -- > > > > > >>> > > > > > > > > > >>> > > > > Best regards, > > > > > >>> > > > > Alexei Scherbakov > > > > > >>> > > > > > > > > > >>> > > > > > > > > >>> > > > > > > > >>> > > > > > > >>> > > > > > > >>> > > > > > > >>> > -- > > > > > >>> > > > > > > >>> > Best regards, > > > > > >>> > Alexei Scherbakov > > > > > >>> > > > > > > >>> > > > > > >> > > > > > >> > > > > > >> > > > > > >> -- > > > > > >> > > > > > >> Best regards, > > > > > >> Alexei Scherbakov > > > > > >> > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > > > > > > Best regards, > > > > > > Alexei Scherbakov > > > > > > > > > > > > > > > > > > > > > > > > > > -- > > > > > > > > > > Best regards, > > > > > Alexei Scherbakov > > > > > > > > > > > > > > > > > > > > -- > > > > Best regards, > > Alexei Scherbakov > > > -- Best regards, Alexei Scherbakov