You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ignite.apache.org by Nikolay Izhikov <ni...@gmail.com> on 2017/08/29 10:36:53 UTC

ContinuousQueryWithTransformer implementation questions - 2

Hello, Igniters.

I'm working on IGNITE-425 [1] issue.

Text of issue:

===
Currently if updated entry passes the filter, it is sent to node 
initiated the query entirely. It would be good to provide user with the 
ability to transform entry and, for example, select only fields that are 
important. This may bring huge economy to traffic and lower GC pressure 
as well.
===

My pull request [2] is ready.
Anton Vinogradov is OK with it.
Nikolay Tikhonov reviewed my changes and want to discuss changes related 
to public API with community.

1. I introduce new query class - ContinuousQueryWithTransformer [3].
Reasons:
* ContinuousQuery is final so user can't extends it. I don't want to
change that.
* ContinuousQuery contains some deprecated methods(setRemoteFilter) so 
with new class we can get rid of them.
* Such public API design disallow usage of existing localEventListener
with new transformedEventListenr in compile time.

Thoughts?

2. What behavior is expected if transformer throws exception for some 
event? I see following options:

* Pass `null` to listener(pull request implementation).
* Skip event. Don't call listener.
* Introduce special callback. onTransformError?

Thoughts?

```
public final class ContinuousQueryWithTransformer<K, V, T> extends 
Query<Cache.Entry<K, V>> {
     //...

     private Factory<? extends CacheEntryEventFilter<K, V>> 
rmtFilterFactory;

     private Factory<? extends IgniteClosure<CacheEntryEvent<? extends 
K, ? extends V>, T>> rmtTransFactory;

     private EventListener<T> locLsnr;

     //...

     public interface EventListener<T> {
         void onUpdated(Iterable<? extends T> events);
     }
}
```

Previous discussion - [4]

[1] https://issues.apache.org/jira/browse/IGNITE-425
[2] https://github.com/apache/ignite/pull/2372
[3] 
https://github.com/apache/ignite/pull/2372/files#diff-22cc0cf0bc428b32a39e6cc0b22b0e3e
[4] 
http://apache-ignite-developers.2346864.n4.nabble.com/ContinuousQueryWithTransformer-implementation-questions-td20078.html

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Hello, guys.

Anton, Yakov, can you, please, share your wisdom, and make final review of
IGNITE-425

Task: https://issues.apache.org/jira/browse/IGNITE-425
PR: https://github.com/apache/ignite/pull/2372

2017-09-18 18:33 GMT+03:00 Николай Ижиков <ni...@gmail.com>:

> So, resuming:
>
> 1) My solution reduces network communication.
> As far as I know, a lot of people want to have this feature at Ignite 2.x.
> It's impossible to gain perfect API right now, it will take months to gain
> it.
> My solution ready right now!, let's merge it and refactor whole Continuous
> Query API at 3.0.
>
> 2) Current API is bad, and, yes, my changes make it a little bit
> complicated.
> But, complication minimized as possible and profit much bigger that
> complication.
>
> 2017-09-18 17:39 GMT+03:00 Николай Ижиков <ni...@gmail.com>:
>
>> Vladimir,
>>
>> Here is a short summary what is wrong with continuous query....
>>
>>
>> OK.
>> I agree - this is problems of current API.
>>
>> How we can fix it by not merging ContinuousQueryWIthTransformer?
>> How we can quickly design, discuss and implement new API?
>> Because at the moment there is no any ticket to start working at.
>> Moreover we can't throw ContinuousQuery away until 3.0 version.
>>
>> > What is worse, this interface is inconsistent with JCache event
>> listeners, which distinguish between create, update and delete events.
>>
>> Can't agree with you.
>>
>> 1. As far as I know jcache doesn't have any Transformer conception.
>> 2. We can distinguish create, update and delete events in transformer and
>> we can push that knowledge to listener.
>>
>>
>>> What I see in your PR is that we add one more confusing concept - in
>>> addition to wrongly named "local listener" now we will also have
>>> "TransformedEventListener".
>>>
>>
>> I think usage of jcache API in some Ignite-specific classes is one more
>> issue of existing ContinuousQuery.
>> I think we must use Ignite only API for Ignite only features and use some
>> wrappers to provide external API support.
>>
>> For these reasons I would still prefer to think of better continuous
>>> queries design first instead of making current API even more complicated.
>>>
>>
>> I think the main reason for some feature is to provide value to the user.
>> Transformers adds value to a product because usage of transformer can
>> lead to significant performance win.
>>
>>
>>> Vladimir.
>>>
>>> On Mon, Sep 18, 2017 at 4:04 PM, Николай Ижиков <ni...@gmail.com>
>>> wrote:
>>>
>>> > Igniters,
>>> >
>>> > I discussed API of ContinuousQuery and ContinuousQueryWithTransformer
>>> with
>>> > Anton Vinogradov one more time.
>>> >
>>> > Since users who use regular ContinuousQuery already knows pros. and
>>> cons of
>>> > using initialQuery and to not to complicate API more and more until
>>> 3.0 we
>>> > agreed that best choice is to stay with existing initialQuery that
>>> return
>>> > Cache.Entry<K, V> for ContinuousQueryWithTransformer.
>>> >
>>> > Notice that initialQuery is not required and can be null.
>>> >
>>> > Thoughts?
>>> >
>>> > 2017-09-15 1:45 GMT+03:00 Denis Magda <dm...@apache.org>:
>>> >
>>> > > Vladimir,
>>> > >
>>> > > If the API is so bad then it might take much more time to make up and
>>> > roll
>>> > > out the new. Plus, there should be a community member who is ready to
>>> > take
>>> > > it over. My suggestion would be to accept this contribution and
>>> initiate
>>> > an
>>> > > activity towards the new API if you like.
>>> > >
>>> > > Personally, I considered this API as one of the most vivid we have
>>> basing
>>> > > on my practical usage experience. I was aware of initial query’s
>>> pitfalls
>>> > > but isn’t it something we can put on paper?
>>> > >
>>> > > —
>>> > > Denis
>>> > >
>>> > > > On Sep 12, 2017, at 6:04 AM, Vladimir Ozerov <vozerov@gridgain.com
>>> >
>>> > > wrote:
>>> > > >
>>> > > > My opinion is that our query API is big piece of ... you know,
>>> > especially
>>> > > > ContinuousQuery. A lot of concepts and features are mixed in a
>>> single
>>> > > > entity, what makes it hard to understand and use. Let's finally
>>> > deprecate
>>> > > > ContinuousQuery and design nice and consistent API. E.g.:
>>> > > >
>>> > > > interface IgniteCache {
>>> > > >    UUID addListener(CacheEntryListener listener)
>>> > > >    void removeListener(UUID listenerId);
>>> > > > }
>>> > > >
>>> > > > This method set's a listener on all nodes which will process event
>>> > > locally,
>>> > > > no network communication. Now if you want semantics similar to
>>> existing
>>> > > > continuous queries, you use special entry listener type:
>>> > > >
>>> > > > class ContinuousQueryCacheEntryListener implements
>>> CacheEntryListener
>>> > {
>>> > > >    ContinuousQueryRemoteFilter rmtFilter;
>>> > > >    ContinuousQueryRemoteTransformer rmtTransformer;
>>> > > >    ContinuousQueryLocalCallback locCb;
>>> > > > }
>>> > > >
>>> > > > Last, "initial query" concept should be dropped from "continuous
>>> query"
>>> > > > feature completely. It doesn't guarantee any kind of atomicity or
>>> > > > visibility wrt to cache events, so it adds no value. The same
>>> behavior
>>> > > > could be achieved as follows:
>>> > > >
>>> > > > cache.addListener(...)
>>> > > > QueryCursor cursor = cache.query(initialQuery);
>>> > > >
>>> > > > Vladimir.
>>> > > >
>>> > > >
>>> > > > On Tue, Sep 12, 2017 at 3:35 PM, Yakov Zhdanov <
>>> yzhdanov@apache.org>
>>> > > wrote:
>>> > > >
>>> > > >> Dmitry, can you please take a look at public API change.
>>> > > >>
>>> > > >> Ticket - https://issues.apache.org/jira/browse/IGNITE-425
>>> > > >> PR - https://github.com/apache/ignite/pull/2372
>>> > > >>
>>> > > >> Issues:
>>> > > >> 1. Do you see any other option other than creating separate
>>> class? As
>>> > > for
>>> > > >> me I don't.
>>> > > >> 2. In a new class we still have initial query which uses <K, V>
>>> types
>>> > > which
>>> > > >> is questionable.
>>> > > >>
>>> > > >> Igniters, please share your thoughts as well. Public API is the
>>> face
>>> > of
>>> > > our
>>> > > >> product we need to make it as convenient and consistent as we can.
>>> > > >>
>>> > > >> --Yakov
>>> > > >>
>>> > >
>>> > >
>>> >
>>> >
>>> > --
>>> > Nikolay Izhikov
>>> > NIzhikov.dev@gmail.com
>>> >
>>>
>>
>>
>>
>> --
>> Nikolay Izhikov
>> NIzhikov.dev@gmail.com
>>
>
>
>
> --
> Nikolay Izhikov
> NIzhikov.dev@gmail.com
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
So, resuming:

1) My solution reduces network communication.
As far as I know, a lot of people want to have this feature at Ignite 2.x.
It's impossible to gain perfect API right now, it will take months to gain
it.
My solution ready right now!, let's merge it and refactor whole Continuous
Query API at 3.0.

2) Current API is bad, and, yes, my changes make it a little bit
complicated.
But, complication minimized as possible and profit much bigger that
complication.

2017-09-18 17:39 GMT+03:00 Николай Ижиков <ni...@gmail.com>:

> Vladimir,
>
> Here is a short summary what is wrong with continuous query....
>
>
> OK.
> I agree - this is problems of current API.
>
> How we can fix it by not merging ContinuousQueryWIthTransformer?
> How we can quickly design, discuss and implement new API?
> Because at the moment there is no any ticket to start working at.
> Moreover we can't throw ContinuousQuery away until 3.0 version.
>
> > What is worse, this interface is inconsistent with JCache event
> listeners, which distinguish between create, update and delete events.
>
> Can't agree with you.
>
> 1. As far as I know jcache doesn't have any Transformer conception.
> 2. We can distinguish create, update and delete events in transformer and
> we can push that knowledge to listener.
>
>
>> What I see in your PR is that we add one more confusing concept - in
>> addition to wrongly named "local listener" now we will also have
>> "TransformedEventListener".
>>
>
> I think usage of jcache API in some Ignite-specific classes is one more
> issue of existing ContinuousQuery.
> I think we must use Ignite only API for Ignite only features and use some
> wrappers to provide external API support.
>
> For these reasons I would still prefer to think of better continuous
>> queries design first instead of making current API even more complicated.
>>
>
> I think the main reason for some feature is to provide value to the user.
> Transformers adds value to a product because usage of transformer can lead
> to significant performance win.
>
>
>> Vladimir.
>>
>> On Mon, Sep 18, 2017 at 4:04 PM, Николай Ижиков <ni...@gmail.com>
>> wrote:
>>
>> > Igniters,
>> >
>> > I discussed API of ContinuousQuery and ContinuousQueryWithTransformer
>> with
>> > Anton Vinogradov one more time.
>> >
>> > Since users who use regular ContinuousQuery already knows pros. and
>> cons of
>> > using initialQuery and to not to complicate API more and more until 3.0
>> we
>> > agreed that best choice is to stay with existing initialQuery that
>> return
>> > Cache.Entry<K, V> for ContinuousQueryWithTransformer.
>> >
>> > Notice that initialQuery is not required and can be null.
>> >
>> > Thoughts?
>> >
>> > 2017-09-15 1:45 GMT+03:00 Denis Magda <dm...@apache.org>:
>> >
>> > > Vladimir,
>> > >
>> > > If the API is so bad then it might take much more time to make up and
>> > roll
>> > > out the new. Plus, there should be a community member who is ready to
>> > take
>> > > it over. My suggestion would be to accept this contribution and
>> initiate
>> > an
>> > > activity towards the new API if you like.
>> > >
>> > > Personally, I considered this API as one of the most vivid we have
>> basing
>> > > on my practical usage experience. I was aware of initial query’s
>> pitfalls
>> > > but isn’t it something we can put on paper?
>> > >
>> > > —
>> > > Denis
>> > >
>> > > > On Sep 12, 2017, at 6:04 AM, Vladimir Ozerov <vo...@gridgain.com>
>> > > wrote:
>> > > >
>> > > > My opinion is that our query API is big piece of ... you know,
>> > especially
>> > > > ContinuousQuery. A lot of concepts and features are mixed in a
>> single
>> > > > entity, what makes it hard to understand and use. Let's finally
>> > deprecate
>> > > > ContinuousQuery and design nice and consistent API. E.g.:
>> > > >
>> > > > interface IgniteCache {
>> > > >    UUID addListener(CacheEntryListener listener)
>> > > >    void removeListener(UUID listenerId);
>> > > > }
>> > > >
>> > > > This method set's a listener on all nodes which will process event
>> > > locally,
>> > > > no network communication. Now if you want semantics similar to
>> existing
>> > > > continuous queries, you use special entry listener type:
>> > > >
>> > > > class ContinuousQueryCacheEntryListener implements
>> CacheEntryListener
>> > {
>> > > >    ContinuousQueryRemoteFilter rmtFilter;
>> > > >    ContinuousQueryRemoteTransformer rmtTransformer;
>> > > >    ContinuousQueryLocalCallback locCb;
>> > > > }
>> > > >
>> > > > Last, "initial query" concept should be dropped from "continuous
>> query"
>> > > > feature completely. It doesn't guarantee any kind of atomicity or
>> > > > visibility wrt to cache events, so it adds no value. The same
>> behavior
>> > > > could be achieved as follows:
>> > > >
>> > > > cache.addListener(...)
>> > > > QueryCursor cursor = cache.query(initialQuery);
>> > > >
>> > > > Vladimir.
>> > > >
>> > > >
>> > > > On Tue, Sep 12, 2017 at 3:35 PM, Yakov Zhdanov <yzhdanov@apache.org
>> >
>> > > wrote:
>> > > >
>> > > >> Dmitry, can you please take a look at public API change.
>> > > >>
>> > > >> Ticket - https://issues.apache.org/jira/browse/IGNITE-425
>> > > >> PR - https://github.com/apache/ignite/pull/2372
>> > > >>
>> > > >> Issues:
>> > > >> 1. Do you see any other option other than creating separate class?
>> As
>> > > for
>> > > >> me I don't.
>> > > >> 2. In a new class we still have initial query which uses <K, V>
>> types
>> > > which
>> > > >> is questionable.
>> > > >>
>> > > >> Igniters, please share your thoughts as well. Public API is the
>> face
>> > of
>> > > our
>> > > >> product we need to make it as convenient and consistent as we can.
>> > > >>
>> > > >> --Yakov
>> > > >>
>> > >
>> > >
>> >
>> >
>> > --
>> > Nikolay Izhikov
>> > NIzhikov.dev@gmail.com
>> >
>>
>
>
>
> --
> Nikolay Izhikov
> NIzhikov.dev@gmail.com
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Vladimir,

