You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@ignite.apache.org by Guilherme Melo <gu...@gmelo.org> on 2017/07/20 10:32:53 UTC

Transformer not called on every ScanQuery

Hello,
Has anyone had issues with Transformers not being called on every instance
of the scanQuery?
only about 10% of the objects get transformed, the rest are added to the
list returned from cache.query() are IgniteBiTuple, not what was passed to
the Transformer.
this does not happen when running the query without the transformer.

Thank you

Re: Transformer not called on every ScanQuery

Posted by Andrey Mashenkov <an...@gmail.com>.
Hi Guilherme,

Thanks for the repro. It is a bug.
I've create a ticket [1]

[1] https://issues.apache.org/jira/browse/IGNITE-5804

On Fri, Jul 21, 2017 at 2:04 PM, Guilherme Melo <gu...@gmail.com>
wrote:

> Hello Andrei, upon further investigation it seems I have found a bug, the
> scenario is this: when running a query from a client node, if the result is
> bigger than the default, or set page size the objects returned are not
> transformed, but rather org.apache.ignite.internal.processors.cache.query.
> GridCacheQueryResponseEntry.
> I found this while running 1.9
> below is a simple test I wrote to prove it, its not the most elegant way
> to show it, but I did not have a lot of time.
>
> public class SimpleAlwaysTruePredicate implements IgniteBiPredicate<String,Integer> {
>     @Override
>     public boolean apply(String s, Integer integer) {
>         return true;
>     }
> }
>
>
> public class SimpleOnlyValueTransformer implements IgniteClosure<Cache.Entry<String, Integer>, Integer> {
>     @Override
>     public Integer apply(Cache.Entry<String, Integer> stringIntegerEntry) {
>         return stringIntegerEntry.getValue();
>     }
> }
>
>
> public static void main(String[] args) {
>
>     Ignition.setClientMode(true);
>     Ignite ignite = Ignition.start();
>
>     IgniteCache<String, Integer> cache = ignite.createCache("test");
>
>     System.out.println("Running a query with a transformer less than the default page size");
>     //less than the default page size, no problem
>     for (int i = 0; i < 1024; i++) {
>         cache.put(String.valueOf(i), i);
>     }
>
>     List<Integer> returned = cache.query(new ScanQuery<String, Integer>(new SimpleAlwaysTruePredicate()),
>             new SimpleOnlyValueTransformer()).getAll();
>
>     returned.forEach(i -> doNothing(i));
>     cache.clear();
>     cache.destroy();
>
>     cache = ignite.createCache("test");
>
>     System.out.println("Running a query with a transformer more than the default page size");
>     //one more than the default page size, throws exception
>     for (int i = 0; i < 1025; i++) {
>         cache.put(String.valueOf(i), i);
>     }
>
>     returned = cache.query(new ScanQuery<String, Integer>(new SimpleAlwaysTruePredicate()),
>             new SimpleOnlyValueTransformer()).getAll();
>
>     try {
>         returned.forEach(i -> doNothing(i));
>
>     } catch (Exception e) {
>         System.out.println(e);
>     }
>     cache.clear();
>     cache.destroy();
>     //just to make sure:
>     System.out.println("Running a query with a default page size and one more entry then it");
>     cache = ignite.createCache("test");
>     cache.put("one", 1);
>     cache.put("two", 2);
>     Query<Cache.Entry<String, Integer>> query = new ScanQuery<>(new SimpleAlwaysTruePredicate());
>     query.setPageSize(1);
>
>     returned = cache.query(query,
>             new SimpleOnlyValueTransformer()).getAll();
>
>
>     try {
>         returned.forEach(i -> doNothing(i));
>
>     } catch (Exception e) {
>         System.out.println(e);
>     }
>     cache.clear();
>     cache.destroy();
>     Ignition.stop(true);
> }
>
> public static void doNothing(Integer i) {
>
> }
>
>
> Cheers,
>
> Guilherme Melo
> www.gmelo.org
>
> On 20 July 2017 at 23:13, Andrey Mashenkov <an...@gmail.com>
> wrote:
>
>> Hi Guilherme,
>>
>> What version of ignite do you use?
>> Can you share a reproducer?
>>
>> On Thu, Jul 20, 2017 at 1:32 PM, Guilherme Melo <gu...@gmelo.org>
>> wrote:
>>
>>> Hello,
>>> Has anyone had issues with Transformers not being called on every
>>> instance of the scanQuery?
>>> only about 10% of the objects get transformed, the rest are added to the
>>> list returned from cache.query() are IgniteBiTuple, not what was passed to
>>> the Transformer.
>>> this does not happen when running the query without the transformer.
>>>
>>> Thank you
>>>
>>>
>>
>>
>> --
>> Best regards,
>> Andrey V. Mashenkov
>>
>
>


