You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2020/04/16 14:28:36 UTC

[GitHub] [ignite] bratwurzt opened a new pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

bratwurzt opened a new pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680
 
 
   Because QueryKeyValueIterable does not implement spliterator, calling `spliterator().hasCharacteristics(Spliterator.SIZED)` will construct a new Spliterator with `java.util.Spliterators#spliteratorUnknownSize(java.util.Iterator<? extends T>, int)` and that will call `org.apache.ignite.internal.processors.cache.QueryCursorImpl#iter` again, thus throwing exception `throw new IgniteException("Iterator is already fetched or query was cancelled.")`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410461125
 
 

 ##########
 File path: modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
 ##########
 @@ -305,6 +312,34 @@ public void testCacheContinuousQueryReconnectNewServer() throws Exception {
         }
     }
 
+    @Test
+    public void testCacheContinuousQuerySpliteratorMultipleCalls() {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
+
+        CacheEventListener lsnr = new CacheEventListener();
+
+        ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+        qry.setInitialQuery(new ScanQuery<>(new IgniteBiPredicate<Object, Object>() {
+            @Override public boolean apply(Object key, Object val) {
+                return key != null;
+            }
+        }));
+        qry.setAutoUnsubscribe(true);
+
+        qry.setLocalListener(lsnr);
+
+        try(QueryCursor<?> cur = clientCache.query(qry)) {
+            cur.iterator();
+            cur.spliterator();
 
 Review comment:
   Note that an instance of `java.util.Spliterators$IteratorSpliterator` is returned here, and only the usage of deprecated `SqlQuery` will cause returning of `QueryKeyValueSpliterator` instance, it meets your expectations?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410411478
 
 

 ##########
 File path: modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
 ##########
 @@ -305,6 +312,34 @@ public void testCacheContinuousQueryReconnectNewServer() throws Exception {
         }
     }
 
+    @Test
+    public void testCacheContinuousQuerySpliteratorMultipleCalls() {
 
 Review comment:
   Add javadoc please!
   https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-JavadocComments
   
   Also, I think you choose wrong test class for your test because that related to client reconnect issues rather than query cursor behavior. You should find more suitable test class in packages org.apache.ignite.internal.processors.cache or org.apache.ignite.internal.processors.query. In fact, such class may not exists, in that case you should create own there.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410427724
 
 

 ##########
 File path: modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
 ##########
 @@ -305,6 +312,34 @@ public void testCacheContinuousQueryReconnectNewServer() throws Exception {
         }
     }
 
+    @Test
+    public void testCacheContinuousQuerySpliteratorMultipleCalls() {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
+
+        CacheEventListener lsnr = new CacheEventListener();
+
+        ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+        qry.setInitialQuery(new ScanQuery<>(new IgniteBiPredicate<Object, Object>() {
+            @Override public boolean apply(Object key, Object val) {
+                return key != null;
+            }
+        }));
+        qry.setAutoUnsubscribe(true);
+
+        qry.setLocalListener(lsnr);
+
+        try(QueryCursor<?> cur = clientCache.query(qry)) {
+            cur.iterator();
+            cur.spliterator();
+            GridTestUtils.assertThrows(log, IgniteException.class, "Iterator is already fetched or query was cancelled.", cur, "iterator");
 
 Review comment:
   Add empty line before.
   
   See https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-Whitespacesandemptylines

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410427387
 
 

 ##########
 File path: modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
 ##########
 @@ -305,6 +312,34 @@ public void testCacheContinuousQueryReconnectNewServer() throws Exception {
         }
     }
 
+    @Test
+    public void testCacheContinuousQuerySpliteratorMultipleCalls() {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
+
+        CacheEventListener lsnr = new CacheEventListener();
+
+        ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+        qry.setInitialQuery(new ScanQuery<>(new IgniteBiPredicate<Object, Object>() {
+            @Override public boolean apply(Object key, Object val) {
+                return key != null;
+            }
+        }));
+        qry.setAutoUnsubscribe(true);
 
 Review comment:
   Either add an empty line before or remove it after if you treated the configuration of `qry` as minimal semantic unit. Also, you may to chain the method calls as `ContinuousQuery` provides the fluent interface.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410421248
 
 

 ##########
 File path: modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
 ##########
 @@ -305,6 +312,34 @@ public void testCacheContinuousQueryReconnectNewServer() throws Exception {
         }
     }
 
+    @Test
+    public void testCacheContinuousQuerySpliteratorMultipleCalls() {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
+
+        CacheEventListener lsnr = new CacheEventListener();
 
 Review comment:
   This variable is redundant, just pass the constructor call as the setter's argument at line 334.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410456507
 
 

 ##########
 File path: modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
 ##########
 @@ -305,6 +312,34 @@ public void testCacheContinuousQueryReconnectNewServer() throws Exception {
         }
     }
 
+    @Test
+    public void testCacheContinuousQuerySpliteratorMultipleCalls() {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
+
+        CacheEventListener lsnr = new CacheEventListener();
+
+        ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+        qry.setInitialQuery(new ScanQuery<>(new IgniteBiPredicate<Object, Object>() {
+            @Override public boolean apply(Object key, Object val) {
+                return key != null;
+            }
+        }));
+        qry.setAutoUnsubscribe(true);
+
+        qry.setLocalListener(lsnr);
+
+        try(QueryCursor<?> cur = clientCache.query(qry)) {
 
 Review comment:
   Here you test the result of `ContinuousQuery` only, while other `Query` variations (e.g. `SqlQuery`, `SqlFieldsQuery`, `ScanQuery`) are available too and need to be tested.
   See `org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl#query(...)` implementation(s).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410296176
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
 ##########
 @@ -726,6 +728,11 @@ private ClusterGroup projection(boolean loc) {
                     return cur != null ? cur.getAll() : Collections.<Cache.Entry<K, V>>emptyList();
                 }
 
+                @Override
 
 Review comment:
   `@Override` annotation should be at the same line with method declaration.
   
   See https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-@Annotations
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] bratwurzt commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
bratwurzt commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410676005
 
 

 ##########
 File path: modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
 ##########
 @@ -305,6 +312,34 @@ public void testCacheContinuousQueryReconnectNewServer() throws Exception {
         }
     }
 
+    @Test
+    public void testCacheContinuousQuerySpliteratorMultipleCalls() {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
+
+        CacheEventListener lsnr = new CacheEventListener();
+
+        ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+        qry.setInitialQuery(new ScanQuery<>(new IgniteBiPredicate<Object, Object>() {
+            @Override public boolean apply(Object key, Object val) {
+                return key != null;
+            }
+        }));
+        qry.setAutoUnsubscribe(true);
+
+        qry.setLocalListener(lsnr);
+
+        try(QueryCursor<?> cur = clientCache.query(qry)) {
+            cur.iterator();
+            cur.spliterator();
 
 Review comment:
   Yes, I am fine with that - as long as it doesn't throw exception "Iterator is already fetched or query was cancelled."

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410299433
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/QueryCursorImpl.java
 ##########
 @@ -96,6 +98,10 @@ public QueryCursorImpl(Iterable<T> iterExec, GridQueryCancel cancel, boolean isQ
         return new AutoClosableCursorIterator<>(this, iter());
     }
 
+    @Override public Spliterator<T> spliterator() {
 
 Review comment:
   Add line with `/** {@inheritDoc} */` before.
   
   See https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-Shorterone-lineJavadoc

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410305186
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueSpliterator.java
 ##########
 @@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query;
+
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Spliterator;
+import java.util.function.Consumer;
+
+/**
+ * SqlQuery key-value spliterator.
+ */
+public class QueryKeyValueSpliterator<K, V> implements Spliterator<Cache.Entry<K, V>> {
 
 Review comment:
   Can you, please, fix the code style of this class according to Ignite rules?
   I mean `@Override` annotations and `/** {@inheritDoc} */` javadocs.
   
   See comments above and https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [ignite] x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable

Posted by GitBox <gi...@apache.org>.
x-kreator commented on a change in pull request #7680: IGNITE-12905 Custom spliterator for QueryKeyValueIterable
URL: https://github.com/apache/ignite/pull/7680#discussion_r410301148
 
 

 ##########
 File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryKeyValueIterable.java
 ##########
 @@ -44,6 +45,11 @@ public QueryKeyValueIterable(QueryCursor<List<?>> cur) {
         return new QueryKeyValueIterator<>(cur.iterator());
     }
 
+    @Override
 
 Review comment:
   Add line with `/** {@inheritDoc} */`, move `@Override` annotation to line of method declaration (see comments above).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services