Here is a short summary what is wrong with continuous query....


OK.
I agree - this is problems of current API.

How we can fix it by not merging ContinuousQueryWIthTransformer?
How we can quickly design, discuss and implement new API?
Because at the moment there is no any ticket to start working at.
Moreover we can't throw ContinuousQuery away until 3.0 version.

> What is worse, this interface is inconsistent with JCache event
listeners, which distinguish between create, update and delete events.

Can't agree with you.

1. As far as I know jcache doesn't have any Transformer conception.
2. We can distinguish create, update and delete events in transformer and
we can push that knowledge to listener.


> What I see in your PR is that we add one more confusing concept - in
> addition to wrongly named "local listener" now we will also have
> "TransformedEventListener".
>

I think usage of jcache API in some Ignite-specific classes is one more
issue of existing ContinuousQuery.
I think we must use Ignite only API for Ignite only features and use some
wrappers to provide external API support.

For these reasons I would still prefer to think of better continuous
> queries design first instead of making current API even more complicated.
>

I think the main reason for some feature is to provide value to the user.
Transformers adds value to a product because usage of transformer can lead
to significant performance win.


> Vladimir.
>
> On Mon, Sep 18, 2017 at 4:04 PM, Николай Ижиков <ni...@gmail.com>
> wrote:
>
> > Igniters,
> >
> > I discussed API of ContinuousQuery and ContinuousQueryWithTransformer
> with
> > Anton Vinogradov one more time.
> >
> > Since users who use regular ContinuousQuery already knows pros. and cons
> of
> > using initialQuery and to not to complicate API more and more until 3.0
> we
> > agreed that best choice is to stay with existing initialQuery that return
> > Cache.Entry<K, V> for ContinuousQueryWithTransformer.
> >
> > Notice that initialQuery is not required and can be null.
> >
> > Thoughts?
> >
> > 2017-09-15 1:45 GMT+03:00 Denis Magda <dm...@apache.org>:
> >
> > > Vladimir,
> > >
> > > If the API is so bad then it might take much more time to make up and
> > roll
> > > out the new. Plus, there should be a community member who is ready to
> > take
> > > it over. My suggestion would be to accept this contribution and
> initiate
> > an
> > > activity towards the new API if you like.
> > >
> > > Personally, I considered this API as one of the most vivid we have
> basing
> > > on my practical usage experience. I was aware of initial query’s
> pitfalls
> > > but isn’t it something we can put on paper?
> > >
> > > —
> > > Denis
> > >
> > > > On Sep 12, 2017, at 6:04 AM, Vladimir Ozerov <vo...@gridgain.com>
> > > wrote:
> > > >
> > > > My opinion is that our query API is big piece of ... you know,
> > especially
> > > > ContinuousQuery. A lot of concepts and features are mixed in a single
> > > > entity, what makes it hard to understand and use. Let's finally
> > deprecate
> > > > ContinuousQuery and design nice and consistent API. E.g.:
> > > >
> > > > interface IgniteCache {
> > > >    UUID addListener(CacheEntryListener listener)
> > > >    void removeListener(UUID listenerId);
> > > > }
> > > >
> > > > This method set's a listener on all nodes which will process event
> > > locally,
> > > > no network communication. Now if you want semantics similar to
> existing
> > > > continuous queries, you use special entry listener type:
> > > >
> > > > class ContinuousQueryCacheEntryListener implements
> CacheEntryListener
> > {
> > > >    ContinuousQueryRemoteFilter rmtFilter;
> > > >    ContinuousQueryRemoteTransformer rmtTransformer;
> > > >    ContinuousQueryLocalCallback locCb;
> > > > }
> > > >
> > > > Last, "initial query" concept should be dropped from "continuous
> query"
> > > > feature completely. It doesn't guarantee any kind of atomicity or
> > > > visibility wrt to cache events, so it adds no value. The same
> behavior
> > > > could be achieved as follows:
> > > >
> > > > cache.addListener(...)
> > > > QueryCursor cursor = cache.query(initialQuery);
> > > >
> > > > Vladimir.
> > > >
> > > >
> > > > On Tue, Sep 12, 2017 at 3:35 PM, Yakov Zhdanov <yz...@apache.org>
> > > wrote:
> > > >
> > > >> Dmitry, can you please take a look at public API change.
> > > >>
> > > >> Ticket - https://issues.apache.org/jira/browse/IGNITE-425
> > > >> PR - https://github.com/apache/ignite/pull/2372
> > > >>
> > > >> Issues:
> > > >> 1. Do you see any other option other than creating separate class?
> As
> > > for
> > > >> me I don't.
> > > >> 2. In a new class we still have initial query which uses <K, V>
> types
> > > which
> > > >> is questionable.
> > > >>
> > > >> Igniters, please share your thoughts as well. Public API is the face
> > of
> > > our
> > > >> product we need to make it as convenient and consistent as we can.
> > > >>
> > > >> --Yakov
> > > >>
> > >
> > >
> >
> >
> > --
> > Nikolay Izhikov
> > NIzhikov.dev@gmail.com
> >
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Vladimir Ozerov <vo...@gridgain.com>.
Nikolay,

Here is a short summary what is wrong with continuous query:
1) It should not have "initialQuery"
2) It should not be called through IgniteCache.query() method, as it has
nothing in common with other "query" types
3) Main thing: our listeners are *ALWAYS* executed on a node which
initiated the query.

p.3 is the major problem. JCache specification doesn't define where
listeners should be invoked. Should we have ability to execute them on data
nodes, there would be much less demand in transformers.

What I see in your PR is that we add one more confusing concept - in
addition to wrongly named "local listener" now we will also have
"TransformedEventListener". What is worse, this interface is inconsistent
with JCache event listeners, which distinguish between create, update and
delete events.

For these reasons I would still prefer to think of better continuous
queries design first instead of making current API even more complicated.

Vladimir.

On Mon, Sep 18, 2017 at 4:04 PM, Николай Ижиков <ni...@gmail.com>
wrote:

> Igniters,
>
> I discussed API of ContinuousQuery and ContinuousQueryWithTransformer with
> Anton Vinogradov one more time.
>
> Since users who use regular ContinuousQuery already knows pros. and cons of
> using initialQuery and to not to complicate API more and more until 3.0 we
> agreed that best choice is to stay with existing initialQuery that return
> Cache.Entry<K, V> for ContinuousQueryWithTransformer.
>
> Notice that initialQuery is not required and can be null.
>
> Thoughts?
>
> 2017-09-15 1:45 GMT+03:00 Denis Magda <dm...@apache.org>:
>
> > Vladimir,
> >
> > If the API is so bad then it might take much more time to make up and
> roll
> > out the new. Plus, there should be a community member who is ready to
> take
> > it over. My suggestion would be to accept this contribution and initiate
> an
> > activity towards the new API if you like.
> >
> > Personally, I considered this API as one of the most vivid we have basing
> > on my practical usage experience. I was aware of initial query’s pitfalls
> > but isn’t it something we can put on paper?
> >
> > —
> > Denis
> >
> > > On Sep 12, 2017, at 6:04 AM, Vladimir Ozerov <vo...@gridgain.com>
> > wrote:
> > >
> > > My opinion is that our query API is big piece of ... you know,
> especially
> > > ContinuousQuery. A lot of concepts and features are mixed in a single
> > > entity, what makes it hard to understand and use. Let's finally
> deprecate
> > > ContinuousQuery and design nice and consistent API. E.g.:
> > >
> > > interface IgniteCache {
> > >    UUID addListener(CacheEntryListener listener)
> > >    void removeListener(UUID listenerId);
> > > }
> > >
> > > This method set's a listener on all nodes which will process event
> > locally,
> > > no network communication. Now if you want semantics similar to existing
> > > continuous queries, you use special entry listener type:
> > >
> > > class ContinuousQueryCacheEntryListener implements CacheEntryListener
> {
> > >    ContinuousQueryRemoteFilter rmtFilter;
> > >    ContinuousQueryRemoteTransformer rmtTransformer;
> > >    ContinuousQueryLocalCallback locCb;
> > > }
> > >
> > > Last, "initial query" concept should be dropped from "continuous query"
> > > feature completely. It doesn't guarantee any kind of atomicity or
> > > visibility wrt to cache events, so it adds no value. The same behavior
> > > could be achieved as follows:
> > >
> > > cache.addListener(...)
> > > QueryCursor cursor = cache.query(initialQuery);
> > >
> > > Vladimir.
> > >
> > >
> > > On Tue, Sep 12, 2017 at 3:35 PM, Yakov Zhdanov <yz...@apache.org>
> > wrote:
> > >
> > >> Dmitry, can you please take a look at public API change.
> > >>
> > >> Ticket - https://issues.apache.org/jira/browse/IGNITE-425
> > >> PR - https://github.com/apache/ignite/pull/2372
> > >>
> > >> Issues:
> > >> 1. Do you see any other option other than creating separate class? As
> > for
> > >> me I don't.
> > >> 2. In a new class we still have initial query which uses <K, V> types
> > which
> > >> is questionable.
> > >>
> > >> Igniters, please share your thoughts as well. Public API is the face
> of
> > our
> > >> product we need to make it as convenient and consistent as we can.
> > >>
> > >> --Yakov
> > >>
> >
> >
>
>
> --
> Nikolay Izhikov
> NIzhikov.dev@gmail.com
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Igniters,

I discussed API of ContinuousQuery and ContinuousQueryWithTransformer with
Anton Vinogradov one more time.

Since users who use regular ContinuousQuery already knows pros. and cons of
using initialQuery and to not to complicate API more and more until 3.0 we
agreed that best choice is to stay with existing initialQuery that return
Cache.Entry<K, V> for ContinuousQueryWithTransformer.

Notice that initialQuery is not required and can be null.

Thoughts?

2017-09-15 1:45 GMT+03:00 Denis Magda <dm...@apache.org>:

> Vladimir,
>
> If the API is so bad then it might take much more time to make up and roll
> out the new. Plus, there should be a community member who is ready to take
> it over. My suggestion would be to accept this contribution and initiate an
> activity towards the new API if you like.
>
> Personally, I considered this API as one of the most vivid we have basing
> on my practical usage experience. I was aware of initial query’s pitfalls
> but isn’t it something we can put on paper?
>
> —
> Denis
>
> > On Sep 12, 2017, at 6:04 AM, Vladimir Ozerov <vo...@gridgain.com>
> wrote:
> >
> > My opinion is that our query API is big piece of ... you know, especially
> > ContinuousQuery. A lot of concepts and features are mixed in a single
> > entity, what makes it hard to understand and use. Let's finally deprecate
> > ContinuousQuery and design nice and consistent API. E.g.:
> >
> > interface IgniteCache {
> >    UUID addListener(CacheEntryListener listener)
> >    void removeListener(UUID listenerId);
> > }
> >
> > This method set's a listener on all nodes which will process event
> locally,
> > no network communication. Now if you want semantics similar to existing
> > continuous queries, you use special entry listener type:
> >
> > class ContinuousQueryCacheEntryListener implements CacheEntryListener {
> >    ContinuousQueryRemoteFilter rmtFilter;
> >    ContinuousQueryRemoteTransformer rmtTransformer;
> >    ContinuousQueryLocalCallback locCb;
> > }
> >
> > Last, "initial query" concept should be dropped from "continuous query"
> > feature completely. It doesn't guarantee any kind of atomicity or
> > visibility wrt to cache events, so it adds no value. The same behavior
> > could be achieved as follows:
> >
> > cache.addListener(...)
> > QueryCursor cursor = cache.query(initialQuery);
> >
> > Vladimir.
> >
> >
> > On Tue, Sep 12, 2017 at 3:35 PM, Yakov Zhdanov <yz...@apache.org>
> wrote:
> >
> >> Dmitry, can you please take a look at public API change.
> >>
> >> Ticket - https://issues.apache.org/jira/browse/IGNITE-425
> >> PR - https://github.com/apache/ignite/pull/2372
> >>
> >> Issues:
> >> 1. Do you see any other option other than creating separate class? As
> for
> >> me I don't.
> >> 2. In a new class we still have initial query which uses <K, V> types
> which
> >> is questionable.
> >>
> >> Igniters, please share your thoughts as well. Public API is the face of
> our
> >> product we need to make it as convenient and consistent as we can.
> >>
> >> --Yakov
> >>
>
>


-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Denis Magda <dm...@apache.org>.
Vladimir,

If the API is so bad then it might take much more time to make up and roll out the new. Plus, there should be a community member who is ready to take it over. My suggestion would be to accept this contribution and initiate an activity towards the new API if you like.

Personally, I considered this API as one of the most vivid we have basing on my practical usage experience. I was aware of initial query’s pitfalls but isn’t it something we can put on paper?

—
Denis