-- 
Best regards,
Andrey V. Mashenkov

Re: Transformer not called on every ScanQuery

Posted by Guilherme Melo <gu...@gmail.com>.
Hello Andrei, upon further investigation it seems I have found a bug, the
scenario is this: when running a query from a client node, if the result is
bigger than the default, or set page size the objects returned are not
transformed, but
rather org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponseEntry.
I found this while running 1.9
below is a simple test I wrote to prove it, its not the most elegant way to
show it, but I did not have a lot of time.

public class SimpleAlwaysTruePredicate implements
IgniteBiPredicate<String,Integer> {
    @Override
    public boolean apply(String s, Integer integer) {
        return true;
    }
}


public class SimpleOnlyValueTransformer implements
IgniteClosure<Cache.Entry<String, Integer>, Integer> {
    @Override
    public Integer apply(Cache.Entry<String, Integer> stringIntegerEntry) {
        return stringIntegerEntry.getValue();
    }
}


public static void main(String[] args) {

    Ignition.setClientMode(true);
    Ignite ignite = Ignition.start();

    IgniteCache<String, Integer> cache = ignite.createCache("test");

    System.out.println("Running a query with a transformer less than
the default page size");
    //less than the default page size, no problem
    for (int i = 0; i < 1024; i++) {
        cache.put(String.valueOf(i), i);
    }

    List<Integer> returned = cache.query(new ScanQuery<String,
Integer>(new SimpleAlwaysTruePredicate()),
            new SimpleOnlyValueTransformer()).getAll();

    returned.forEach(i -> doNothing(i));
    cache.clear();
    cache.destroy();

    cache = ignite.createCache("test");

    System.out.println("Running a query with a transformer more than
the default page size");
    //one more than the default page size, throws exception
    for (int i = 0; i < 1025; i++) {
        cache.put(String.valueOf(i), i);
    }

    returned = cache.query(new ScanQuery<String, Integer>(new
SimpleAlwaysTruePredicate()),
            new SimpleOnlyValueTransformer()).getAll();

    try {
        returned.forEach(i -> doNothing(i));

    } catch (Exception e) {
        System.out.println(e);
    }
    cache.clear();
    cache.destroy();
    //just to make sure:
    System.out.println("Running a query with a default page size and
one more entry then it");
    cache = ignite.createCache("test");
    cache.put("one", 1);
    cache.put("two", 2);
    Query<Cache.Entry<String, Integer>> query = new ScanQuery<>(new
SimpleAlwaysTruePredicate());
    query.setPageSize(1);

    returned = cache.query(query,
            new SimpleOnlyValueTransformer()).getAll();


    try {
        returned.forEach(i -> doNothing(i));

    } catch (Exception e) {
        System.out.println(e);
    }
    cache.clear();
    cache.destroy();
    Ignition.stop(true);
}

public static void doNothing(Integer i) {

}


Cheers,

Guilherme Melo
www.gmelo.org

On 20 July 2017 at 23:13, Andrey Mashenkov <an...@gmail.com>
wrote:

> Hi Guilherme,
>
> What version of ignite do you use?
> Can you share a reproducer?
>
> On Thu, Jul 20, 2017 at 1:32 PM, Guilherme Melo <gu...@gmelo.org>
> wrote:
>
>> Hello,
>> Has anyone had issues with Transformers not being called on every
>> instance of the scanQuery?
>> only about 10% of the objects get transformed, the rest are added to the
>> list returned from cache.query() are IgniteBiTuple, not what was passed to
>> the Transformer.
>> this does not happen when running the query without the transformer.
>>
>> Thank you
>>
>>
>
>
> --
> Best regards,
> Andrey V. Mashenkov
>

Re: Transformer not called on every ScanQuery

Posted by Andrey Mashenkov <an...@gmail.com>.
Hi Guilherme,

What version of ignite do you use?
Can you share a reproducer?

On Thu, Jul 20, 2017 at 1:32 PM, Guilherme Melo <gu...@gmelo.org> wrote:

> Hello,
> Has anyone had issues with Transformers not being called on every instance
> of the scanQuery?
> only about 10% of the objects get transformed, the rest are added to the
> list returned from cache.query() are IgniteBiTuple, not what was passed to
> the Transformer.
> this does not happen when running the query without the transformer.
>
> Thank you
>
>


-- 
Best regards,
Andrey V. Mashenkov