> On Sep 12, 2017, at 6:04 AM, Vladimir Ozerov <vo...@gridgain.com> wrote:
> 
> My opinion is that our query API is big piece of ... you know, especially
> ContinuousQuery. A lot of concepts and features are mixed in a single
> entity, what makes it hard to understand and use. Let's finally deprecate
> ContinuousQuery and design nice and consistent API. E.g.:
> 
> interface IgniteCache {
>    UUID addListener(CacheEntryListener listener)
>    void removeListener(UUID listenerId);
> }
> 
> This method set's a listener on all nodes which will process event locally,
> no network communication. Now if you want semantics similar to existing
> continuous queries, you use special entry listener type:
> 
> class ContinuousQueryCacheEntryListener implements CacheEntryListener {
>    ContinuousQueryRemoteFilter rmtFilter;
>    ContinuousQueryRemoteTransformer rmtTransformer;
>    ContinuousQueryLocalCallback locCb;
> }
> 
> Last, "initial query" concept should be dropped from "continuous query"
> feature completely. It doesn't guarantee any kind of atomicity or
> visibility wrt to cache events, so it adds no value. The same behavior
> could be achieved as follows:
> 
> cache.addListener(...)
> QueryCursor cursor = cache.query(initialQuery);
> 
> Vladimir.
> 
> 
> On Tue, Sep 12, 2017 at 3:35 PM, Yakov Zhdanov <yz...@apache.org> wrote:
> 
>> Dmitry, can you please take a look at public API change.
>> 
>> Ticket - https://issues.apache.org/jira/browse/IGNITE-425
>> PR - https://github.com/apache/ignite/pull/2372
>> 
>> Issues:
>> 1. Do you see any other option other than creating separate class? As for
>> me I don't.
>> 2. In a new class we still have initial query which uses <K, V> types which
>> is questionable.
>> 
>> Igniters, please share your thoughts as well. Public API is the face of our
>> product we need to make it as convenient and consistent as we can.
>> 
>> --Yakov
>> 


Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Nikolay Izhikov <ni...@gmail.com>.
Hello, Vova.

Can you, please, share your knowledge: Why we have abandon ContinuousQuery?

> Guys.
> 
> I'm new in the project so can someone more experienced tell me:
> 
> What's is wrong with the current implementation of ContinuousQuery?
> 
> 1. initialQuery is useless - OK, understood.
> 
> What else is wrong?
> 
> 
> 2017-09-12 20:02 GMT+03:00 Николай Ижиков <nizhikov.dev@gmail.com 
> <ma...@gmail.com>>:
> 
>     Vova,
> 
>     > Public API is the face of our product. 
>     > We cannot and do not want to change it in a rush.
>     > It is not a big problem if we spend additional week or month for API
>     design
> 
>     Fully agreed.
> 
>     I'm not trying to speed up changes, all I try is separate two
>     discussions:
> 
>     * ticket implementation based on existing API
>     * design of new API
> 
>     > It is much better *than* extend*ing* confusing behavior *of *already
>     confusing and overengineered API
> 
>     Ignite already have a ContinuousQuery
>     It's a matter of fact.
>     Ticket goal is provide some useful feature to the user.
>     I think it a good thing.
> 
>     Can you list confusions that added by ticket implementation?
> 
> 
>     2017-09-12 19:47 GMT+03:00 Vladimir Ozerov <vozerov@gridgain.com
>     <ma...@gridgain.com>>:
> 
>         I meant "It is much better *than* extend*ing* confusing behavior
>         *of *already
>         confusing and overengineered API."
> 
>         On Tue, Sep 12, 2017 at 7:35 PM, Vladimir Ozerov
>         <vozerov@gridgain.com <ma...@gridgain.com>>
>         wrote:
> 
>          > Nikolay,
>          >
>          > Public API is the face of our product. We cannot and do not
>         want to change
>          > it in a rush. This ticket was in a backlog for more than 2
>         years. It is not
>          > a big problem if we spend additional week or month for API
>         design. It is
>          > much better to extend confusing behavior on already confusing and
>          > overengineered API.
>          >
>          > On Tue, Sep 12, 2017 at 6:47 PM, Николай Ижиков
>         <nizhikov.dev@gmail.com <ma...@gmail.com>>
>          > wrote:
>          >
>          >> Vova
>          >>
>          >> > I propose to deprecate current continuous queries and
>         develop new API.
>          >> > This should not break anything.
>          >>
>          >> If the community agrees that *whole* continuous query API is
>         bad - it OK.
>          >> Let's develop new API.
>          >>
>          >> But developing new public API and implementing it is a very
>         long process.
>          >> One can see it based on this thread :)
>          >>
>          >> I think my implementation [1] of transformers for
>         ContinuousQuery make
>          >> things better for a user because remote transformers can lead to
>          >> significant performance win.
>          >>
>          >> > Adding transformers on top of current API will make it
>         total mess.
>          >>
>          >> I propose two things:
>          >>
>          >> 1. Continue discussion of current task [2] with scope
>         limited by current
>          >> API.
>          >> 2. Start a discussion and work on new API. I think we can
>         start with
>          >> listing things that make current API bad. I can drive such a
>         discussion.
>          >>
>          >> [1] https://github.com/apache/ignite/pull/2372
>         <https://github.com/apache/ignite/pull/2372>
>          >> [2] https://issues.apache.org/jira/browse/IGNITE-425
>         <https://issues.apache.org/jira/browse/IGNITE-425>
>          >>
>          >>
>          >> 2017-09-12 17:55 GMT+03:00 Dmitriy Setrakyan
>         <dsetrakyan@apache.org <ma...@apache.org>>:
>          >>
>          >> > Vladimir, are their factories for the proposed listeners?
>          >> >
>          >> > On Tue, Sep 12, 2017 at 7:52 AM, Alexey Goncharuk <
>          >> > alexey.goncharuk@gmail.com
>         <ma...@gmail.com>> wrote:
>          >> >
>          >> > > Vladimir,
>          >> > >
>          >> > > Can you please clarify how the proposed API will work?
>          >> > >
>          >> > > My opinion is that our query API is big piece of ... you
>         know,
>          >> especially
>          >> > > > ContinuousQuery. A lot of concepts and features are
>         mixed in a
>          >> single
>          >> > > > entity, what makes it hard to understand and use.
>         Let's finally
>          >> > deprecate
>          >> > > > ContinuousQuery and design nice and consistent API. E.g.:
>          >> > > >
>          >> > > > interface IgniteCache {
>          >> > > >     UUID addListener(CacheEntryListener listener)
>          >> > > >     void removeListener(UUID listenerId);
>          >> > > > }
>          >> > > >
>          >> > > > This method set's a listener on all nodes which will
>         process event
>          >> > > locally,
>          >> > > > no network communication.
>          >> > >
>          >> > >
>          >> > > Do I understand correctly that CacheEntryListener will
>         have a method
>          >> like
>          >> > > onEvent() which will accept the cache event?
>          >> > >
>          >> > >
>          >> > > > Now if you want semantics similar to existing
>          >> > > > continuous queries, you use special entry listener type:
>          >> > > >
>          >> > > > class ContinuousQueryCacheEntryListener implements
>          >> CacheEntryListener
>          >> > {
>          >> > > >     ContinuousQueryRemoteFilter rmtFilter;
>          >> > > > ContinuousQueryRemoteTransformer rmtTransformer;
>          >> > > >     ContinuousQueryLocalCallback locCb;
>          >> > > > }
>          >> > > >
>          >> > > >
>          >> > > This becomes confusing: while the
>         ContinuousQueryCacheEntryListener
>          >> > itself
>          >> > > has the onEvent() method, which is supposed to be called
>         on event
>          >> nodes,
>          >> > it
>          >> > > also has a rmtFilter, which will also be called on event
>         nodes. Will
>          >> the
>          >> > > onEvent() then invoked on the listener anyway,
>         regardless of the
>          >> filter
>          >> > > result? Finally, the listener will have a local callback
>         field, which
>          >> > will
>          >> > > be called on the originating node. This sounds way more
>         tricky to me
>          >> than
>          >> > > the current API.
>          >> > >
>          >> > >
>          >> > > > Last, "initial query" concept should be dropped from
>         "continuous
>          >> query"
>          >> > > > feature completely. It doesn't guarantee any kind of
>         atomicity or
>          >> > > > visibility wrt to cache events, so it adds no value.
>         The same
>          >> behavior
>          >> > > > could be achieved as follows:
>          >> > > >
>          >> > > > cache.addListener(...)
>          >> > > > QueryCursor cursor = cache.query(initialQuery);
>          >> > > >
>          >> > > >
>          >> > > Agree with this.
>          >> > >
>          >> >
>          >>
>          >>
>          >>
>          >> --
>          >> Nikolay Izhikov
>          >> NIzhikov.dev@gmail.com <ma...@gmail.com>
>          >>
>          >
>          >
> 
> 
> 
> 
>     -- 
>     Nikolay Izhikov
>     NIzhikov.dev@gmail.com <ma...@gmail.com>
> 
> 
> 
> 
> -- 
> Nikolay Izhikov
> NIzhikov.dev@gmail.com <ma...@gmail.com>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Guys.

I'm new in the project so can someone more experienced tell me:

What's is wrong with the current implementation of ContinuousQuery?

1. initialQuery is useless - OK, understood.

What else is wrong?


2017-09-12 20:02 GMT+03:00 Николай Ижиков <ni...@gmail.com>:

> Vova,
>
> > Public API is the face of our product.
> > We cannot and do not want to change it in a rush.
> > It is not a big problem if we spend additional week or month for API
> design
>
> Fully agreed.
>
> I'm not trying to speed up changes, all I try is separate two discussions:
>
> * ticket implementation based on existing API
> * design of new API
>
> > It is much better *than* extend*ing* confusing behavior *of *already confusing
> and overengineered API
>
> Ignite already have a ContinuousQuery
> It's a matter of fact.
> Ticket goal is provide some useful feature to the user.
> I think it a good thing.
>
> Can you list confusions that added by ticket implementation?
>
>
> 2017-09-12 19:47 GMT+03:00 Vladimir Ozerov <vo...@gridgain.com>:
>
>> I meant "It is much better *than* extend*ing* confusing behavior *of
>> *already
>> confusing and overengineered API."
>>
>> On Tue, Sep 12, 2017 at 7:35 PM, Vladimir Ozerov <vo...@gridgain.com>
>> wrote:
>>
>> > Nikolay,
>> >
>> > Public API is the face of our product. We cannot and do not want to
>> change
>> > it in a rush. This ticket was in a backlog for more than 2 years. It is
>> not
>> > a big problem if we spend additional week or month for API design. It is
>> > much better to extend confusing behavior on already confusing and
>> > overengineered API.
>> >
>> > On Tue, Sep 12, 2017 at 6:47 PM, Николай Ижиков <nizhikov.dev@gmail.com
>> >
>> > wrote:
>> >
>> >> Vova
>> >>
>> >> > I propose to deprecate current continuous queries and develop new
>> API.
>> >> > This should not break anything.
>> >>
>> >> If the community agrees that *whole* continuous query API is bad - it
>> OK.
>> >> Let's develop new API.
>> >>
>> >> But developing new public API and implementing it is a very long
>> process.
>> >> One can see it based on this thread :)
>> >>
>> >> I think my implementation [1] of transformers for ContinuousQuery make
>> >> things better for a user because remote transformers can lead to
>> >> significant performance win.
>> >>
>> >> > Adding transformers on top of current API will make it total mess.
>> >>
>> >> I propose two things:
>> >>
>> >> 1. Continue discussion of current task [2] with scope limited by
>> current
>> >> API.
>> >> 2. Start a discussion and work on new API. I think we can start with
>> >> listing things that make current API bad. I can drive such a
>> discussion.
>> >>
>> >> [1] https://github.com/apache/ignite/pull/2372
>> >> [2] https://issues.apache.org/jira/browse/IGNITE-425
>> >>
>> >>
>> >> 2017-09-12 17:55 GMT+03:00 Dmitriy Setrakyan <ds...@apache.org>:
>> >>
>> >> > Vladimir, are their factories for the proposed listeners?
>> >> >
>> >> > On Tue, Sep 12, 2017 at 7:52 AM, Alexey Goncharuk <
>> >> > alexey.goncharuk@gmail.com> wrote:
>> >> >
>> >> > > Vladimir,
>> >> > >
>> >> > > Can you please clarify how the proposed API will work?
>> >> > >
>> >> > > My opinion is that our query API is big piece of ... you know,
>> >> especially
>> >> > > > ContinuousQuery. A lot of concepts and features are mixed in a
>> >> single
>> >> > > > entity, what makes it hard to understand and use. Let's finally
>> >> > deprecate
>> >> > > > ContinuousQuery and design nice and consistent API. E.g.:
>> >> > > >
>> >> > > > interface IgniteCache {
>> >> > > >     UUID addListener(CacheEntryListener listener)
>> >> > > >     void removeListener(UUID listenerId);
>> >> > > > }
>> >> > > >
>> >> > > > This method set's a listener on all nodes which will process
>> event
>> >> > > locally,
>> >> > > > no network communication.
>> >> > >
>> >> > >
>> >> > > Do I understand correctly that CacheEntryListener will have a
>> method
>> >> like
>> >> > > onEvent() which will accept the cache event?
>> >> > >
>> >> > >
>> >> > > > Now if you want semantics similar to existing
>> >> > > > continuous queries, you use special entry listener type:
>> >> > > >
>> >> > > > class ContinuousQueryCacheEntryListener implements
>> >> CacheEntryListener
>> >> > {
>> >> > > >     ContinuousQueryRemoteFilter rmtFilter;
>> >> > > >     ContinuousQueryRemoteTransformer rmtTransformer;
>> >> > > >     ContinuousQueryLocalCallback locCb;
>> >> > > > }
>> >> > > >
>> >> > > >
>> >> > > This becomes confusing: while the ContinuousQueryCacheEntryListe
>> ner
>> >> > itself
>> >> > > has the onEvent() method, which is supposed to be called on event
>> >> nodes,
>> >> > it
>> >> > > also has a rmtFilter, which will also be called on event nodes.
>> Will
>> >> the
>> >> > > onEvent() then invoked on the listener anyway, regardless of the
>> >> filter
>> >> > > result? Finally, the listener will have a local callback field,
>> which
>> >> > will
>> >> > > be called on the originating node. This sounds way more tricky to
>> me
>> >> than
>> >> > > the current API.
>> >> > >
>> >> > >
>> >> > > > Last, "initial query" concept should be dropped from "continuous
>> >> query"
>> >> > > > feature completely. It doesn't guarantee any kind of atomicity or
>> >> > > > visibility wrt to cache events, so it adds no value. The same
>> >> behavior
>> >> > > > could be achieved as follows:
>> >> > > >
>> >> > > > cache.addListener(...)
>> >> > > > QueryCursor cursor = cache.query(initialQuery);
>> >> > > >
>> >> > > >
>> >> > > Agree with this.
>> >> > >
>> >> >
>> >>
>> >>
>> >>
>> >> --
>> >> Nikolay Izhikov
>> >> NIzhikov.dev@gmail.com
>> >>
>> >
>> >
>>
>
>
>
> --
> Nikolay Izhikov
> NIzhikov.dev@gmail.com
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Vova,

> Public API is the face of our product.
> We cannot and do not want to change it in a rush.
> It is not a big problem if we spend additional week or month for API
design

Fully agreed.

I'm not trying to speed up changes, all I try is separate two discussions:

* ticket implementation based on existing API
* design of new API

> It is much better *than* extend*ing* confusing behavior *of *already confusing
and overengineered API

Ignite already have a ContinuousQuery
It's a matter of fact.
Ticket goal is provide some useful feature to the user.
I think it a good thing.

Can you list confusions that added by ticket implementation?


2017-09-12 19:47 GMT+03:00 Vladimir Ozerov <vo...@gridgain.com>:

> I meant "It is much better *than* extend*ing* confusing behavior *of
> *already
> confusing and overengineered API."
>
> On Tue, Sep 12, 2017 at 7:35 PM, Vladimir Ozerov <vo...@gridgain.com>
> wrote:
>
> > Nikolay,
> >
> > Public API is the face of our product. We cannot and do not want to
> change
> > it in a rush. This ticket was in a backlog for more than 2 years. It is
> not
> > a big problem if we spend additional week or month for API design. It is
> > much better to extend confusing behavior on already confusing and
> > overengineered API.
> >
> > On Tue, Sep 12, 2017 at 6:47 PM, Николай Ижиков <ni...@gmail.com>
> > wrote:
> >
> >> Vova
> >>
> >> > I propose to deprecate current continuous queries and develop new API.
> >> > This should not break anything.
> >>
> >> If the community agrees that *whole* continuous query API is bad - it
> OK.
> >> Let's develop new API.
> >>
> >> But developing new public API and implementing it is a very long
> process.
> >> One can see it based on this thread :)
> >>
> >> I think my implementation [1] of transformers for ContinuousQuery make
> >> things better for a user because remote transformers can lead to
> >> significant performance win.
> >>
> >> > Adding transformers on top of current API will make it total mess.
> >>
> >> I propose two things:
> >>
> >> 1. Continue discussion of current task [2] with scope limited by current
> >> API.
> >> 2. Start a discussion and work on new API. I think we can start with
> >> listing things that make current API bad. I can drive such a discussion.
> >>
> >> [1] https://github.com/apache/ignite/pull/2372
> >> [2] https://issues.apache.org/jira/browse/IGNITE-425
> >>
> >>
> >> 2017-09-12 17:55 GMT+03:00 Dmitriy Setrakyan <ds...@apache.org>:
> >>
> >> > Vladimir, are their factories for the proposed listeners?
> >> >
> >> > On Tue, Sep 12, 2017 at 7:52 AM, Alexey Goncharuk <
> >> > alexey.goncharuk@gmail.com> wrote:
> >> >
> >> > > Vladimir,
> >> > >
> >> > > Can you please clarify how the proposed API will work?
> >> > >
> >> > > My opinion is that our query API is big piece of ... you know,
> >> especially
> >> > > > ContinuousQuery. A lot of concepts and features are mixed in a
> >> single
> >> > > > entity, what makes it hard to understand and use. Let's finally
> >> > deprecate
> >> > > > ContinuousQuery and design nice and consistent API. E.g.:
> >> > > >
> >> > > > interface IgniteCache {
> >> > > >     UUID addListener(CacheEntryListener listener)
> >> > > >     void removeListener(UUID listenerId);
> >> > > > }
> >> > > >
> >> > > > This method set's a listener on all nodes which will process event
> >> > > locally,
> >> > > > no network communication.
> >> > >
> >> > >
> >> > > Do I understand correctly that CacheEntryListener will have a method
> >> like
> >> > > onEvent() which will accept the cache event?
> >> > >
> >> > >
> >> > > > Now if you want semantics similar to existing
> >> > > > continuous queries, you use special entry listener type:
> >> > > >
> >> > > > class ContinuousQueryCacheEntryListener implements
> >> CacheEntryListener
> >> > {
> >> > > >     ContinuousQueryRemoteFilter rmtFilter;
> >> > > >     ContinuousQueryRemoteTransformer rmtTransformer;
> >> > > >     ContinuousQueryLocalCallback locCb;
> >> > > > }
> >> > > >
> >> > > >
> >> > > This becomes confusing: while the ContinuousQueryCacheEntryListener
> >> > itself
> >> > > has the onEvent() method, which is supposed to be called on event
> >> nodes,
> >> > it
> >> > > also has a rmtFilter, which will also be called on event nodes. Will
> >> the
> >> > > onEvent() then invoked on the listener anyway, regardless of the
> >> filter
> >> > > result? Finally, the listener will have a local callback field,
> which
> >> > will
> >> > > be called on the originating node. This sounds way more tricky to me
> >> than
> >> > > the current API.
> >> > >
> >> > >
> >> > > > Last, "initial query" concept should be dropped from "continuous
> >> query"
> >> > > > feature completely. It doesn't guarantee any kind of atomicity or
> >> > > > visibility wrt to cache events, so it adds no value. The same
> >> behavior
> >> > > > could be achieved as follows:
> >> > > >
> >> > > > cache.addListener(...)
> >> > > > QueryCursor cursor = cache.query(initialQuery);
> >> > > >
> >> > > >
> >> > > Agree with this.
> >> > >
> >> >
> >>
> >>
> >>
> >> --
> >> Nikolay Izhikov
> >> NIzhikov.dev@gmail.com
> >>
> >
> >
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Vladimir Ozerov <vo...@gridgain.com>.
I meant "It is much better *than* extend*ing* confusing behavior *of *already
confusing and overengineered API."

On Tue, Sep 12, 2017 at 7:35 PM, Vladimir Ozerov <vo...@gridgain.com>
wrote:

> Nikolay,
>
> Public API is the face of our product. We cannot and do not want to change
> it in a rush. This ticket was in a backlog for more than 2 years. It is not
> a big problem if we spend additional week or month for API design. It is
> much better to extend confusing behavior on already confusing and
> overengineered API.
>
> On Tue, Sep 12, 2017 at 6:47 PM, Николай Ижиков <ni...@gmail.com>
> wrote:
>
>> Vova
>>
>> > I propose to deprecate current continuous queries and develop new API.
>> > This should not break anything.
>>
>> If the community agrees that *whole* continuous query API is bad - it OK.
>> Let's develop new API.
>>
>> But developing new public API and implementing it is a very long process.
>> One can see it based on this thread :)
>>
>> I think my implementation [1] of transformers for ContinuousQuery make
>> things better for a user because remote transformers can lead to
>> significant performance win.
>>
>> > Adding transformers on top of current API will make it total mess.
>>
>> I propose two things:
>>
>> 1. Continue discussion of current task [2] with scope limited by current
>> API.
>> 2. Start a discussion and work on new API. I think we can start with
>> listing things that make current API bad. I can drive such a discussion.
>>
>> [1] https://github.com/apache/ignite/pull/2372
>> [2] https://issues.apache.org/jira/browse/IGNITE-425
>>
>>
>> 2017-09-12 17:55 GMT+03:00 Dmitriy Setrakyan <ds...@apache.org>:
>>
>> > Vladimir, are their factories for the proposed listeners?
>> >
>> > On Tue, Sep 12, 2017 at 7:52 AM, Alexey Goncharuk <
>> > alexey.goncharuk@gmail.com> wrote:
>> >
>> > > Vladimir,
>> > >
>> > > Can you please clarify how the proposed API will work?
>> > >
>> > > My opinion is that our query API is big piece of ... you know,
>> especially
>> > > > ContinuousQuery. A lot of concepts and features are mixed in a
>> single
>> > > > entity, what makes it hard to understand and use. Let's finally
>> > deprecate
>> > > > ContinuousQuery and design nice and consistent API. E.g.:
>> > > >
>> > > > interface IgniteCache {
>> > > >     UUID addListener(CacheEntryListener listener)
>> > > >     void removeListener(UUID listenerId);
>> > > > }
>> > > >
>> > > > This method set's a listener on all nodes which will process event
>> > > locally,
>> > > > no network communication.
>> > >
>> > >
>> > > Do I understand correctly that CacheEntryListener will have a method
>> like
>> > > onEvent() which will accept the cache event?
>> > >
>> > >
>> > > > Now if you want semantics similar to existing
>> > > > continuous queries, you use special entry listener type:
>> > > >
>> > > > class ContinuousQueryCacheEntryListener implements
>> CacheEntryListener
>> > {
>> > > >     ContinuousQueryRemoteFilter rmtFilter;
>> > > >     ContinuousQueryRemoteTransformer rmtTransformer;
>> > > >     ContinuousQueryLocalCallback locCb;
>> > > > }
>> > > >
>> > > >
>> > > This becomes confusing: while the ContinuousQueryCacheEntryListener
>> > itself
>> > > has the onEvent() method, which is supposed to be called on event
>> nodes,
>> > it
>> > > also has a rmtFilter, which will also be called on event nodes. Will
>> the
>> > > onEvent() then invoked on the listener anyway, regardless of the
>> filter
>> > > result? Finally, the listener will have a local callback field, which
>> > will
>> > > be called on the originating node. This sounds way more tricky to me
>> than
>> > > the current API.
>> > >
>> > >
>> > > > Last, "initial query" concept should be dropped from "continuous
>> query"
>> > > > feature completely. It doesn't guarantee any kind of atomicity or
>> > > > visibility wrt to cache events, so it adds no value. The same
>> behavior
>> > > > could be achieved as follows:
>> > > >
>> > > > cache.addListener(...)
>> > > > QueryCursor cursor = cache.query(initialQuery);
>> > > >
>> > > >
>> > > Agree with this.
>> > >
>> >
>>
>>
>>
>> --
>> Nikolay Izhikov
>> NIzhikov.dev@gmail.com
>>
>
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Vladimir Ozerov <vo...@gridgain.com>.
Nikolay,

Public API is the face of our product. We cannot and do not want to change
it in a rush. This ticket was in a backlog for more than 2 years. It is not
a big problem if we spend additional week or month for API design. It is
much better to extend confusing behavior on already confusing and
overengineered API.

On Tue, Sep 12, 2017 at 6:47 PM, Николай Ижиков <ni...@gmail.com>
wrote:

> Vova
>
> > I propose to deprecate current continuous queries and develop new API.
> > This should not break anything.
>
> If the community agrees that *whole* continuous query API is bad - it OK.
> Let's develop new API.
>
> But developing new public API and implementing it is a very long process.
> One can see it based on this thread :)
>
> I think my implementation [1] of transformers for ContinuousQuery make
> things better for a user because remote transformers can lead to
> significant performance win.
>
> > Adding transformers on top of current API will make it total mess.
>
> I propose two things:
>
> 1. Continue discussion of current task [2] with scope limited by current
> API.
> 2. Start a discussion and work on new API. I think we can start with
> listing things that make current API bad. I can drive such a discussion.
>
> [1] https://github.com/apache/ignite/pull/2372
> [2] https://issues.apache.org/jira/browse/IGNITE-425
>
>
> 2017-09-12 17:55 GMT+03:00 Dmitriy Setrakyan <ds...@apache.org>:
>
> > Vladimir, are their factories for the proposed listeners?
> >
> > On Tue, Sep 12, 2017 at 7:52 AM, Alexey Goncharuk <
> > alexey.goncharuk@gmail.com> wrote:
> >
> > > Vladimir,
> > >
> > > Can you please clarify how the proposed API will work?
> > >
> > > My opinion is that our query API is big piece of ... you know,
> especially
> > > > ContinuousQuery. A lot of concepts and features are mixed in a single
> > > > entity, what makes it hard to understand and use. Let's finally
> > deprecate
> > > > ContinuousQuery and design nice and consistent API. E.g.:
> > > >
> > > > interface IgniteCache {
> > > >     UUID addListener(CacheEntryListener listener)
> > > >     void removeListener(UUID listenerId);
> > > > }
> > > >
> > > > This method set's a listener on all nodes which will process event
> > > locally,
> > > > no network communication.
> > >
> > >
> > > Do I understand correctly that CacheEntryListener will have a method
> like
> > > onEvent() which will accept the cache event?
> > >
> > >
> > > > Now if you want semantics similar to existing
> > > > continuous queries, you use special entry listener type:
> > > >
> > > > class ContinuousQueryCacheEntryListener implements
> CacheEntryListener
> > {
> > > >     ContinuousQueryRemoteFilter rmtFilter;
> > > >     ContinuousQueryRemoteTransformer rmtTransformer;
> > > >     ContinuousQueryLocalCallback locCb;
> > > > }
> > > >
> > > >
> > > This becomes confusing: while the ContinuousQueryCacheEntryListener
> > itself
> > > has the onEvent() method, which is supposed to be called on event
> nodes,
> > it
> > > also has a rmtFilter, which will also be called on event nodes. Will
> the
> > > onEvent() then invoked on the listener anyway, regardless of the filter
> > > result? Finally, the listener will have a local callback field, which
> > will
> > > be called on the originating node. This sounds way more tricky to me
> than
> > > the current API.
> > >
> > >
> > > > Last, "initial query" concept should be dropped from "continuous
> query"
> > > > feature completely. It doesn't guarantee any kind of atomicity or
> > > > visibility wrt to cache events, so it adds no value. The same
> behavior
> > > > could be achieved as follows:
> > > >
> > > > cache.addListener(...)
> > > > QueryCursor cursor = cache.query(initialQuery);
> > > >
> > > >
> > > Agree with this.
> > >
> >
>
>
>
> --
> Nikolay Izhikov
> NIzhikov.dev@gmail.com
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Vova

> I propose to deprecate current continuous queries and develop new API.
> This should not break anything.

If the community agrees that *whole* continuous query API is bad - it OK.
Let's develop new API.

But developing new public API and implementing it is a very long process.
One can see it based on this thread :)

I think my implementation [1] of transformers for ContinuousQuery make
things better for a user because remote transformers can lead to
significant performance win.

> Adding transformers on top of current API will make it total mess.

I propose two things:

1. Continue discussion of current task [2] with scope limited by current
API.
2. Start a discussion and work on new API. I think we can start with
listing things that make current API bad. I can drive such a discussion.

[1] https://github.com/apache/ignite/pull/2372
[2] https://issues.apache.org/jira/browse/IGNITE-425


2017-09-12 17:55 GMT+03:00 Dmitriy Setrakyan <ds...@apache.org>:

> Vladimir, are their factories for the proposed listeners?
>
> On Tue, Sep 12, 2017 at 7:52 AM, Alexey Goncharuk <
> alexey.goncharuk@gmail.com> wrote:
>
> > Vladimir,
> >
> > Can you please clarify how the proposed API will work?
> >
> > My opinion is that our query API is big piece of ... you know, especially
> > > ContinuousQuery. A lot of concepts and features are mixed in a single
> > > entity, what makes it hard to understand and use. Let's finally
> deprecate
> > > ContinuousQuery and design nice and consistent API. E.g.:
> > >
> > > interface IgniteCache {
> > >     UUID addListener(CacheEntryListener listener)
> > >     void removeListener(UUID listenerId);
> > > }
> > >
> > > This method set's a listener on all nodes which will process event
> > locally,
> > > no network communication.
> >
> >
> > Do I understand correctly that CacheEntryListener will have a method like
> > onEvent() which will accept the cache event?
> >
> >
> > > Now if you want semantics similar to existing
> > > continuous queries, you use special entry listener type:
> > >
> > > class ContinuousQueryCacheEntryListener implements CacheEntryListener
> {
> > >     ContinuousQueryRemoteFilter rmtFilter;
> > >     ContinuousQueryRemoteTransformer rmtTransformer;
> > >     ContinuousQueryLocalCallback locCb;
> > > }
> > >
> > >
> > This becomes confusing: while the ContinuousQueryCacheEntryListener
> itself
> > has the onEvent() method, which is supposed to be called on event nodes,
> it
> > also has a rmtFilter, which will also be called on event nodes. Will the
> > onEvent() then invoked on the listener anyway, regardless of the filter
> > result? Finally, the listener will have a local callback field, which
> will
> > be called on the originating node. This sounds way more tricky to me than
> > the current API.
> >
> >
> > > Last, "initial query" concept should be dropped from "continuous query"
> > > feature completely. It doesn't guarantee any kind of atomicity or
> > > visibility wrt to cache events, so it adds no value. The same behavior
> > > could be achieved as follows:
> > >
> > > cache.addListener(...)
> > > QueryCursor cursor = cache.query(initialQuery);
> > >
> > >
> > Agree with this.
> >
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Vladimir Ozerov <vo...@gridgain.com>.
Dima,

We definitely can have factories if we want.

On Tue, Sep 12, 2017 at 5:55 PM, Dmitriy Setrakyan <ds...@apache.org>
wrote:

> Vladimir, are their factories for the proposed listeners?
>
> On Tue, Sep 12, 2017 at 7:52 AM, Alexey Goncharuk <
> alexey.goncharuk@gmail.com> wrote:
>
> > Vladimir,
> >
> > Can you please clarify how the proposed API will work?
> >
> > My opinion is that our query API is big piece of ... you know, especially
> > > ContinuousQuery. A lot of concepts and features are mixed in a single
> > > entity, what makes it hard to understand and use. Let's finally
> deprecate
> > > ContinuousQuery and design nice and consistent API. E.g.:
> > >
> > > interface IgniteCache {
> > >     UUID addListener(CacheEntryListener listener)
> > >     void removeListener(UUID listenerId);
> > > }
> > >
> > > This method set's a listener on all nodes which will process event
> > locally,
> > > no network communication.
> >
> >
> > Do I understand correctly that CacheEntryListener will have a method like
> > onEvent() which will accept the cache event?
> >
> >
> > > Now if you want semantics similar to existing
> > > continuous queries, you use special entry listener type:
> > >
> > > class ContinuousQueryCacheEntryListener implements CacheEntryListener
> {
> > >     ContinuousQueryRemoteFilter rmtFilter;
> > >     ContinuousQueryRemoteTransformer rmtTransformer;
> > >     ContinuousQueryLocalCallback locCb;
> > > }
> > >
> > >
> > This becomes confusing: while the ContinuousQueryCacheEntryListener
> itself
> > has the onEvent() method, which is supposed to be called on event nodes,
> it
> > also has a rmtFilter, which will also be called on event nodes. Will the
> > onEvent() then invoked on the listener anyway, regardless of the filter
> > result? Finally, the listener will have a local callback field, which
> will
> > be called on the originating node. This sounds way more tricky to me than
> > the current API.
> >
> >
> > > Last, "initial query" concept should be dropped from "continuous query"
> > > feature completely. It doesn't guarantee any kind of atomicity or
> > > visibility wrt to cache events, so it adds no value. The same behavior
> > > could be achieved as follows:
> > >
> > > cache.addListener(...)
> > > QueryCursor cursor = cache.query(initialQuery);
> > >
> > >
> > Agree with this.
> >
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Dmitriy Setrakyan <ds...@apache.org>.
Vladimir, are their factories for the proposed listeners?

On Tue, Sep 12, 2017 at 7:52 AM, Alexey Goncharuk <
alexey.goncharuk@gmail.com> wrote:

> Vladimir,
>
> Can you please clarify how the proposed API will work?
>
> My opinion is that our query API is big piece of ... you know, especially
> > ContinuousQuery. A lot of concepts and features are mixed in a single
> > entity, what makes it hard to understand and use. Let's finally deprecate
> > ContinuousQuery and design nice and consistent API. E.g.:
> >
> > interface IgniteCache {
> >     UUID addListener(CacheEntryListener listener)
> >     void removeListener(UUID listenerId);
> > }
> >
> > This method set's a listener on all nodes which will process event
> locally,
> > no network communication.
>
>
> Do I understand correctly that CacheEntryListener will have a method like
> onEvent() which will accept the cache event?
>
>
> > Now if you want semantics similar to existing
> > continuous queries, you use special entry listener type:
> >
> > class ContinuousQueryCacheEntryListener implements CacheEntryListener {
> >     ContinuousQueryRemoteFilter rmtFilter;
> >     ContinuousQueryRemoteTransformer rmtTransformer;
> >     ContinuousQueryLocalCallback locCb;
> > }
> >
> >
> This becomes confusing: while the ContinuousQueryCacheEntryListener itself
> has the onEvent() method, which is supposed to be called on event nodes, it
> also has a rmtFilter, which will also be called on event nodes. Will the
> onEvent() then invoked on the listener anyway, regardless of the filter
> result? Finally, the listener will have a local callback field, which will
> be called on the originating node. This sounds way more tricky to me than
> the current API.
>
>
> > Last, "initial query" concept should be dropped from "continuous query"
> > feature completely. It doesn't guarantee any kind of atomicity or
> > visibility wrt to cache events, so it adds no value. The same behavior
> > could be achieved as follows:
> >
> > cache.addListener(...)
> > QueryCursor cursor = cache.query(initialQuery);
> >
> >
> Agree with this.
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Vladimir Ozerov <vo...@gridgain.com>.
Alex,

The source of our confusion in current API is that we called "filter" what
actually should be "listener". I propose to set has the simplest API
possible. Note that CacheEntryListener is a part of jcache API. Call of
this method will set passed listener on remote nodes. Nothing more.

UUID listen(CacheEntryListener listener)

Next, I propose to treat our current continuous queries implementation as a
specific implementation of CacheEntryListener interface. This
implementation will filter/transform events remotely, and then transfer
them to some "local" callback. This is it.


On Tue, Sep 12, 2017 at 5:52 PM, Alexey Goncharuk <
alexey.goncharuk@gmail.com> wrote:

> Vladimir,
>
> Can you please clarify how the proposed API will work?
>
> My opinion is that our query API is big piece of ... you know, especially
> > ContinuousQuery. A lot of concepts and features are mixed in a single
> > entity, what makes it hard to understand and use. Let's finally deprecate
> > ContinuousQuery and design nice and consistent API. E.g.:
> >
> > interface IgniteCache {
> >     UUID addListener(CacheEntryListener listener)
> >     void removeListener(UUID listenerId);
> > }
> >
> > This method set's a listener on all nodes which will process event
> locally,
> > no network communication.
>
>
> Do I understand correctly that CacheEntryListener will have a method like
> onEvent() which will accept the cache event?
>
>
> > Now if you want semantics similar to existing
> > continuous queries, you use special entry listener type:
> >
> > class ContinuousQueryCacheEntryListener implements CacheEntryListener {
> >     ContinuousQueryRemoteFilter rmtFilter;
> >     ContinuousQueryRemoteTransformer rmtTransformer;
> >     ContinuousQueryLocalCallback locCb;
> > }
> >
> >
> This becomes confusing: while the ContinuousQueryCacheEntryListener itself
> has the onEvent() method, which is supposed to be called on event nodes, it
> also has a rmtFilter, which will also be called on event nodes. Will the
> onEvent() then invoked on the listener anyway, regardless of the filter
> result? Finally, the listener will have a local callback field, which will
> be called on the originating node. This sounds way more tricky to me than
> the current API.
>
>
> > Last, "initial query" concept should be dropped from "continuous query"
> > feature completely. It doesn't guarantee any kind of atomicity or
> > visibility wrt to cache events, so it adds no value. The same behavior
> > could be achieved as follows:
> >
> > cache.addListener(...)
> > QueryCursor cursor = cache.query(initialQuery);
> >
> >
> Agree with this.
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Alexey Goncharuk <al...@gmail.com>.
Vladimir,

Can you please clarify how the proposed API will work?

My opinion is that our query API is big piece of ... you know, especially
> ContinuousQuery. A lot of concepts and features are mixed in a single
> entity, what makes it hard to understand and use. Let's finally deprecate
> ContinuousQuery and design nice and consistent API. E.g.:
>
> interface IgniteCache {
>     UUID addListener(CacheEntryListener listener)
>     void removeListener(UUID listenerId);
> }
>
> This method set's a listener on all nodes which will process event locally,
> no network communication.


Do I understand correctly that CacheEntryListener will have a method like
onEvent() which will accept the cache event?


> Now if you want semantics similar to existing
> continuous queries, you use special entry listener type:
>
> class ContinuousQueryCacheEntryListener implements CacheEntryListener {
>     ContinuousQueryRemoteFilter rmtFilter;
>     ContinuousQueryRemoteTransformer rmtTransformer;
>     ContinuousQueryLocalCallback locCb;
> }
>
>
This becomes confusing: while the ContinuousQueryCacheEntryListener itself
has the onEvent() method, which is supposed to be called on event nodes, it
also has a rmtFilter, which will also be called on event nodes. Will the
onEvent() then invoked on the listener anyway, regardless of the filter
result? Finally, the listener will have a local callback field, which will
be called on the originating node. This sounds way more tricky to me than
the current API.


> Last, "initial query" concept should be dropped from "continuous query"
> feature completely. It doesn't guarantee any kind of atomicity or
> visibility wrt to cache events, so it adds no value. The same behavior
> could be achieved as follows:
>
> cache.addListener(...)
> QueryCursor cursor = cache.query(initialQuery);
>
>
Agree with this.

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Yakov Zhdanov <yz...@apache.org>.
Agree with Vladimir. However, this will break API compatibility. So, at
this point this is impossible.

--Yakov

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Hello, Vladimir.


class ContinuousQueryCacheEntryListener implements CacheEntryListener {
    ContinuousQueryRemoteFilter rmtFilter;
    ContinuousQueryRemoteTransformer rmtTransformer;
    ContinuousQueryLocalCallback locCb;
}

I think class you proposed should be separated in two because of generic(as
I do in my pull request with existing ContinuousQuery).

If we setup ContinuousQuery *without* transformer we got CacheEntryEvent<K,
V> in listener.

If we setup ContinuousQuery *with* transformer we got T in listener -
object of type T produced by transformer.


2017-09-12 16:04 GMT+03:00 Vladimir Ozerov <vo...@gridgain.com>:

> My opinion is that our query API is big piece of ... you know, especially
> ContinuousQuery. A lot of concepts and features are mixed in a single
> entity, what makes it hard to understand and use. Let's finally deprecate
> ContinuousQuery and design nice and consistent API. E.g.:
>
> interface IgniteCache {
>     UUID addListener(CacheEntryListener listener)
>     void removeListener(UUID listenerId);
> }
>
> This method set's a listener on all nodes which will process event locally,
> no network communication. Now if you want semantics similar to existing
> continuous queries, you use special entry listener type:
>
> class ContinuousQueryCacheEntryListener implements CacheEntryListener {
>     ContinuousQueryRemoteFilter rmtFilter;
>     ContinuousQueryRemoteTransformer rmtTransformer;
>     ContinuousQueryLocalCallback locCb;
> }
>
> Last, "initial query" concept should be dropped from "continuous query"
> feature completely. It doesn't guarantee any kind of atomicity or
> visibility wrt to cache events, so it adds no value. The same behavior
> could be achieved as follows:
>
> cache.addListener(...)
> QueryCursor cursor = cache.query(initialQuery);
>
> Vladimir.
>
>
> On Tue, Sep 12, 2017 at 3:35 PM, Yakov Zhdanov <yz...@apache.org>
> wrote:
>
> > Dmitry, can you please take a look at public API change.
> >
> > Ticket - https://issues.apache.org/jira/browse/IGNITE-425
> > PR - https://github.com/apache/ignite/pull/2372
> >
> > Issues:
> > 1. Do you see any other option other than creating separate class? As for
> > me I don't.
> > 2. In a new class we still have initial query which uses <K, V> types
> which
> > is questionable.
> >
> > Igniters, please share your thoughts as well. Public API is the face of
> our
> > product we need to make it as convenient and consistent as we can.
> >
> > --Yakov
> >
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Vladimir Ozerov <vo...@gridgain.com>.
My opinion is that our query API is big piece of ... you know, especially
ContinuousQuery. A lot of concepts and features are mixed in a single
entity, what makes it hard to understand and use. Let's finally deprecate
ContinuousQuery and design nice and consistent API. E.g.:

interface IgniteCache {
    UUID addListener(CacheEntryListener listener)
    void removeListener(UUID listenerId);
}

This method set's a listener on all nodes which will process event locally,
no network communication. Now if you want semantics similar to existing
continuous queries, you use special entry listener type:

class ContinuousQueryCacheEntryListener implements CacheEntryListener {
    ContinuousQueryRemoteFilter rmtFilter;
    ContinuousQueryRemoteTransformer rmtTransformer;
    ContinuousQueryLocalCallback locCb;
}

Last, "initial query" concept should be dropped from "continuous query"
feature completely. It doesn't guarantee any kind of atomicity or
visibility wrt to cache events, so it adds no value. The same behavior
could be achieved as follows:

cache.addListener(...)
QueryCursor cursor = cache.query(initialQuery);

Vladimir.


On Tue, Sep 12, 2017 at 3:35 PM, Yakov Zhdanov <yz...@apache.org> wrote:

> Dmitry, can you please take a look at public API change.
>
> Ticket - https://issues.apache.org/jira/browse/IGNITE-425
> PR - https://github.com/apache/ignite/pull/2372
>
> Issues:
> 1. Do you see any other option other than creating separate class? As for
> me I don't.
> 2. In a new class we still have initial query which uses <K, V> types which
> is questionable.
>
> Igniters, please share your thoughts as well. Public API is the face of our
> product we need to make it as convenient and consistent as we can.
>
> --Yakov
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Vladimir Ozerov <vo...@gridgain.com>.
Yakov,

I propose to deprecate current continuous queries and develop new API. This
should not break anything. Adding transformers on top of current API will
make it total mess.

On Tue, Sep 12, 2017 at 5:39 PM, Николай Ижиков <ni...@gmail.com>
wrote:

> Yakov.
>
> > 2. In a new class we still have initial query which uses <K, V> types
> which
> is questionable.
>
> After some discussion with a colleague, I think I got your point.
>
> If I understand your question right:
>
> Why does someone want to execute an initial query that returns <K, V>?
> If one use ContinuousQueryWithTransformer that transform <K, V> to <T>.
>
> I think you right and we have to transform initialQuery results as well.
>
> However, at the moment remote transformer supported only for a ScanQuery
> [1]: "Currently transformers are supported ONLY for ScanQuery
> <https://ignite.apache.org/releases/latest/javadoc/org/
> apache/ignite/cache/query/ScanQuery.html>
> ."
>
> So I propose next:
>
> 1. Change ContinuousQueryWithTransformer to support only ScanQuery as
> initial query.
> If in future we will want to support all kinds of Query we can change
> from setInitialQuery(ScanQuery) to setInitialQuery(Query) without breaking
> API.
> 2. Run initial scan query with the same transformer as ContinuousQuery.
>
> Thoughts?
>
> [1] https://ignite.apache.org/releases/latest/javadoc/org/
> apache/ignite/IgniteCache.html#query(org.apache.ignite.
> cache.query.Query,%20org.apache.ignite.lang.IgniteClosure)
>
>
> 2017-09-12 15:35 GMT+03:00 Yakov Zhdanov <yz...@apache.org>:
>
> > Dmitry, can you please take a look at public API change.
> >
> > Ticket - https://issues.apache.org/jira/browse/IGNITE-425
> > PR - https://github.com/apache/ignite/pull/2372
> >
> > Issues:
> > 1. Do you see any other option other than creating separate class? As for
> > me I don't.
> > 2. In a new class we still have initial query which uses <K, V> types
> which
> > is questionable.
> >
> > Igniters, please share your thoughts as well. Public API is the face of
> our
> > product we need to make it as convenient and consistent as we can.
> >
> > --Yakov
> >
>
>
>
> --
> Nikolay Izhikov
> NIzhikov.dev@gmail.com
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Yakov.

> 2. In a new class we still have initial query which uses <K, V> types
which
is questionable.

After some discussion with a colleague, I think I got your point.

If I understand your question right:

Why does someone want to execute an initial query that returns <K, V>?
If one use ContinuousQueryWithTransformer that transform <K, V> to <T>.

I think you right and we have to transform initialQuery results as well.

However, at the moment remote transformer supported only for a ScanQuery
[1]: "Currently transformers are supported ONLY for ScanQuery
<https://ignite.apache.org/releases/latest/javadoc/org/apache/ignite/cache/query/ScanQuery.html>
."

So I propose next:

1. Change ContinuousQueryWithTransformer to support only ScanQuery as
initial query.
If in future we will want to support all kinds of Query we can change
from setInitialQuery(ScanQuery) to setInitialQuery(Query) without breaking
API.
2. Run initial scan query with the same transformer as ContinuousQuery.

Thoughts?

[1] https://ignite.apache.org/releases/latest/javadoc/org/
apache/ignite/IgniteCache.html#query(org.apache.ignite.
cache.query.Query,%20org.apache.ignite.lang.IgniteClosure)


2017-09-12 15:35 GMT+03:00 Yakov Zhdanov <yz...@apache.org>:

> Dmitry, can you please take a look at public API change.
>
> Ticket - https://issues.apache.org/jira/browse/IGNITE-425
> PR - https://github.com/apache/ignite/pull/2372
>
> Issues:
> 1. Do you see any other option other than creating separate class? As for
> me I don't.
> 2. In a new class we still have initial query which uses <K, V> types which
> is questionable.
>
> Igniters, please share your thoughts as well. Public API is the face of our
> product we need to make it as convenient and consistent as we can.
>
> --Yakov
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Yakov Zhdanov <yz...@apache.org>.
Dmitry, can you please take a look at public API change.

Ticket - https://issues.apache.org/jira/browse/IGNITE-425
PR - https://github.com/apache/ignite/pull/2372

Issues:
1. Do you see any other option other than creating separate class? As for
me I don't.
2. In a new class we still have initial query which uses <K, V> types which
is questionable.

Igniters, please share your thoughts as well. Public API is the face of our
product we need to make it as convenient and consistent as we can.

--Yakov

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Hello, Nikolay.

I think we can't use this method:

We need to set:
* transformer
* *local listener* - differs from regular ContinuousQuery listener
* other continuous query params.

So at least we have to add local listener param.

If we use ContinuousQuery instance as first param to get all
ContinuousQuery params: timeout, pageSize, filter, etc... then we have to
ignore local listener inside it.

What do you think?


2017-09-05 15:48 GMT+03:00 Nikolai Tikhonov <nt...@apache.org>:

> Guys,
>
> This API looks very complicated. We already have similar functionality for
> ScanQuery, let's reuse it. Please, have a look
> at IgniteCache#query(org.apache.ignite.cache.query.Query<T>,
> org.apache.ignite.lang.IgniteClosure<T,R>) method. It looks more pretty
> for
> me. Any objections?
>
> Thanks,
> Nikolay
>
> On Mon, Sep 4, 2017 at 3:22 PM, Николай Ижиков <ni...@gmail.com>
> wrote:
>
> > Yakov, Anton, thank you!
> >
> > 2017-09-04 15:16 GMT+03:00 Yakov Zhdanov <yz...@apache.org>:
> >
> > > Well, let's leave it as is for now, since I don't see any easy way to
> > > consistently solve this.
> > >
> > > --Yakov
> > >
> >
> >
> >
> > --
> > Nikolay Izhikov
> > NIzhikov.dev@gmail.com
> >
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Nikolai Tikhonov <nt...@apache.org>.
Guys,

This API looks very complicated. We already have similar functionality for
ScanQuery, let's reuse it. Please, have a look
at IgniteCache#query(org.apache.ignite.cache.query.Query<T>,
org.apache.ignite.lang.IgniteClosure<T,R>) method. It looks more pretty for
me. Any objections?

Thanks,
Nikolay

On Mon, Sep 4, 2017 at 3:22 PM, Николай Ижиков <ni...@gmail.com>
wrote:

> Yakov, Anton, thank you!
>
> 2017-09-04 15:16 GMT+03:00 Yakov Zhdanov <yz...@apache.org>:
>
> > Well, let's leave it as is for now, since I don't see any easy way to
> > consistently solve this.
> >
> > --Yakov
> >
>
>
>
> --
> Nikolay Izhikov
> NIzhikov.dev@gmail.com
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Yakov, Anton, thank you!

2017-09-04 15:16 GMT+03:00 Yakov Zhdanov <yz...@apache.org>:

> Well, let's leave it as is for now, since I don't see any easy way to
> consistently solve this.
>
> --Yakov
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Yakov Zhdanov <yz...@apache.org>.
Well, let's leave it as is for now, since I don't see any easy way to
consistently solve this.

--Yakov

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Anton Vinogradov <av...@apache.org>.
+1 to keep current behavior and start new thread to solve notification
issue.

On Mon, Sep 4, 2017 at 2:05 PM, Николай Ижиков <ni...@gmail.com>
wrote:

> Hello, Yakov.
>
> I made a bit of investigation about your proposal of handling filter and
> transformer exceptions:
>
> 1. If we cancel continuous query from remote node user can't know it.
>     There is no public API to check "Is continuous query still alive?".
>     The only consequence of canceling query - listener not called on a
> local node.
>
> 2. If we change a behavior of filter exception handling then we broke
> backward compatibility. Is it OK?
>
> 3. If we implement query cancel only for transformer exception - behavior
> would be different for filter and transformer.
>
> I think changes for consistent exception handling requires additional
> discussion.
> I will start such discussion in another thread but seems that it not
> related to current issue as it also touches current ContinuousQuery
> implementation.
>
> Can we stay with current behavior for current task(IGNITE-425)?
>
> * filter exception treats as true
> * transformer exception treats as null
>
>
> 2017-08-30 17:16 GMT+03:00 Yakov Zhdanov <yz...@gridgain.com>:
>
> > I would postpone review until we come to a clear decision on what should
> be
> > done if filter or transformer fails. I don't think cancelling query is
> too
> > much. From my standpoint this is a kind of heuristic exception and may
> > break some sensitive logic.
> >
> > Thanks!
> > --
> > Yakov Zhdanov, Director R&D
> > *GridGain Systems*
> > www.gridgain.com
> >
> > 2017-08-30 16:24 GMT+03:00 Nikolay Izhikov <ni...@gmail.com>:
> >
> > > Hello, Yakov.
> > >
> > > The new class is OK - got it. Thanks!
> > >
> > > > Should we extract a super class?
> > >
> > > Yes, we should.
> > > I already have done it.
> > >
> > > See my last commit in PR - https://github.com/apache/igni
> > > te/pull/2372/commits/af1ed2e4dbef4ba5999f8566198cb75ad922f93b
> > >
> > > > We can put hard requirement that filter and transformer cannot throw
> > > > exception (same as cache interceptor).
> > >
> > > I think to cancel the whole query on transformer exception is too much.
> > > After discussion, I like the idea to skip event if transformer throws
> > > exception. As far as it "like regular filter" behavior.
> > >
> > > Thoughts?
> > >
> > >
> > > 30.08.2017 16:03, Yakov Zhdanov пишет:
> > >
> > > I think I have already agreed on a separate class since it seems to be
> > the
> > >> only option due to generics issue. Should we extract a super class?
> > >>
> > >> We can put hard requirement that filter and transformer cannot throw
> > >> exception (same as cache interceptor). If exception is thrown then we
> > >> cancel the query globally and unregister all the listeners. This may
> > sound
> > >> too much but inconsistencies brought by listener notifications may be
> > >> terrible for app.
> > >>
> > >> --Yakov
> > >>
> > >>
> >
>
>
>
> --
> Nikolay Izhikov
> NIzhikov.dev@gmail.com
>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Николай Ижиков <ni...@gmail.com>.
Hello, Yakov.

I made a bit of investigation about your proposal of handling filter and
transformer exceptions:

1. If we cancel continuous query from remote node user can't know it.
    There is no public API to check "Is continuous query still alive?".
    The only consequence of canceling query - listener not called on a
local node.

2. If we change a behavior of filter exception handling then we broke
backward compatibility. Is it OK?

3. If we implement query cancel only for transformer exception - behavior
would be different for filter and transformer.

I think changes for consistent exception handling requires additional
discussion.
I will start such discussion in another thread but seems that it not
related to current issue as it also touches current ContinuousQuery
implementation.

Can we stay with current behavior for current task(IGNITE-425)?

* filter exception treats as true
* transformer exception treats as null


2017-08-30 17:16 GMT+03:00 Yakov Zhdanov <yz...@gridgain.com>:

> I would postpone review until we come to a clear decision on what should be
> done if filter or transformer fails. I don't think cancelling query is too
> much. From my standpoint this is a kind of heuristic exception and may
> break some sensitive logic.
>
> Thanks!
> --
> Yakov Zhdanov, Director R&D
> *GridGain Systems*
> www.gridgain.com
>
> 2017-08-30 16:24 GMT+03:00 Nikolay Izhikov <ni...@gmail.com>:
>
> > Hello, Yakov.
> >
> > The new class is OK - got it. Thanks!
> >
> > > Should we extract a super class?
> >
> > Yes, we should.
> > I already have done it.
> >
> > See my last commit in PR - https://github.com/apache/igni
> > te/pull/2372/commits/af1ed2e4dbef4ba5999f8566198cb75ad922f93b
> >
> > > We can put hard requirement that filter and transformer cannot throw
> > > exception (same as cache interceptor).
> >
> > I think to cancel the whole query on transformer exception is too much.
> > After discussion, I like the idea to skip event if transformer throws
> > exception. As far as it "like regular filter" behavior.
> >
> > Thoughts?
> >
> >
> > 30.08.2017 16:03, Yakov Zhdanov пишет:
> >
> > I think I have already agreed on a separate class since it seems to be
> the
> >> only option due to generics issue. Should we extract a super class?
> >>
> >> We can put hard requirement that filter and transformer cannot throw
> >> exception (same as cache interceptor). If exception is thrown then we
> >> cancel the query globally and unregister all the listeners. This may
> sound
> >> too much but inconsistencies brought by listener notifications may be
> >> terrible for app.
> >>
> >> --Yakov
> >>
> >>
>



-- 
Nikolay Izhikov
NIzhikov.dev@gmail.com

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Yakov Zhdanov <yz...@gridgain.com>.
I would postpone review until we come to a clear decision on what should be
done if filter or transformer fails. I don't think cancelling query is too
much. From my standpoint this is a kind of heuristic exception and may
break some sensitive logic.

Thanks!
--
Yakov Zhdanov, Director R&D
*GridGain Systems*
www.gridgain.com

2017-08-30 16:24 GMT+03:00 Nikolay Izhikov <ni...@gmail.com>:

> Hello, Yakov.
>
> The new class is OK - got it. Thanks!
>
> > Should we extract a super class?
>
> Yes, we should.
> I already have done it.
>
> See my last commit in PR - https://github.com/apache/igni
> te/pull/2372/commits/af1ed2e4dbef4ba5999f8566198cb75ad922f93b
>
> > We can put hard requirement that filter and transformer cannot throw
> > exception (same as cache interceptor).
>
> I think to cancel the whole query on transformer exception is too much.
> After discussion, I like the idea to skip event if transformer throws
> exception. As far as it "like regular filter" behavior.
>
> Thoughts?
>
>
> 30.08.2017 16:03, Yakov Zhdanov пишет:
>
> I think I have already agreed on a separate class since it seems to be the
>> only option due to generics issue. Should we extract a super class?
>>
>> We can put hard requirement that filter and transformer cannot throw
>> exception (same as cache interceptor). If exception is thrown then we
>> cancel the query globally and unregister all the listeners. This may sound
>> too much but inconsistencies brought by listener notifications may be
>> terrible for app.
>>
>> --Yakov
>>
>>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Anton Vinogradov <av...@gridgain.com>.
Nikolay,

You can also use:

1) Global Exception Registry
ctx.kernalContext().exceptionRegistry().onException(shortMsg, ex);

2) Statistic
final boolean statsEnabled = cctx.config().isStatisticsEnabled();
if (statsEnabled)
   metrics.incrementOnFails();

P.s. Please make sure that code still valid, I found this at 1.6.

On Wed, Aug 30, 2017 at 4:24 PM, Nikolay Izhikov <ni...@gmail.com>
wrote:

> Hello, Yakov.
>
> The new class is OK - got it. Thanks!
>
> > Should we extract a super class?
>
> Yes, we should.
> I already have done it.
>
> See my last commit in PR - https://github.com/apache/igni
> te/pull/2372/commits/af1ed2e4dbef4ba5999f8566198cb75ad922f93b
>
> > We can put hard requirement that filter and transformer cannot throw
> > exception (same as cache interceptor).
>
> I think to cancel the whole query on transformer exception is too much.
> After discussion, I like the idea to skip event if transformer throws
> exception. As far as it "like regular filter" behavior.
>
> Thoughts?
>
>
> 30.08.2017 16:03, Yakov Zhdanov пишет:
>
> I think I have already agreed on a separate class since it seems to be the
>> only option due to generics issue. Should we extract a super class?
>>
>> We can put hard requirement that filter and transformer cannot throw
>> exception (same as cache interceptor). If exception is thrown then we
>> cancel the query globally and unregister all the listeners. This may sound
>> too much but inconsistencies brought by listener notifications may be
>> terrible for app.
>>
>> --Yakov
>>
>>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Nikolay Izhikov <ni...@gmail.com>.
Hello, Yakov.

The new class is OK - got it. Thanks!

 > Should we extract a super class?

Yes, we should.
I already have done it.

See my last commit in PR - 
https://github.com/apache/ignite/pull/2372/commits/af1ed2e4dbef4ba5999f8566198cb75ad922f93b

 > We can put hard requirement that filter and transformer cannot throw
 > exception (same as cache interceptor).

I think to cancel the whole query on transformer exception is too much.
After discussion, I like the idea to skip event if transformer throws 
exception. As far as it "like regular filter" behavior.

Thoughts?


30.08.2017 16:03, Yakov Zhdanov пишет:
> I think I have already agreed on a separate class since it seems to be the
> only option due to generics issue. Should we extract a super class?
> 
> We can put hard requirement that filter and transformer cannot throw
> exception (same as cache interceptor). If exception is thrown then we
> cancel the query globally and unregister all the listeners. This may sound
> too much but inconsistencies brought by listener notifications may be
> terrible for app.
> 
> --Yakov
> 

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Yakov Zhdanov <yz...@apache.org>.
I think I have already agreed on a separate class since it seems to be the
only option due to generics issue. Should we extract a super class?

We can put hard requirement that filter and transformer cannot throw
exception (same as cache interceptor). If exception is thrown then we
cancel the query globally and unregister all the listeners. This may sound
too much but inconsistencies brought by listener notifications may be
terrible for app.

--Yakov

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Nikolay Izhikov <ni...@gmail.com>.
Semyon, Yakov,

Here is ticket - https://issues.apache.org/jira/browse/IGNITE-6221

Thank you.

Can we now return to my questions about IGNITE-425?

1. I introduce new query class - ContinuousQueryWithTransformer [3].
Reasons:
* ContinuousQuery is final so user can't extends it. I don't want to
change that.
* ContinuousQuery contains some deprecated methods(setRemoteFilter) so
with new class we can get rid of them.
* Such public API design disallow usage of existing localEventListener
with new transformedEventListenr in compile time.

Thoughts?

2. What behavior is expected if transformer throws exception for some
event? I see following options:

* Pass `null` to listener(pull request implementation).
* Skip event. Don't call listener.
* Introduce special callback. onTransformError?

Thoughts?


30.08.2017 10:38, Semyon Boikov пишет:
> Hi,
> 
> I had an impression that current behavior is required by jcache, but now I
> can not find anything about this neither in spec nor in jcache tck tests.
> So I think we can change current behavior.
> 
> Thanks
> 
> On Tue, Aug 29, 2017 at 9:48 PM, Nikolay Izhikov <ni...@gmail.com>
> wrote:
> 
>> Yakov.
>>
>> I found following description in jcache javadoc [1] -
>>
>> ===
>> Returns:
>>     true if the evaluation passes, otherwise false.
>>     *The effect of returning true is that listener will be invoked*
>> ===
>>
>> JSR doesn't specify how filter exception has to be handled.
>> As far as I can understand *only* way to pass filter is return true from
>> `evaluate`.
>>
>> I think we has to change current behavior.
>>
>> Should I file a ticket?
>>
>> [1] https://static.javadoc.io/javax.cache/cache-api/1.0.0/javax/
>> cache/event/CacheEntryEventFilter.html#evaluate(javax.cache.
>> event.CacheEntryEvent)
>>
>>
>> 29.08.2017 17:09, Yakov Zhdanov пишет:
>>
>> Igniters,
>>>
>>> Does anyone else see potential issues on user side with current approach?
>>>
>>> Sam, is this JCache requirement?
>>>
>>> --Yakov
>>>
>>> 2017-08-29 15:16 GMT+03:00 Nikolay Izhikov <ni...@gmail.com>:
>>>
>>> Yakov.
>>>>
>>>> I think exception equals `true` is intended behavior.
>>>>
>>>> Filter evaluation implementation from master - [1]
>>>> Test from master to check filter exception(without explicit asserts
>>>> checking listeners call) - [2]
>>>>
>>>> Here is my quick test with asserts on listener call after filter
>>>> exception:
>>>>
>>>> ```
>>>> package org.apache.ignite.internal.processors.cache.query.continuous;
>>>>
>>>> //... imports
>>>>
>>>> public class GridCacheContinuousQueryFilterExceptionTest extends
>>>> GridCacheContinuousQueryAbstractSelfTest implements Serializable {
>>>>       /**
>>>>        * @throws Exception If failed.
>>>>        */
>>>>       public void testListenerAfterFilterException() throws Exception {
>>>>           IgniteCache<Integer, Integer> cache =
>>>> grid(0).cache(DEFAULT_CACHE_NAME);
>>>>
>>>>           ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
>>>>
>>>>           final CountDownLatch latch = new CountDownLatch(100);
>>>>
>>>>           qry.setLocalListener(new CacheEntryUpdatedListener<Integer,
>>>> Integer>() {
>>>>               @Override public void onUpdated(Iterable<CacheEntryEvent<?
>>>> extends Integer, ? extends Integer>> evts) {
>>>>                   for (CacheEntryEvent<? extends Integer, ? extends
>>>> Integer>
>>>> evt : evts)
>>>>                       latch.countDown();
>>>>               }
>>>>           });
>>>>
>>>>           qry.setRemoteFilter(new CacheEntryEventSerializableFil
>>>> ter<Integer,
>>>> Integer>() {
>>>>               @Override public boolean evaluate(CacheEntryEvent<? extends
>>>> Integer, ? extends Integer> evt) {
>>>>                   throw new RuntimeException("Test error.");
>>>>               }
>>>>           });
>>>>
>>>>           try (QueryCursor<Cache.Entry<Integer, Integer>> ignored =
>>>> cache.query(qry)) {
>>>>               for (int i = 0; i < 100; i++)
>>>>                   cache.put(i, i);
>>>>
>>>>               assertTrue(latch.await(10, SECONDS));
>>>>           }
>>>>       }
>>>>
>>>>       @Override protected CacheMode cacheMode() {
>>>>           return CacheMode.REPLICATED;
>>>>       }
>>>>
>>>>       @Override protected int gridCount() {
>>>>           return 1;
>>>>       }
>>>> }
>>>> ```
>>>>
>>>> [1] https://github.com/apache/ignite/blob/master/modules/core/
>>>> src/main/java/org/apache/ignite/internal/processors/cache/
>>>> query/continuous/CacheContinuousQueryHandler.java#L791
>>>>
>>>> [2] https://github.com/apache/ignite/blob/master/modules/core/
>>>> src/test/java/org/apache/ignite/internal/processors/cache/
>>>> query/continuous/GridCacheContinuousQueryAbstractSelfTest.java#L359
>>>>
>>>>
>>>> 29.08.2017 14:46, Yakov Zhdanov пишет:
>>>>
>>>> If filter throws exception entry would be passed to listener.
>>>>
>>>>>
>>>>>>
>>>>> this is strange. Imagine a filter that very rarely throws some runtime
>>>>> exception due to external or environmental reasons, but in case of
>>>>> normal
>>>>> execution filter evaluates to false. In case of error entry is passed
>>>>> to a
>>>>> local listener which can lead to some serious consequences and
>>>>> inconsistencies in business logic. We probably need to send entry with a
>>>>> notion that there was an error on server.
>>>>>
>>>>> --Yakov
>>>>>
>>>>>
>>>>>
>>>
> 

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Semyon Boikov <sb...@gridgain.com>.
Hi,

I had an impression that current behavior is required by jcache, but now I
can not find anything about this neither in spec nor in jcache tck tests.
So I think we can change current behavior.

Thanks

On Tue, Aug 29, 2017 at 9:48 PM, Nikolay Izhikov <ni...@gmail.com>
wrote:

> Yakov.
>
> I found following description in jcache javadoc [1] -
>
> ===
> Returns:
>    true if the evaluation passes, otherwise false.
>    *The effect of returning true is that listener will be invoked*
> ===
>
> JSR doesn't specify how filter exception has to be handled.
> As far as I can understand *only* way to pass filter is return true from
> `evaluate`.
>
> I think we has to change current behavior.
>
> Should I file a ticket?
>
> [1] https://static.javadoc.io/javax.cache/cache-api/1.0.0/javax/
> cache/event/CacheEntryEventFilter.html#evaluate(javax.cache.
> event.CacheEntryEvent)
>
>
> 29.08.2017 17:09, Yakov Zhdanov пишет:
>
> Igniters,
>>
>> Does anyone else see potential issues on user side with current approach?
>>
>> Sam, is this JCache requirement?
>>
>> --Yakov
>>
>> 2017-08-29 15:16 GMT+03:00 Nikolay Izhikov <ni...@gmail.com>:
>>
>> Yakov.
>>>
>>> I think exception equals `true` is intended behavior.
>>>
>>> Filter evaluation implementation from master - [1]
>>> Test from master to check filter exception(without explicit asserts
>>> checking listeners call) - [2]
>>>
>>> Here is my quick test with asserts on listener call after filter
>>> exception:
>>>
>>> ```
>>> package org.apache.ignite.internal.processors.cache.query.continuous;
>>>
>>> //... imports
>>>
>>> public class GridCacheContinuousQueryFilterExceptionTest extends
>>> GridCacheContinuousQueryAbstractSelfTest implements Serializable {
>>>      /**
>>>       * @throws Exception If failed.
>>>       */
>>>      public void testListenerAfterFilterException() throws Exception {
>>>          IgniteCache<Integer, Integer> cache =
>>> grid(0).cache(DEFAULT_CACHE_NAME);
>>>
>>>          ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
>>>
>>>          final CountDownLatch latch = new CountDownLatch(100);
>>>
>>>          qry.setLocalListener(new CacheEntryUpdatedListener<Integer,
>>> Integer>() {
>>>              @Override public void onUpdated(Iterable<CacheEntryEvent<?
>>> extends Integer, ? extends Integer>> evts) {
>>>                  for (CacheEntryEvent<? extends Integer, ? extends
>>> Integer>
>>> evt : evts)
>>>                      latch.countDown();
>>>              }
>>>          });
>>>
>>>          qry.setRemoteFilter(new CacheEntryEventSerializableFil
>>> ter<Integer,
>>> Integer>() {
>>>              @Override public boolean evaluate(CacheEntryEvent<? extends
>>> Integer, ? extends Integer> evt) {
>>>                  throw new RuntimeException("Test error.");
>>>              }
>>>          });
>>>
>>>          try (QueryCursor<Cache.Entry<Integer, Integer>> ignored =
>>> cache.query(qry)) {
>>>              for (int i = 0; i < 100; i++)
>>>                  cache.put(i, i);
>>>
>>>              assertTrue(latch.await(10, SECONDS));
>>>          }
>>>      }
>>>
>>>      @Override protected CacheMode cacheMode() {
>>>          return CacheMode.REPLICATED;
>>>      }
>>>
>>>      @Override protected int gridCount() {
>>>          return 1;
>>>      }
>>> }
>>> ```
>>>
>>> [1] https://github.com/apache/ignite/blob/master/modules/core/
>>> src/main/java/org/apache/ignite/internal/processors/cache/
>>> query/continuous/CacheContinuousQueryHandler.java#L791
>>>
>>> [2] https://github.com/apache/ignite/blob/master/modules/core/
>>> src/test/java/org/apache/ignite/internal/processors/cache/
>>> query/continuous/GridCacheContinuousQueryAbstractSelfTest.java#L359
>>>
>>>
>>> 29.08.2017 14:46, Yakov Zhdanov пишет:
>>>
>>> If filter throws exception entry would be passed to listener.
>>>
>>>>
>>>>>
>>>> this is strange. Imagine a filter that very rarely throws some runtime
>>>> exception due to external or environmental reasons, but in case of
>>>> normal
>>>> execution filter evaluates to false. In case of error entry is passed
>>>> to a
>>>> local listener which can lead to some serious consequences and
>>>> inconsistencies in business logic. We probably need to send entry with a
>>>> notion that there was an error on server.
>>>>
>>>> --Yakov
>>>>
>>>>
>>>>
>>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Nikolay Izhikov <ni...@gmail.com>.
Yakov.

I found following description in jcache javadoc [1] -

===
Returns:
    true if the evaluation passes, otherwise false.
    *The effect of returning true is that listener will be invoked*
===

JSR doesn't specify how filter exception has to be handled.
As far as I can understand *only* way to pass filter is return true from 
`evaluate`.

I think we has to change current behavior.

Should I file a ticket?

[1] 
https://static.javadoc.io/javax.cache/cache-api/1.0.0/javax/cache/event/CacheEntryEventFilter.html#evaluate(javax.cache.event.CacheEntryEvent)


29.08.2017 17:09, Yakov Zhdanov пишет:
> Igniters,
> 
> Does anyone else see potential issues on user side with current approach?
> 
> Sam, is this JCache requirement?
> 
> --Yakov
> 
> 2017-08-29 15:16 GMT+03:00 Nikolay Izhikov <ni...@gmail.com>:
> 
>> Yakov.
>>
>> I think exception equals `true` is intended behavior.
>>
>> Filter evaluation implementation from master - [1]
>> Test from master to check filter exception(without explicit asserts
>> checking listeners call) - [2]
>>
>> Here is my quick test with asserts on listener call after filter exception:
>>
>> ```
>> package org.apache.ignite.internal.processors.cache.query.continuous;
>>
>> //... imports
>>
>> public class GridCacheContinuousQueryFilterExceptionTest extends
>> GridCacheContinuousQueryAbstractSelfTest implements Serializable {
>>      /**
>>       * @throws Exception If failed.
>>       */
>>      public void testListenerAfterFilterException() throws Exception {
>>          IgniteCache<Integer, Integer> cache =
>> grid(0).cache(DEFAULT_CACHE_NAME);
>>
>>          ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
>>
>>          final CountDownLatch latch = new CountDownLatch(100);
>>
>>          qry.setLocalListener(new CacheEntryUpdatedListener<Integer,
>> Integer>() {
>>              @Override public void onUpdated(Iterable<CacheEntryEvent<?
>> extends Integer, ? extends Integer>> evts) {
>>                  for (CacheEntryEvent<? extends Integer, ? extends Integer>
>> evt : evts)
>>                      latch.countDown();
>>              }
>>          });
>>
>>          qry.setRemoteFilter(new CacheEntryEventSerializableFilter<Integer,
>> Integer>() {
>>              @Override public boolean evaluate(CacheEntryEvent<? extends
>> Integer, ? extends Integer> evt) {
>>                  throw new RuntimeException("Test error.");
>>              }
>>          });
>>
>>          try (QueryCursor<Cache.Entry<Integer, Integer>> ignored =
>> cache.query(qry)) {
>>              for (int i = 0; i < 100; i++)
>>                  cache.put(i, i);
>>
>>              assertTrue(latch.await(10, SECONDS));
>>          }
>>      }
>>
>>      @Override protected CacheMode cacheMode() {
>>          return CacheMode.REPLICATED;
>>      }
>>
>>      @Override protected int gridCount() {
>>          return 1;
>>      }
>> }
>> ```
>>
>> [1] https://github.com/apache/ignite/blob/master/modules/core/
>> src/main/java/org/apache/ignite/internal/processors/cache/
>> query/continuous/CacheContinuousQueryHandler.java#L791
>>
>> [2] https://github.com/apache/ignite/blob/master/modules/core/
>> src/test/java/org/apache/ignite/internal/processors/cache/
>> query/continuous/GridCacheContinuousQueryAbstractSelfTest.java#L359
>>
>>
>> 29.08.2017 14:46, Yakov Zhdanov пишет:
>>
>> If filter throws exception entry would be passed to listener.
>>>>
>>>
>>> this is strange. Imagine a filter that very rarely throws some runtime
>>> exception due to external or environmental reasons, but in case of normal
>>> execution filter evaluates to false. In case of error entry is passed to a
>>> local listener which can lead to some serious consequences and
>>> inconsistencies in business logic. We probably need to send entry with a
>>> notion that there was an error on server.
>>>
>>> --Yakov
>>>
>>>
> 

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Yakov Zhdanov <yz...@apache.org>.
Igniters,

Does anyone else see potential issues on user side with current approach?

Sam, is this JCache requirement?

--Yakov

2017-08-29 15:16 GMT+03:00 Nikolay Izhikov <ni...@gmail.com>:

> Yakov.
>
> I think exception equals `true` is intended behavior.
>
> Filter evaluation implementation from master - [1]
> Test from master to check filter exception(without explicit asserts
> checking listeners call) - [2]
>
> Here is my quick test with asserts on listener call after filter exception:
>
> ```
> package org.apache.ignite.internal.processors.cache.query.continuous;
>
> //... imports
>
> public class GridCacheContinuousQueryFilterExceptionTest extends
> GridCacheContinuousQueryAbstractSelfTest implements Serializable {
>     /**
>      * @throws Exception If failed.
>      */
>     public void testListenerAfterFilterException() throws Exception {
>         IgniteCache<Integer, Integer> cache =
> grid(0).cache(DEFAULT_CACHE_NAME);
>
>         ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();
>
>         final CountDownLatch latch = new CountDownLatch(100);
>
>         qry.setLocalListener(new CacheEntryUpdatedListener<Integer,
> Integer>() {
>             @Override public void onUpdated(Iterable<CacheEntryEvent<?
> extends Integer, ? extends Integer>> evts) {
>                 for (CacheEntryEvent<? extends Integer, ? extends Integer>
> evt : evts)
>                     latch.countDown();
>             }
>         });
>
>         qry.setRemoteFilter(new CacheEntryEventSerializableFilter<Integer,
> Integer>() {
>             @Override public boolean evaluate(CacheEntryEvent<? extends
> Integer, ? extends Integer> evt) {
>                 throw new RuntimeException("Test error.");
>             }
>         });
>
>         try (QueryCursor<Cache.Entry<Integer, Integer>> ignored =
> cache.query(qry)) {
>             for (int i = 0; i < 100; i++)
>                 cache.put(i, i);
>
>             assertTrue(latch.await(10, SECONDS));
>         }
>     }
>
>     @Override protected CacheMode cacheMode() {
>         return CacheMode.REPLICATED;
>     }
>
>     @Override protected int gridCount() {
>         return 1;
>     }
> }
> ```
>
> [1] https://github.com/apache/ignite/blob/master/modules/core/
> src/main/java/org/apache/ignite/internal/processors/cache/
> query/continuous/CacheContinuousQueryHandler.java#L791
>
> [2] https://github.com/apache/ignite/blob/master/modules/core/
> src/test/java/org/apache/ignite/internal/processors/cache/
> query/continuous/GridCacheContinuousQueryAbstractSelfTest.java#L359
>
>
> 29.08.2017 14:46, Yakov Zhdanov пишет:
>
> If filter throws exception entry would be passed to listener.
>>>
>>
>> this is strange. Imagine a filter that very rarely throws some runtime
>> exception due to external or environmental reasons, but in case of normal
>> execution filter evaluates to false. In case of error entry is passed to a
>> local listener which can lead to some serious consequences and
>> inconsistencies in business logic. We probably need to send entry with a
>> notion that there was an error on server.
>>
>> --Yakov
>>
>>

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Nikolay Izhikov <ni...@gmail.com>.
Yakov.

I think exception equals `true` is intended behavior.

Filter evaluation implementation from master - [1]
Test from master to check filter exception(without explicit asserts 
checking listeners call) - [2]

Here is my quick test with asserts on listener call after filter exception:

```
package org.apache.ignite.internal.processors.cache.query.continuous;

//... imports

public class GridCacheContinuousQueryFilterExceptionTest extends 
GridCacheContinuousQueryAbstractSelfTest implements Serializable {
     /**
      * @throws Exception If failed.
      */
     public void testListenerAfterFilterException() throws Exception {
         IgniteCache<Integer, Integer> cache = 
grid(0).cache(DEFAULT_CACHE_NAME);

         ContinuousQuery<Integer, Integer> qry = new ContinuousQuery<>();

         final CountDownLatch latch = new CountDownLatch(100);

         qry.setLocalListener(new CacheEntryUpdatedListener<Integer, 
Integer>() {
             @Override public void onUpdated(Iterable<CacheEntryEvent<? 
extends Integer, ? extends Integer>> evts) {
                 for (CacheEntryEvent<? extends Integer, ? extends 
Integer> evt : evts)
                     latch.countDown();
             }
         });

         qry.setRemoteFilter(new 
CacheEntryEventSerializableFilter<Integer, Integer>() {
             @Override public boolean evaluate(CacheEntryEvent<? extends 
Integer, ? extends Integer> evt) {
                 throw new RuntimeException("Test error.");
             }
         });

         try (QueryCursor<Cache.Entry<Integer, Integer>> ignored = 
cache.query(qry)) {
             for (int i = 0; i < 100; i++)
                 cache.put(i, i);

             assertTrue(latch.await(10, SECONDS));
         }
     }

     @Override protected CacheMode cacheMode() {
         return CacheMode.REPLICATED;
     }

     @Override protected int gridCount() {
         return 1;
     }
}
```

[1] 
https://github.com/apache/ignite/blob/master/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java#L791

[2] 
https://github.com/apache/ignite/blob/master/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java#L359


29.08.2017 14:46, Yakov Zhdanov пишет:
>> If filter throws exception entry would be passed to listener.
> 
> this is strange. Imagine a filter that very rarely throws some runtime
> exception due to external or environmental reasons, but in case of normal
> execution filter evaluates to false. In case of error entry is passed to a
> local listener which can lead to some serious consequences and
> inconsistencies in business logic. We probably need to send entry with a
> notion that there was an error on server.
> 
> --Yakov
> 

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Yakov Zhdanov <yz...@apache.org>.
> If filter throws exception entry would be passed to listener.

this is strange. Imagine a filter that very rarely throws some runtime
exception due to external or environmental reasons, but in case of normal
execution filter evaluates to false. In case of error entry is passed to a
local listener which can lead to some serious consequences and
inconsistencies in business logic. We probably need to send entry with a
notion that there was an error on server.

--Yakov

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Nikolay Izhikov <ni...@gmail.com>.
Hi, Yakov.

If filter throws exception entry would be passed to listener.

 > Nikolay, I would also suggest you extract some super class for continuous
 > query. It will help to avoid code duplicates.

Yes, I will do this after reaching consensus on API changes.

29.08.2017 14:04, Yakov Zhdanov пишет:
> I don't like the idea of having separate class, but it seems to be the only
> way as there are too many API and generics differences.
> 
> Nikolay, I would also suggest you extract some super class for continuous
> query. It will help to avoid code duplicates.
> 
> As far as remote transformer failure - we should react in the same way as
> we react now for filter failure. I don't think there should be a
> difference. What is the reaction now?
> 
> --Yakov
> 

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Yakov Zhdanov <yz...@apache.org>.
I don't like the idea of having separate class, but it seems to be the only
way as there are too many API and generics differences.

Nikolay, I would also suggest you extract some super class for continuous
query. It will help to avoid code duplicates.

As far as remote transformer failure - we should react in the same way as
we react now for filter failure. I don't think there should be a
difference. What is the reaction now?

--Yakov

Re: ContinuousQueryWithTransformer implementation questions - 2

Posted by Anton Vinogradov <av...@gridgain.com>.
Nikolay,

> 2. What behavior is expected if transformer throws exception for some
event? I see following options:

Client should be notified, I vote for
> Introduce special callback. onTransformError?


On Tue, Aug 29, 2017 at 1:36 PM, Nikolay Izhikov <ni...@gmail.com>
wrote:

> Hello, Igniters.
>
> I'm working on IGNITE-425 [1] issue.
>
> Text of issue:
>
> ===
> Currently if updated entry passes the filter, it is sent to node initiated
> the query entirely. It would be good to provide user with the ability to
> transform entry and, for example, select only fields that are important.
> This may bring huge economy to traffic and lower GC pressure as well.
> ===
>
> My pull request [2] is ready.
> Anton Vinogradov is OK with it.
> Nikolay Tikhonov reviewed my changes and want to discuss changes related
> to public API with community.
>
> 1. I introduce new query class - ContinuousQueryWithTransformer [3].
> Reasons:
> * ContinuousQuery is final so user can't extends it. I don't want to
> change that.
> * ContinuousQuery contains some deprecated methods(setRemoteFilter) so
> with new class we can get rid of them.
> * Such public API design disallow usage of existing localEventListener
> with new transformedEventListenr in compile time.
>
> Thoughts?
>
> 2. What behavior is expected if transformer throws exception for some
> event? I see following options:
>
> * Pass `null` to listener(pull request implementation).
> * Skip event. Don't call listener.
> * Introduce special callback. onTransformError?
>
> Thoughts?
>
> ```
> public final class ContinuousQueryWithTransformer<K, V, T> extends
> Query<Cache.Entry<K, V>> {
>     //...
>
>     private Factory<? extends CacheEntryEventFilter<K, V>>
> rmtFilterFactory;
>
>     private Factory<? extends IgniteClosure<CacheEntryEvent<? extends K,
> ? extends V>, T>> rmtTransFactory;
>
>     private EventListener<T> locLsnr;
>
>     //...
>
>     public interface EventListener<T> {
>         void onUpdated(Iterable<? extends T> events);
>     }
> }
> ```
>
> Previous discussion - [4]
>
> [1] https://issues.apache.org/jira/browse/IGNITE-425
> [2] https://github.com/apache/ignite/pull/2372
> [3] https://github.com/apache/ignite/pull/2372/files#diff-22cc0c
> f0bc428b32a39e6cc0b22b0e3e
> [4] http://apache-ignite-developers.2346864.n4.nabble.com/Contin
> uousQueryWithTransformer-implementation-questions-td20078.html
>