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 2021/05/05 17:09:57 UTC

[GitHub] [ignite] timoninmaxim opened a new pull request #9081: IGNITE-12291. Add MergeSort distributed cache query reducer.

timoninmaxim opened a new pull request #9081:
URL: https://github.com/apache/ignite/pull/9081


   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r685276203



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;

Review comment:
       Now use PriorityQueue instead.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r693806037



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Checks whether node with provided {@code nodeId} is a map node for the query.
+     * Note: if all pages were received from this node, then the method will return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if node with provided {@code nodeId} is a map node for the query, {@code false} otherwise.
+     */
+    public boolean mapNode(UUID nodeId);

Review comment:
       WDYT if we violate this rule and name it `isMapNode`?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642337612



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public abstract class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** */
+    CacheQueryPageRequester(final GridCacheContext cctx) {
+        this.cctx = cctx;
+        this.log = cctx.kernalContext().config().getGridLogger();
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.subjectId(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        sendRequest(fut, req, nodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.subjectId(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            Collection<ClusterNode> n = new ArrayList<>();
+            for (UUID id: nodes)
+                n.add(cctx.node(id));
+
+            sendRequest(fut, req, n);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQueryRequest(long reqId, Collection<ClusterNode> nodes, boolean fieldsQry) {
+        final GridCacheQueryManager qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            // Process cancel query directly (without sending) for local node.
+            sendLocal(req);
+
+            if (!nodes.isEmpty()) {
+                for (ClusterNode node : nodes) {
+                    try {
+                        cctx.io().send(node, req, cctx.ioPolicy());
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to send cancel request, node failed: " + node);
+                        }
+                        else
+                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
+                    }
+                }
+            }
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        final GridCacheQueryFutureAdapter fut,
+        final GridCacheQueryRequest req,
+        Collection<ClusterNode> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        final UUID locNodeId = cctx.localNodeId();
+
+        ClusterNode locNode = null;
+
+        Collection<ClusterNode> rmtNodes = null;
+
+        for (ClusterNode n : nodes) {
+            if (n.id().equals(locNodeId))
+                locNode = n;
+            else {
+                if (rmtNodes == null)
+                    rmtNodes = new ArrayList<>(nodes.size());
+
+                rmtNodes.add(n);
+            }
+        }
+
+        // Request should be sent to remote nodes before the query is processed on the local node.
+        // For example, a remote reducer has a state, we should not serialize and then send
+        // the reducer changed by the local node.
+        if (!F.isEmpty(rmtNodes)) {
+            for (ClusterNode node : rmtNodes) {
+                try {
+                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
+                }
+                catch (IgniteCheckedException e) {
+                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
+                        fut.onNodeLeft(node.id());
+
+                        if (fut.isDone())
+                            return;
+                    }
+                    else
+                        throw e;
+                }
+            }
+        }
+
+        if (locNode != null)
+            sendLocal(req);
+    }
+
+    /** Send and handle request to local node. */

Review comment:
       ```suggestion
       /** 
       * Handles request locally. 
       */
   ```




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642341404



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheEntryWithPayload.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.cache.query;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/** Represents cache key-value pair and additional payload to compare cache entity by custom rule. */
+public class CacheEntryWithPayload<K, V, P> extends IgniteBiTuple<K, V> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private P payload;
+
+    /** */
+    public CacheEntryWithPayload() {}
+
+    /** */
+    public CacheEntryWithPayload(K key, V val, P payload) {
+        super(key, val);
+
+        this.payload = payload;
+    }
+
+    /** */
+    public Object payload() {

Review comment:
       ```suggestion
       public P payload() {
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645843569



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream(node.id());
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {

Review comment:
       Actually, I see a way to avoid shared lock for multi-stream reducer. We don't need such collection as `rcvd` in multi-stream, as there a stream per node. So, this code can be simplified. I'll try to implement it.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642336759



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheEntryWithPayload.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.cache.query;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/** Represents cache key-value pair and additional payload to compare cache entity by custom rule. */
+public class CacheEntryWithPayload<K, V, P> extends IgniteBiTuple<K, V> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private P payload;
+
+    /** */
+    public CacheEntryWithPayload() {}
+
+    /** */
+    public CacheEntryWithPayload(K key, V val, P payload) {
+        super(key, val);
+
+        this.payload = payload;
+    }
+
+    /** */
+    public Object payload() {
+        return payload;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        out.writeObject(payload);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        payload = (P)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "CacheEntryWithPayload{" +
+            "entry=" + super.toString() +
+            ", payload=" + payload +
+            '}';
+    }

Review comment:
       ```suggestion
       @Override public String toString() {
           return S.toString(CacheEntryWithPayload.class, this);
       }
   ```




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645168223



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;

Review comment:
       After you got the last item from the first page, you will be blocked on `next()` call until the second page will be received from the node, otherwise you will not be able to resort streams correctly.
   You need to have at least one page from every node to be ready at any time for non-blocking reducing.
   
   I wouldn't bet on equal data distribution. Assume, one node will have the largest part of the query result,
   then you will end up in synchronous ping-pong with a single node.
   
   Effective prefetching is to request next page right after you start processing the previous one.
   If you bother about fetching few additional pages, then you can configure twice smaller page, aren't you?




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655441529



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #onPage(UUID, Collection, boolean)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.

Review comment:
       ```suggestion
    * This class is responsible for reducing results of cache query. 
    *
    * Query results are delivered via callback {@link #onPage(UUID, Collection, boolean)}.
    * @see GridCacheQueryFutureAdapter.
   ```




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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r721294937



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -143,43 +126,62 @@ public GridCacheQueryBean query() {
         return qry;
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteLogger logger() {
+        return log;
+    }
+
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
+
+            checkError();
+
+            R next = null;
 
-            cnt.decrementAndGet();
+            if (reducer.hasNext()) {
+                next = unmaskNull(reducer.next());
+
+                if (!limitDisabled) {
+                    cnt++;
+
+                    // Exceed limit, stop page loading and cancel queries.
+                    if (cnt == capacity)
+                        cancel();
+                }
+            }
+
+            checkError();

Review comment:
       Should we handle any kind of exceptions here and close the `NodeStream` handlers by calling the `onPageError` method here?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r718596118



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Local handler of cache query request. */
+    private final Consumer<GridCacheQueryRequest> sendLoc;
+
+    /**
+     * @param cctx Cache context.
+     * @param sendLoc Local handler of cache query request.
+     */
+    CacheQueryPageRequester(GridCacheContext<?, ?> cctx, Consumer<GridCacheQueryRequest> sendLoc) {
+        this.cctx = cctx;
+        this.sendLoc = sendLoc;
+
+        log = cctx.kernalContext().config().getGridLogger();
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture<?, ?, ?> fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter<?> qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.idxQryDesc(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        List<UUID> sendNodes = new ArrayList<>();
+
+        for (ClusterNode n: nodes)
+            sendNodes.add(n.id());
+
+        sendRequest(fut, req, sendNodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter<?, ?, ?> fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter<?> qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            sendRequest(fut, req, nodes);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Send cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQuery(long reqId, Collection<UUID> nodes, boolean fieldsQry) {
+        GridCacheQueryManager<?, ?> qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            sendRequest(null, req, nodes);
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param fut Cache query future. {@code null} in case of cancel request.
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        UUID locNodeId = cctx.localNodeId();
+
+        boolean loc = false;
+
+        for (UUID nodeId : nodes) {
+            if (nodeId.equals(locNodeId))
+                loc = true;
+            else {
+                if (req.cancel())
+                    sendNodeCancelRequest(nodeId, req);
+                else if (!sendNodePageRequest(nodeId, req, fut))
+                    return;
+            }
+        }
+
+        if (loc)
+            sendLoc.accept(req);
+    }
+
+    /** */
+    private void sendNodeCancelRequest(UUID nodeId, GridCacheQueryRequest req) throws IgniteCheckedException {
+        try {
+            cctx.io().send(nodeId, req, GridIoPolicy.QUERY_POOL);
+        }
+        catch (IgniteCheckedException e) {
+            if (cctx.io().checkNodeLeft(nodeId, e, false)) {

Review comment:
       I'd like to remain this check. There is a logic of node left. And this code interfere to this situation. I think this behavior should be investigated with some tests. I will create a separate ticket for that.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {

Review comment:
       Done.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645160559



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {
+            if (!loadAllowed)
+                return;
+
+            nodes = new ArrayList<>();
+
+            for (int i = streamOff; i < streams.length; i++) {
+                UUID nodeId = streams[i].nodeId;
+
+                // Try to contain 2 pages for every stream to avoid waits. A node has to be present in collections:
+                // 1. rcvd - as requested and already handled before, so currently no parallel request to this node.
+                // 2. subgrid - as this node still has pages to request.
+                if (streams[i].queue.size() < 1 && rcvd.remove(nodeId) && subgrid.contains(nodeId))
+                    nodes.add(nodeId);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        boolean qryLast = super.onPage(nodeId, last);
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        if (last && streamsMap.containsKey(nodeId)) {
+            streamsMap.get(nodeId).allPagesReady = true;
+
+            streamsMap.remove(nodeId);
+        }
+
+        return qryLast;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        assert Thread.holdsLock(queueLock());
+
+        // Local node.
+        if (nodeId == null) {
+            nodeId = cctx.localNodeId();
+
+            // If nodeId is NULL and query doesn't execute on local node, then it is error, notify all streams.
+            if (!streamsMap.containsKey(nodeId)) {
+                assert data.isEmpty();
+
+                for (PageStream stream: streamsMap.values())
+                    stream.addPage(data);

Review comment:
       So, why we have a magic constant NULL instead of a separate callback for this with error as a param?
   It is totally unclear, why UnsortedReducer doesn't have a similar null check...
   
   Also, passing an empty page looks like a magic constant as well.
   Do we need to notify other streams? Do we know at this point, that prefetched result (failed) will ever be needed? 




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645594505



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;

Review comment:
       > Effective prefetching is to request next page right after you start processing the previous one
   
   It is how it's done now. By "queue is empty" I mean the queue variable in PageStream, that stores page's collections. When the queue is empty it means that we start processing last page (and data moved from queue to iterator [1]) and we request a new page in parallel. 
   
   By the link above I trigger this load right after moving page to iterator, check the condition: queue.size() < 1. So queue is empty but we still have data in iterator.
   
   We can change this condition to queue.size() < 2, so we will have at least 2 pages in any moment (1 in iterator, 1 in a queue). So it can be useful if user processing page in iterator faster then remote node responses.
   
   Am I right that you propose to do this (increment 1 to 2)? Should we do it configurable?
   
   [1] https://github.com/apache/ignite/pull/9081/files#diff-068da2d055341b4686f2dc28d687b5c58678a255cde712e4062588bf0fc4f46fR75




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682500736



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       AFAIU, only Text queries have a 'payload'. Why is it named 'payload' and not score?
   Does 'null' score make sense for the result? 




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645133977



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;
+
+    /**
+     * Callback to handle node left.
+     *
+     * @param nodeId Node ID that left a cluster.
+     * @return {@code true} if specified node runs this query.

Review comment:
       It is unclear, what result will be returned if the node has gone after the last page was received and how it should be interpreted.
   Or what if the node has gone, but we already received sufficient data for finishing the query, 
   e.g. query has limits, but the user app consumes data slowly. 
   
   I think Reducer should only check internal invariants and mark node as failed.
   And caller should recheck 'if the reducer is alive' right after calling this method.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r637840435



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryResultFetcher.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public abstract class CacheQueryResultFetcher {
+    /**
+     * Map (requestId -> query future) where request id is unique for all requests per query.
+     * This map is populated by query manager.
+     */
+    private final ConcurrentMap<Long, GridCacheDistributedQueryFuture<?, ?, ?>> qryFuts;
+
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** */
+    CacheQueryResultFetcher(
+        final GridCacheContext cctx,
+        final ConcurrentMap<Long, GridCacheDistributedQueryFuture<?, ?, ?>> qryFuts) {

Review comment:
       Agree, that the "fetcher" name isn't good. I think it's much as Trigger or Requester. 
   
   Moment to load page is depends on reducer implementation, so reducer is responsible for triggering pages. Reducer is created per request, and fetcher is single per Cache, so we can hold a reference with the same lifecycle.  
   
   Alternative is to store a future in reducer (and we can do it, as get fut in constructor), and provide it to fetcher instead of reqId. Also it help us to get rid of query qryFuts. Looks OK for me. I'll do this.
   
   
   
   
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645105503



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {

Review comment:
       MergeSorted reducer uses per-node Stream, while Unsorted uses single shared Stream and they have different logic of merging results.
   Why do you extend Unsorted reducer instead of Abstract one here?
   Some common fields can be safely pulled up into the Abstract reduceer.
   
   Also, I'd move all these classes hierarchy to the package  'reducer' and made PageStream/NodeStream to top-level classes, with the package visibility. 




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704375343



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheEntryWithPayload.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/** Represents cache key-value pair and additional payload to compare cache entity by custom rule. */
+public class CacheEntryWithPayload<K, V, P> extends IgniteBiTuple<K, V> {

Review comment:
       Would you mind to rename this to ScoredCacheEntry or WeightedCacheEntry?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r705176656



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Local handler of cache query request. */
+    private final Consumer<GridCacheQueryRequest> sendLoc;
+
+    /**
+     * @param cctx Cache context.
+     * @param sendLoc Local handler of cache query request.
+     */
+    CacheQueryPageRequester(final GridCacheContext cctx, Consumer<GridCacheQueryRequest> sendLoc) {
+        this.cctx = cctx;
+        this.log = cctx.kernalContext().config().getGridLogger();
+        this.sendLoc = sendLoc;
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        sendRequest(fut, req, nodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            Collection<ClusterNode> n = new ArrayList<>();
+            for (UUID id: nodes)
+                n.add(cctx.node(id));
+
+            sendRequest(fut, req, n);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Send cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQuery(long reqId, Collection<UUID> nodes, boolean fieldsQry) {
+        final GridCacheQueryManager qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            for (UUID node : nodes) {
+                try {
+                    if (cctx.localNodeId().equals(node)) {
+                        sendLoc.accept(req);
+
+                        continue;
+                    }
+
+                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
+                }
+                catch (IgniteCheckedException e) {
+                    if (cctx.io().checkNodeLeft(node, e, false)) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to send cancel request, node failed: " + node);
+                    }
+                    else
+                        U.error(log, "Failed to send cancel request [node=" + node + ']', e);
+                }
+            }
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        final GridCacheQueryFutureAdapter fut,
+        final GridCacheQueryRequest req,
+        Collection<ClusterNode> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        final UUID locNodeId = cctx.localNodeId();
+
+        ClusterNode locNode = null;
+
+        Collection<ClusterNode> rmtNodes = null;
+
+        for (ClusterNode n : nodes) {
+            if (n.id().equals(locNodeId))
+                locNode = n;
+            else {
+                if (rmtNodes == null)
+                    rmtNodes = new ArrayList<>(nodes.size());
+
+                rmtNodes.add(n);
+            }
+        }
+
+        // Request should be sent to remote nodes before the query is processed on the local node.
+        // For example, a remote reducer has a state, we should not serialize and then send
+        // the reducer changed by the local node.
+        if (!F.isEmpty(rmtNodes)) {
+            for (ClusterNode node : rmtNodes) {

Review comment:
       I applied this suggestions, also I unified a code of sending cancel a new pages requests.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r649802220



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/PageStream.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Iterator over pages stream. Pages are stored in a queue. After polling a queue try load a new page instead of it.
+ */
+class PageStream<R> {
+    /** Queue of data of results pages. */
+    protected final Queue<Collection<R>> queue = new LinkedList<>();
+
+    /** Iterator over current page. */
+    private Iterator<R> iter;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    private final Collection<UUID> subgrid;
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    private final Collection<UUID> rcvd = new HashSet<>();

Review comment:
       The semantic is the same. We still have a `subgrid` even it's a single node, and it also has to track `rcvd` to avoid duplicate messages. I think we should reuse this code instead of implementing separate logic for single node stream, that actually will be pretty the same.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return Head of stream and then clean it.
+     */
+    public R get() {
+        R ret = head;
+
+        head = null;
+
+        return ret;
+    }

Review comment:
       You're correct. I wanted to separate logic of filling head() and cleaning(), but actually there is no need to do it.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -324,8 +242,8 @@ public void onPage(@Nullable UUID nodeId, @Nullable Collection<?> data, @Nullabl
 
         try {
             if (err != null)
-                synchronized (this) {
-                    enqueue(Collections.emptyList());
+                synchronized (lock) {
+                    reducer().onError();

Review comment:
       We still need synchronization there, as it's required to change reducer queue atomically with future onDone. If user hands on latch await, we has to count down it after future is done. Otherwise we can miss exception, we have tests for this logic. Please check this comment:
   https://github.com/apache/ignite/blob/6b6c32bf1582d50f349f522c04b11d1603049797/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java#L330
   
   We can control future.onDone() within reducer, but I think it isn't responsibility of reducer.
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -338,55 +256,83 @@ public void onPage(@Nullable UUID nodeId, @Nullable Collection<?> data, @Nullabl
                                 "query", qry, true),
                             err));
 
-                    onPage(nodeId, true);
-
-                    notifyAll();
+                    lock.notifyAll();
                 }
             else {
                 if (data == null)
                     data = Collections.emptyList();
 
-                data = dedupIfRequired((Collection<Object>)data);
+                data = dedupIfRequired(data);
+
+                if (qry.query().type() == GridCacheQueryType.TEXT) {
+                    ArrayList unwrapped = new ArrayList();
+
+                    for (Object o: data) {
+                        CacheEntryWithPayload e = (CacheEntryWithPayload) o;
+
+                        Object uKey = CacheObjectUtils.unwrapBinary(
+                            cctx.cacheObjectContext(), e.getKey(), qry.query().keepBinary(), true, null);
+
+                        Object uVal = CacheObjectUtils.unwrapBinary(
+                            cctx.cacheObjectContext(), e.getValue(), qry.query().keepBinary(), true, null);
 
-                data = cctx.unwrapBinariesIfNeeded((Collection<Object>)data, qry.query().keepBinary());
+                        if (uKey != e.getKey() || uVal != e.getValue())
+                            unwrapped.add(new CacheEntryWithPayload<>(uKey, uVal, e.payload()));
+                        else
+                            unwrapped.add(o);
+                    }
+
+                    data = unwrapped;
+
+                } else
+                    data = cctx.unwrapBinariesIfNeeded((Collection<Object>)data, qry.query().keepBinary());
 
-                synchronized (this) {
-                    enqueue(data);
+                synchronized (lock) {

Review comment:
       Yes, it requires for syncing `queue`, `rcvd` and `fut.isDone` to avoid duplicate requests sending for nodes that finish query. Also there is required to use shared lock for onError.  

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return Head of stream and then clean it.
+     */
+    public R get() {
+        R ret = head;
+
+        head = null;
+
+        return ret;
+    }

Review comment:
       This method helps simplify NodePageStream code. Sequence of steps is:
   1. hasNext() to check stream queue;
   2. next() to fill `head`;
   3. head() to use `head` for comparison;
   4. get() to return value and clean `head`.
   
   If we replace get() with next() then next() should also invoke hasNext() method, also it's requires to store info about last hasNext() invocation from next() to avoid double invokation of hasNext() (steps 1, 4).
   
   So I suggest to stay with this separate method, as it pretty clean.
   
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -786,40 +720,25 @@ private Object convert(Object obj) {
 
         long reqId = cctx.io().nextIoId();
 
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
+        final GridCacheDistributedFieldsQueryFuture fut = new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry);
+
+        initDistributedQuery(reqId, fut, nodes);
+
+        return fut;
+    }
 
+    /** Initialize distributed query: stores future, sends query requests to nodes. */
+    private void initDistributedQuery(long reqId, GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
         try {
-            qry.query().validate();
+            DistributedCacheQueryReducer reducer = createReducer(fut.qry.query().type(), reqId, fut, nodes);
 
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
+            fut.reducer(reducer);
 
-            addQueryFuture(req.id(), fut);
+            fut.qry.query().validate();
 
-            final Object topic = topic(cctx.nodeId(), req.id());
+            addQueryFuture(reqId, fut);

Review comment:
       Please check, I've done it, but actually code doesn't look clearer I think.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682484307



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)
+            );
+
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head(), o2.head());
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId());
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId());
+                streams[i] = null;
+                streamOff++;
+            }
+        }
+
+        if (streamOff == streams.length)
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(UUID nodeId, Collection<R> data, boolean last) {
+        NodePageStream<R> stream = streamsMap.get(nodeId);
+
+        if (stream == null)
+            return streamsMap.isEmpty();
+
+        stream.addPage(nodeId, data, last);
+
+        return last && (streamsMap.remove(nodeId) != null) && streamsMap.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.onError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCancel() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.cancel(ns -> pageRequester.cancelQueryRequest(reqId, ns, fut.fields()));
+
+        streamsMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean mapNode(UUID nodeId) {
+        return streamsMap.containsKey(nodeId);
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param cmp Comparator.
+     */
+    private static <Z> void bubbleUp(Z[] arr, int off, Comparator<Z> cmp) {

Review comment:
       Why don't use logN algorithm here?
   E.g. use MinHeap instead.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)
+            );
+
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head(), o2.head());
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId());
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId());
+                streams[i] = null;
+                streamOff++;
+            }
+        }
+
+        if (streamOff == streams.length)
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(UUID nodeId, Collection<R> data, boolean last) {
+        NodePageStream<R> stream = streamsMap.get(nodeId);
+
+        if (stream == null)
+            return streamsMap.isEmpty();
+
+        stream.addPage(nodeId, data, last);
+
+        return last && (streamsMap.remove(nodeId) != null) && streamsMap.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.onError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCancel() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.cancel(ns -> pageRequester.cancelQueryRequest(reqId, ns, fut.fields()));
+
+        streamsMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean mapNode(UUID nodeId) {
+        return streamsMap.containsKey(nodeId);
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param cmp Comparator.
+     */
+    private static <Z> void bubbleUp(Z[] arr, int off, Comparator<Z> cmp) {

Review comment:
       Why don't use logN algorithm here?
   E.g. use MinHeap structure instead.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682501375



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       Comparator instance can be static.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r705177929



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/AbstractDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.DistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Abstract class for distributed reducer implementations.
+ */
+abstract class AbstractDistributedCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /** Collection of streams that don't deliver all pages yet. */
+    private final Map<UUID, NodePageStream<R>> remoteStreams;

Review comment:
       Fix issue with `onError`, also I make `rcvdFirstPage` ordinary HashSet, as we already have a guard on this var.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/AbstractDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.DistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Abstract class for distributed reducer implementations.
+ */
+abstract class AbstractDistributedCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /** Collection of streams that don't deliver all pages yet. */
+    private final Map<UUID, NodePageStream<R>> remoteStreams;

Review comment:
       Fixed issue with `onError`, also I make `rcvdFirstPage` ordinary HashSet, as we already have a guard on this var.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642618133



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;

Review comment:
       Next pages are requested asynchronously in moment after preparing a head for a stream and finding that queue is empty. We compare heads and bubble them on every user's `next()`. So for multiple streams (1 stream per node), for even-distributed between nodes data we will get head from new stream for every next() call. Then we can expect that page loading is non-blocking in most cases.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r648223801



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;
+
+    /**
+     * Callback to handle node left.
+     *
+     * @param nodeId Node ID that left a cluster.
+     * @return {@code true} if specified node runs this query.

Review comment:
       Renamed this method and made docs more clear. Now it just checks whether node is running a query in this moment. `onNodeLeft` now handles with `onError` method.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726940584



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -583,17 +518,25 @@ else if (!cancelled.contains(res.requestId()))
                 }
             });
 
-            sendRequest(fut, req, nodes);
+            futs.put(reqId, fut);
+
+            if (cctx.kernalContext().clientDisconnected()) {
+                IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(
+                    cctx.kernalContext().cluster().clientReconnectFuture(),
+                    "Query was cancelled, client node disconnected.");
+
+                fut.onError(err);
+            }

Review comment:
       There are additional actions in `fut.onError`, for example clearing timeout object. So we should invoke `onError` directly.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r695568976



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -101,12 +104,30 @@
     /** Event listener. */
     private GridLocalEventListener lsnr;
 
+    /** Requester of cache query result pages. */
+    private CacheQueryPageRequester pageRequester;
+
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() != LOCAL;
 
+        pageRequester = new CacheQueryPageRequester(cctx) {
+            /** {@inheritDoc} */
+            @Override protected void sendLocal(GridCacheQueryRequest req) {

Review comment:
       made sendLocal as Consumer param.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r725847348



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Base class for distributed cache query reducers.
+ */
+public abstract class DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Page streams collection. */
+    protected final Map<UUID, NodePageStream<T>> pageStreams;
+
+    /** Timestamp of query timeout. */
+    protected final long endTime;

Review comment:
       There is a different approach for timeout handing. There is a handler in the future `GridCacheQueryFutureAdapter#onTimeout`. The timeout should not be handled from a user thread (`nextPage`), to clean remote resources faster - `onTimeout` cancels query. Also this handler completes cache query future, and user thread just checks this status in next iteration.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r714037531



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.NoSuchElementException;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /**
+     * Queue of pages from all nodes. Order of streams is set with {@link #pageCmp}.
+     */
+    private final PriorityQueue<NodePage<R>> nodePages;
+
+    /**
+     * Compares head pages from all nodes to get the lowest value at the moment.
+     */
+    private final Comparator<NodePage<R>> pageCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester, queueLock, nodes);
+
+        pageCmp = (o1, o2) -> rowCmp.compare(o1.head(), o2.head());
+
+        nodePages = new PriorityQueue<>(nodes.size(), pageCmp);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        // Initial sort.
+        if (nodePages.isEmpty() && !streams.isEmpty()) {
+            Set<UUID> nodes = new HashSet<>(streams.keySet());
+
+            for (UUID nodeId : nodes)
+                fillWithPage(nodeId);
+        }
+
+        return !nodePages.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        if (nodePages.isEmpty())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        NodePage<R> page = nodePages.poll();
+
+        R o = page.next();
+
+        if (page.hasNext())
+            nodePages.offer(page);
+        else
+            fillWithPage(page.nodeId());
+
+        return o;
+    }
+
+    /** */
+    private void fillWithPage(UUID nodeId) throws IgniteCheckedException {
+        NodePage<R> page = streams.get(nodeId).nextPage();
+
+        if (!page.hasNext())
+            streams.remove(nodeId);
+        else
+            nodePages.offer(page);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        super.cancel();
+
+        nodePages.clear();

Review comment:
       Hi, this is OK. If a query is cancelled it means that cache query future is already done and we don't need more pages. Cancellation is invoked whether query is finished with timeout, if user closes cursor, if iterator reaches user specified limit, or with any exception that stops this future. So no need to wait those pages.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r685260622



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -101,12 +104,30 @@
     /** Event listener. */
     private GridLocalEventListener lsnr;
 
+    /** Requester of cache query result pages. */
+    private CacheQueryPageRequester pageRequester;
+
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() != LOCAL;
 
+        pageRequester = new CacheQueryPageRequester(cctx) {
+            /** {@inheritDoc} */
+            @Override protected void sendLocal(GridCacheQueryRequest req) {

Review comment:
       I've tried to refactor it, but actually failed to make it better. 
   
   There are 3 events to send a request to other nodes: initial load, ask for next page after finish previous, cancel query. Initiators of those events are independent on each other and the only common place is `reducer`, that is triggered and  performs requests. 
   
   PageRequester's `sendRequest()` also invokes `sendLocal()` that depends much on variables defined in `GridCacheDistributedQueryManager`. The initiators doesn't have dependency on this manager. Then it's required to accumulate all logic of sending request and local handling in single object (`pageRequester`).
   
   Alternatives is make initiators depend on `GridCacheDistributedQueryManager`, that makes code more spaghetti. Split requester on request factory and request sender also doesn't make code clearer. 
   
   I think current solution is OK, it clear enough. The only bad place is `sendLocal` that depends on the manager variables. I think we should not refactor this manager now, as it is out of scope of this ticket. WDYT?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642428288



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {

Review comment:
       Why we need any synchronization here?
   No statements have any side effects.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r633556166



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -169,103 +97,30 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    @Override protected boolean onPage(@Nullable UUID nodeId, boolean last) {
+        return reducer.onPage(nodeId, last);
     }
 
     /** {@inheritDoc} */
     @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+        reducer.loadAll();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override protected Reducer<R> reducer() {
+        return reducer;
     }
 
     /** {@inheritDoc} */
     @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         return super.onCancelled();
     }
 
     /** {@inheritDoc} */
     @Override public void onTimeout() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         super.onTimeout();
     }

Review comment:
       Do we need all these callbacks in QueryFutureAdapter just to delegate calls to reducer
   and having reducer() at same time?
   I see a pattern you already use: reducer().onPage() and similar.
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream(node.id());
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {

Review comment:
       You use the same lock to synchronize access to streams[] and all queues in all NodeStrems objects.
   What is the purpose to use a shared lock?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {

Review comment:
       MergeSorted reducer uses per-node Stream, while Unsorted uses single shared Stream and they have different logic of merging results.
   Why do you extend Unsorted reducer instead of Abstract one here?
   Some common fields can be safely pulled up into the Abstract reduceer.
   
   Also, I'd move all these classes hierarchy to the package  'reducer' and made PageStream/NodeStream to top-level classes, with the package visibility. 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
+        Collection<ClusterNode> nodes;
+
+        synchronized (queueLock()) {
+            nodes = F.retain(allNodes, true,
+                new P1<ClusterNode>() {
+                    @Override public boolean apply(ClusterNode node) {
+                        return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
+                    }
+                }
+            );
+
+            rcvd.clear();
+            subgrid.clear();
+        }
+
+        pageRequester.cancelQueryRequest(reqId, nodes, fut.fields());
+
+        pageStream.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            // Loads only queue is empty to avoid memory consumption on additional pages.
+            if (!pageStream.queue.isEmpty())
+                return;
+
+            if (loadAllowed && rcvd.containsAll(subgrid)) {
+                rcvd.clear();
+
+                nodes = new ArrayList<>(subgrid);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadAll() throws IgniteInterruptedCheckedException {
+        assert !Thread.holdsLock(queueLock());
+
+        U.await(firstPageLatch);
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            if (loadAllowed && !subgrid.isEmpty())
+                nodes = new ArrayList<>(subgrid);
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {

Review comment:
       onPage with `boolean last` and onLastPage() looks confusing.
   Looks like an architectural lack.
   What is the difference between these methods?  

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {

Review comment:
       Let's have separate classes/interfaces hierarchy for local and distributed queries.
   Because they have different contracts for similar methods. E.g. onLastPage() and onPage(nodeId, true);
   Having both methods looks ambiguous.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;
+
+    /**
+     * Callback to handle node left.
+     *
+     * @param nodeId Node ID that left a cluster.
+     * @return {@code true} if specified node runs this query.
+     */
+    public boolean onNodeLeft(UUID nodeId);
+
+    /** Blocks while reducer doesn't get first result item for this query. */
+    public void awaitFirstItem() throws InterruptedException;

Review comment:
       ```suggestion
       /** 
       * Blocks current thread until reducer will be ready to return the very first result item for the query. 
       */
       public void awaitInitialization() throws InterruptedException;
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;
+
+    /**
+     * Callback to handle node left.
+     *
+     * @param nodeId Node ID that left a cluster.
+     * @return {@code true} if specified node runs this query.

Review comment:
       It is unclear, what result will be returned if the node has gone after the last page was received and how it should be interpreted.
   Or what if the node has gone, but we already received sufficient data for finishing the query, 
   e.g. query has limits, but the user app consumes data slowly. 
   
   I think Reducer should only check internal invariants and mark node as failed.
   And caller should recheck 'if the reducer is alive' right after calling this method.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.

Review comment:
       It looks like the method is called when we got a new page and notify the Reducer.
   What the purpose to get a flag to the caller back?
   Why reducer can't apply additional logic by itself or call a listener (like in true OOP) instead of return magic value.
   
   What the purpose of the `last` parameter? What does `false` value mean and what is the impact to return value?
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;

Review comment:
       Ok. One can forcibly load all the pages, but where will the data be returned or saved?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;
+
+    /**
+     * Callback to handle node left.
+     *
+     * @param nodeId Node ID that left a cluster.
+     * @return {@code true} if specified node runs this query.
+     */
+    public boolean onNodeLeft(UUID nodeId);
+
+    /** Blocks while reducer doesn't get first result item for this query. */
+    public void awaitFirstItem() throws InterruptedException;
+
+    /** Callback that invokes when this query is cancelled. */
+    public void cancel();

Review comment:
       ```suggestion
       /** 
       * Callback is invoked on the query cancellation. 
       */
       public void onCancel();
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #addPage(UUID, Collection)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Offer query result page for reduce. Note that the data collection may contain extension of type T.
+     * In such cases it stores additional payload for custom reducer logic.
+     *
+     * @param nodeId Node ID that sent this page. {@code null} means local node or error page.
+     * @param data Page data rows.
+     */
+    public void addPage(@Nullable UUID nodeId, Collection<T> data);
+
+    /**
+     * Callback that invokes after reducer get last query result page.
+     * Also invokes for failed queries to let reducer know that there won't be new pages.

Review comment:
       Why there is no onCancel() or onError() callbacks that will cause Exception throwing in `next` method?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {
+            if (!loadAllowed)
+                return;
+
+            nodes = new ArrayList<>();
+
+            for (int i = streamOff; i < streams.length; i++) {
+                UUID nodeId = streams[i].nodeId;
+
+                // Try to contain 2 pages for every stream to avoid waits. A node has to be present in collections:
+                // 1. rcvd - as requested and already handled before, so currently no parallel request to this node.
+                // 2. subgrid - as this node still has pages to request.
+                if (streams[i].queue.size() < 1 && rcvd.remove(nodeId) && subgrid.contains(nodeId))
+                    nodes.add(nodeId);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        boolean qryLast = super.onPage(nodeId, last);
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        if (last && streamsMap.containsKey(nodeId)) {
+            streamsMap.get(nodeId).allPagesReady = true;
+
+            streamsMap.remove(nodeId);
+        }
+
+        return qryLast;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        assert Thread.holdsLock(queueLock());
+
+        // Local node.
+        if (nodeId == null) {
+            nodeId = cctx.localNodeId();
+
+            // If nodeId is NULL and query doesn't execute on local node, then it is error, notify all streams.
+            if (!streamsMap.containsKey(nodeId)) {
+                assert data.isEmpty();
+
+                for (PageStream stream: streamsMap.values())
+                    stream.addPage(data);

Review comment:
       So, why we have a magic constant NULL instead of a separate callback for this with error as a param?
   It is totally unclear, why UnsortedReducer doesn't have a similar null check...
   
   Also, passing an empty page looks like a magic constant as well.
   Do we need to notify other streams? Do we know at this point, that prefetched result (failed) will ever be needed? 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
+        Collection<ClusterNode> nodes;
+
+        synchronized (queueLock()) {
+            nodes = F.retain(allNodes, true,
+                new P1<ClusterNode>() {
+                    @Override public boolean apply(ClusterNode node) {
+                        return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
+                    }
+                }
+            );
+
+            rcvd.clear();
+            subgrid.clear();
+        }
+
+        pageRequester.cancelQueryRequest(reqId, nodes, fut.fields());
+
+        pageStream.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            // Loads only queue is empty to avoid memory consumption on additional pages.
+            if (!pageStream.queue.isEmpty())
+                return;
+
+            if (loadAllowed && rcvd.containsAll(subgrid)) {
+                rcvd.clear();
+
+                nodes = new ArrayList<>(subgrid);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadAll() throws IgniteInterruptedCheckedException {
+        assert !Thread.holdsLock(queueLock());
+
+        U.await(firstPageLatch);
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            if (loadAllowed && !subgrid.isEmpty())
+                nodes = new ArrayList<>(subgrid);
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        assert Thread.holdsLock(queueLock());
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        rcvd.add(nodeId);
+
+        if (!loadAllowed && rcvd.containsAll(subgrid) ) {
+            firstPageLatch.countDown();
+            loadAllowed = true;
+        }
+
+        return last && subgrid.remove(nodeId) && subgrid.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onNodeLeft(UUID nodeId) {
+        synchronized (queueLock()) {
+            return subgrid.contains(nodeId);

Review comment:
       You can have 2 separate calls instead.
   fut.onNodeLeft -> rdc.onNodeLeft 
   fut.onNodeLeft -> rdc.isFailed/isFinished

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;

Review comment:
       After you got the last item from the first page, you will be blocked on `next()` call until the second page will be received from the node, otherwise you will not be able to resort streams correctly.
   You need to have at least one page from every node to be ready at any time for non-blocking reducing.
   
   I wouldn't bet on equal data distribution. Assume, one node will have the largest part of the query result,
   then you will end up in synchronous ping-pong with a single node.
   
   Effective prefetching is to request next page right after you start processing the previous one.
   If you bother about fetching few additional pages, then you can configure twice smaller page, aren't you?




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r693800036



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered via callback
+ * {@link #onPage(UUID, Collection, boolean)}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Callback that invoked on receiving a new page.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.
+     */
+    public boolean onPage(UUID nodeId, Collection<T> data, boolean last);
+
+    /**
+     * Callback in case of page with error.
+     */
+    public void onError();
+
+    /**
+     * Callback that invokes after a query future is done.
+     */
+    public void onFinish();

Review comment:
       Is it possible to release the latch in onError() and onPage() method?
   This will reduce `future` and `reducer` coupling.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655450023



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -523,57 +544,31 @@ else if (!cancelled.contains(res.requestId()))
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
     @Override public CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, false);
+    }
+
+    /** */
+    private CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes, boolean fields) {
         assert cctx.config().getCacheMode() != LOCAL;
 
         if (log.isDebugEnabled())
             log.debug("Executing distributed query: " + qry);
 
         long reqId = cctx.io().nextIoId();
 
-        final GridCacheDistributedQueryFuture<K, V, ?> fut =
-            new GridCacheDistributedQueryFuture<>(cctx, reqId, qry, nodes);
+        final GridCacheDistributedQueryFuture fut = fields ?
+            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry)
+            : new GridCacheDistributedQueryFuture(cctx, reqId, qry);

Review comment:
       ```suggestion
               new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry) :
               new GridCacheDistributedQueryFuture(cctx, reqId, qry);
   ```




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642338609



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/AbstractCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Base abstract class for all Reducer descenders. It declares dependencies on cache query, and base logic for handling
+ * remote pages.
+ */
+public abstract class AbstractCacheQueryReducer<R> implements CacheQueryReducer<R> {
+    /** Query info. */
+    private final GridCacheQueryAdapter qry;
+
+    /** Flag shows whether all pages are ready. */
+    protected volatile boolean allPagesReady;
+
+    /** Timestamp when a query fails with timeout. */
+    private final long timeoutTime;
+
+    /** Lock shared between this reducer and future. */
+    private final Object sharedLock;

Review comment:
       ```suggestion
       private final Object queueLock;
   ```




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645124950



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {

Review comment:
       Let's have separate classes/interfaces hierarchy for local and distributed queries.
   Because they have different contracts for similar methods. E.g. onLastPage() and onPage(nodeId, true);
   Having both methods looks ambiguous.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645119761



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
+        Collection<ClusterNode> nodes;
+
+        synchronized (queueLock()) {
+            nodes = F.retain(allNodes, true,
+                new P1<ClusterNode>() {
+                    @Override public boolean apply(ClusterNode node) {
+                        return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
+                    }
+                }
+            );
+
+            rcvd.clear();
+            subgrid.clear();
+        }
+
+        pageRequester.cancelQueryRequest(reqId, nodes, fut.fields());
+
+        pageStream.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            // Loads only queue is empty to avoid memory consumption on additional pages.
+            if (!pageStream.queue.isEmpty())
+                return;
+
+            if (loadAllowed && rcvd.containsAll(subgrid)) {
+                rcvd.clear();
+
+                nodes = new ArrayList<>(subgrid);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadAll() throws IgniteInterruptedCheckedException {
+        assert !Thread.holdsLock(queueLock());
+
+        U.await(firstPageLatch);
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            if (loadAllowed && !subgrid.isEmpty())
+                nodes = new ArrayList<>(subgrid);
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {

Review comment:
       onPage with `boolean last` and onLastPage() looks confusing.
   Looks like an architectural lack.
   What is the difference between these methods?  




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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r717459702



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -69,205 +53,66 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
         GridCacheQueryManager<K, V> mgr = ctx.queries();
 
         assert mgr != null;
-
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
-
-        assert qryMgr != null;
-
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
-
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
+    @Override protected void cancelQuery() {
+        reducer.cancel();
 
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
-
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
     @Override protected void onNodeLeft(UUID nodeId) {
-        boolean callOnPage;
-
-        synchronized (this) {
-            callOnPage = !loc && subgrid.contains(nodeId);
-        }
+        boolean qryNode = reducer.remoteQueryNode(nodeId);
 
-        if (callOnPage)
-            onPage(nodeId, Collections.emptyList(),
+        if (qryNode)

Review comment:
       Missing braces.
   
   https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-BracesandIndentation

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/AbstractDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.DistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Abstract class for distributed reducer implementations.
+ */
+abstract class AbstractDistributedCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /** Collection of streams that don't deliver all pages yet. */
+    private final Map<UUID, NodePageStream<R>> remoteStreams;

Review comment:
       Maxim, I agree with Andrey. Your solution will work, however, I think it's better to merge these collections for code simplicity. We can easily filter the closed streams by some kind of `close` flag (since you name it as a `streams` probably they should implement a `Closable` interface too).

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -776,135 +721,24 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                null,
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,

Review comment:
       Can you fix the raw type usage?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class provides an interface {@link #nextPage()} that returns a {@link NodePage} of cache query result
+ * from single node. Pages are stored in a queue, after polling a queue it tries to load a new page.
+ */
+class NodePageStream<R> {
+    /** Queue of data of results pages. */
+    private final Queue<NodePage<R>> queue = new LinkedList<>();
+
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new

Review comment:
       It seems the JavaDoc doesn't correspond to the class param and should be fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {

Review comment:
       Let's move all the methods of this class to the `GridCacheDistributedQueryManager`? It seems there is no reason to share them through the dedicated class.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Local handler of cache query request. */
+    private final Consumer<GridCacheQueryRequest> sendLoc;
+
+    /**
+     * @param cctx Cache context.
+     * @param sendLoc Local handler of cache query request.
+     */
+    CacheQueryPageRequester(GridCacheContext<?, ?> cctx, Consumer<GridCacheQueryRequest> sendLoc) {
+        this.cctx = cctx;
+        this.sendLoc = sendLoc;
+
+        log = cctx.kernalContext().config().getGridLogger();
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture<?, ?, ?> fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter<?> qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.idxQryDesc(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        List<UUID> sendNodes = new ArrayList<>();
+
+        for (ClusterNode n: nodes)
+            sendNodes.add(n.id());
+
+        sendRequest(fut, req, sendNodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter<?, ?, ?> fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter<?> qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            sendRequest(fut, req, nodes);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Send cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQuery(long reqId, Collection<UUID> nodes, boolean fieldsQry) {
+        GridCacheQueryManager<?, ?> qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            sendRequest(null, req, nodes);
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param fut Cache query future. {@code null} in case of cancel request.
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        UUID locNodeId = cctx.localNodeId();
+
+        boolean loc = false;
+
+        for (UUID nodeId : nodes) {
+            if (nodeId.equals(locNodeId))
+                loc = true;
+            else {
+                if (req.cancel())
+                    sendNodeCancelRequest(nodeId, req);
+                else if (!sendNodePageRequest(nodeId, req, fut))
+                    return;
+            }
+        }
+
+        if (loc)
+            sendLoc.accept(req);
+    }
+
+    /** */
+    private void sendNodeCancelRequest(UUID nodeId, GridCacheQueryRequest req) throws IgniteCheckedException {
+        try {
+            cctx.io().send(nodeId, req, GridIoPolicy.QUERY_POOL);
+        }
+        catch (IgniteCheckedException e) {
+            if (cctx.io().checkNodeLeft(nodeId, e, false)) {

Review comment:
       It seems this is a legacy check, however, it can be removed also, since the DiscoveryEventListener already exists for the futures handling class.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r718596423



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {

Review comment:
       Done.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r660482719



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #onPage(UUID, Collection, boolean)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.

Review comment:
       Actually, shared lock now is part of PageStream only. So we can remove this mention.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726944317



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -17,257 +17,172 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.util.lang.GridPlainCallable;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.processors.cache.query.reducer.MergeSortDistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.reducer.NodePageStream;
+import org.apache.ignite.internal.processors.cache.query.reducer.UnsortedDistributedCacheQueryReducer;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
+
 /**
  * Distributed query future.
  */
 public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutureAdapter<K, V, R> {
     /** */
-    private static final long serialVersionUID = 0L;
+    private final long reqId;
 
-    /** */
-    private long reqId;
+    /** Helps to send cache query requests to other nodes. */
+    private final GridCacheDistributedQueryManager<K, V> qryMgr;
 
-    /** */
-    private final Collection<UUID> subgrid = new HashSet<>();
+    /** Collection of streams. */
+    private final Map<UUID, NodePageStream<R>> streams;
 
-    /** */
-    private final Collection<UUID> rcvd = new HashSet<>();
+    /** Count of streams that finish receiving remote pages. */
+    private final AtomicInteger noRemotePagesStreamsCnt = new AtomicInteger();
 
-    /** */
-    private CountDownLatch firstPageLatch = new CountDownLatch(1);
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Set of nodes that deliver their first page. */
+    private Set<UUID> rcvdFirstPage = ConcurrentHashMap.newKeySet();
 
     /**
      * @param ctx Cache context.
      * @param reqId Request ID.
      * @param qry Query.
-     * @param nodes Nodes.
      */
-    protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId, GridCacheQueryBean qry,
-        Iterable<ClusterNode> nodes) {
+    protected GridCacheDistributedQueryFuture(
+        GridCacheContext<K, V> ctx,
+        long reqId,
+        GridCacheQueryBean qry,
+        Collection<ClusterNode> nodes
+    ) {
         super(ctx, qry, false);
 
         assert reqId > 0;
 
         this.reqId = reqId;
 
-        GridCacheQueryManager<K, V> mgr = ctx.queries();
+        qryMgr = (GridCacheDistributedQueryManager<K, V>) ctx.queries();
 
-        assert mgr != null;
+        streams = new ConcurrentHashMap<>(nodes.size());
 
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
-    }
+        for (ClusterNode node : nodes) {
+            NodePageStream<R> s = new NodePageStream<>(node.id(), () -> requestPages(node.id()), () -> cancelPages(node.id()));
 
-    /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
+            streams.put(node.id(), s);
 
-        assert qryMgr != null;
+            startQuery(node.id());
+        }
 
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
+        Map<UUID, NodePageStream<R>> streamsMap = Collections.unmodifiableMap(streams);
 
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
+        reducer = qry.query().type() == TEXT ?
+            new MergeSortDistributedCacheQueryReducer<>(streamsMap)
+            : new UnsortedDistributedCacheQueryReducer<>(streamsMap);
+    }
 
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
+    /** {@inheritDoc} */
+    @Override protected void cancelQuery(Throwable err) {
+        firstPageLatch.countDown();
 
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
+        for (NodePageStream<R> s : streams.values())
+            s.cancel(err);
 
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
     @Override protected void onNodeLeft(UUID nodeId) {

Review comment:
       We should not mark future with error if node left after delivering all pages, as it's save to ignore this. But we can invoke `onError` within if block directly. 




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645795856



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream(node.id());
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {

Review comment:
       In master branch (single stream) one of purposes of the lock is to guard consistency of `queue` and collections `rcvd`, `subgrid`. As far as I undertstand it's required to avoid excess requests to nodes that already finish query. It could lead to data duplication in final result, as query runs from beginning on such node. Actually as I can see it should be an extremely rare case, but it's possible. Also I don't find any other valuable reasons for this lock.
   
   In implementation with multiple streams we still have shared between streams `rdc` and `subgrid` variables, that has to be in consisten state with all stream queues. So this is a reason to use shared lock 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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r660482719



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #onPage(UUID, Collection, boolean)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.

Review comment:
       Actually, shared lock now is part of PageStream only. So we can remove this mention.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682497615



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -101,12 +104,30 @@
     /** Event listener. */
     private GridLocalEventListener lsnr;
 
+    /** Requester of cache query result pages. */
+    private CacheQueryPageRequester pageRequester;
+
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() != LOCAL;
 
+        pageRequester = new CacheQueryPageRequester(cctx) {
+            /** {@inheritDoc} */
+            @Override protected void sendLocal(GridCacheQueryRequest req) {

Review comment:
       This looks ambiguous.
   Why don't have just a factory for the requests instead of Requester?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r633556166



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -169,103 +97,30 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    @Override protected boolean onPage(@Nullable UUID nodeId, boolean last) {
+        return reducer.onPage(nodeId, last);
     }
 
     /** {@inheritDoc} */
     @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+        reducer.loadAll();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override protected Reducer<R> reducer() {
+        return reducer;
     }
 
     /** {@inheritDoc} */
     @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         return super.onCancelled();
     }
 
     /** {@inheritDoc} */
     @Override public void onTimeout() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         super.onTimeout();
     }

Review comment:
       Do we need all these callbacks in QueryFutureAdapter just to delegate calls to reducer
   and having reducer() at same time?
   I see a pattern you already use: reducer().onPage() and similar.
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645139164



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.

Review comment:
       It looks like the method is called when we got a new page and notify the Reducer.
   What the purpose to get a flag to the caller back?
   Why reducer can't apply additional logic by itself or call a listener (like in true OOP) instead of return magic value.
   
   What the purpose of the `last` parameter? What does `false` value mean and what is the impact to return value?
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r693804731



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -101,12 +104,30 @@
     /** Event listener. */
     private GridLocalEventListener lsnr;
 
+    /** Requester of cache query result pages. */
+    private CacheQueryPageRequester pageRequester;
+
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() != LOCAL;
 
+        pageRequester = new CacheQueryPageRequester(cctx) {
+            /** {@inheritDoc} */
+            @Override protected void sendLocal(GridCacheQueryRequest req) {

Review comment:
       You can pass a Consumer<Request> instead of creating anonymous class for now or 
   even revert back message processing logic and create a ticket for improvement.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r633556166



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -169,103 +97,30 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    @Override protected boolean onPage(@Nullable UUID nodeId, boolean last) {
+        return reducer.onPage(nodeId, last);
     }
 
     /** {@inheritDoc} */
     @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+        reducer.loadAll();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override protected Reducer<R> reducer() {
+        return reducer;
     }
 
     /** {@inheritDoc} */
     @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         return super.onCancelled();
     }
 
     /** {@inheritDoc} */
     @Override public void onTimeout() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         super.onTimeout();
     }

Review comment:
       Do we need all these callbacks in QueryFutureAdapter just to delegate calls to reducer
   and having reducer() at same time?
   I see a patter you alresy use: reducer().onPage() and similar.
   




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642587837



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {
+            if (!loadAllowed)
+                return;
+
+            nodes = new ArrayList<>();
+
+            for (int i = streamOff; i < streams.length; i++) {
+                UUID nodeId = streams[i].nodeId;
+
+                // Try to contain 2 pages for every stream to avoid waits. A node has to be present in collections:
+                // 1. rcvd - as requested and already handled before, so currently no parallel request to this node.
+                // 2. subgrid - as this node still has pages to request.
+                if (streams[i].queue.size() < 1 && rcvd.remove(nodeId) && subgrid.contains(nodeId))
+                    nodes.add(nodeId);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        boolean qryLast = super.onPage(nodeId, last);
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        if (last && streamsMap.containsKey(nodeId)) {
+            streamsMap.get(nodeId).allPagesReady = true;
+
+            streamsMap.remove(nodeId);
+        }
+
+        return qryLast;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        assert Thread.holdsLock(queueLock());
+
+        // Local node.
+        if (nodeId == null) {
+            nodeId = cctx.localNodeId();
+
+            // If nodeId is NULL and query doesn't execute on local node, then it is error, notify all streams.
+            if (!streamsMap.containsKey(nodeId)) {
+                assert data.isEmpty();
+
+                for (PageStream stream: streamsMap.values())
+                    stream.addPage(data);

Review comment:
       In this case we know that nodeId set to NULL only in case of error, that data is empty. So it's safe to put data. Note that we put this to all nodes to faster finish stream traverse to faster check future for error.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r667813983



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -423,48 +372,35 @@ private R unmaskNull(R obj) {
         return obj != NULL ? obj : null;
     }
 
-    /** {@inheritDoc} */
+    /** Forces to stop query, don't care about result.  */

Review comment:
       Why waiting on future causes query stop?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -423,48 +372,35 @@ private R unmaskNull(R obj) {
         return obj != NULL ? obj : null;
     }
 
-    /** {@inheritDoc} */
+    /** Forces to stop query, don't care about result.  */

Review comment:
       Why waiting on a future causes query to stop?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#issuecomment-895308234


   @korlov42 @AMashenkov  thanks for review. I pushed an update and answer on some your comments. Could you please have a look again. The re-request review button works only for Andrey, and not for Konstantin. Don't know why :(


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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682456160



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered via callback
+ * {@link #onPage(UUID, Collection, boolean)}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Callback that invoked on receiving a new page.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.
+     */
+    public boolean onPage(UUID nodeId, Collection<T> data, boolean last);
+
+    /**
+     * Callback in case of page with error.
+     */
+    public void onError();
+
+    /**
+     * Callback that invokes after a query future is done.
+     */
+    public void onFinish();

Review comment:
       Why do we need this method if we already have onError and onPage with 'last' parameter?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642930254



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {
+            if (!loadAllowed)
+                return;
+
+            nodes = new ArrayList<>();
+
+            for (int i = streamOff; i < streams.length; i++) {
+                UUID nodeId = streams[i].nodeId;
+
+                // Try to contain 2 pages for every stream to avoid waits. A node has to be present in collections:
+                // 1. rcvd - as requested and already handled before, so currently no parallel request to this node.
+                // 2. subgrid - as this node still has pages to request.
+                if (streams[i].queue.size() < 1 && rcvd.remove(nodeId) && subgrid.contains(nodeId))
+                    nodes.add(nodeId);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        boolean qryLast = super.onPage(nodeId, last);
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        if (last && streamsMap.containsKey(nodeId)) {
+            streamsMap.get(nodeId).allPagesReady = true;
+
+            streamsMap.remove(nodeId);
+        }
+
+        return qryLast;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        assert Thread.holdsLock(queueLock());

Review comment:
       You're right. We need to check streamsMap.get() for NPE in case of concurrent `onProcessQueryResponse()` and `onNodeLeft()`. Guarding with lock doesn't help us in this case. 




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642337612



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public abstract class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** */
+    CacheQueryPageRequester(final GridCacheContext cctx) {
+        this.cctx = cctx;
+        this.log = cctx.kernalContext().config().getGridLogger();
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.subjectId(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        sendRequest(fut, req, nodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.subjectId(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            Collection<ClusterNode> n = new ArrayList<>();
+            for (UUID id: nodes)
+                n.add(cctx.node(id));
+
+            sendRequest(fut, req, n);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQueryRequest(long reqId, Collection<ClusterNode> nodes, boolean fieldsQry) {
+        final GridCacheQueryManager qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            // Process cancel query directly (without sending) for local node.
+            sendLocal(req);
+
+            if (!nodes.isEmpty()) {
+                for (ClusterNode node : nodes) {
+                    try {
+                        cctx.io().send(node, req, cctx.ioPolicy());
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to send cancel request, node failed: " + node);
+                        }
+                        else
+                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
+                    }
+                }
+            }
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        final GridCacheQueryFutureAdapter fut,
+        final GridCacheQueryRequest req,
+        Collection<ClusterNode> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        final UUID locNodeId = cctx.localNodeId();
+
+        ClusterNode locNode = null;
+
+        Collection<ClusterNode> rmtNodes = null;
+
+        for (ClusterNode n : nodes) {
+            if (n.id().equals(locNodeId))
+                locNode = n;
+            else {
+                if (rmtNodes == null)
+                    rmtNodes = new ArrayList<>(nodes.size());
+
+                rmtNodes.add(n);
+            }
+        }
+
+        // Request should be sent to remote nodes before the query is processed on the local node.
+        // For example, a remote reducer has a state, we should not serialize and then send
+        // the reducer changed by the local node.
+        if (!F.isEmpty(rmtNodes)) {
+            for (ClusterNode node : rmtNodes) {
+                try {
+                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
+                }
+                catch (IgniteCheckedException e) {
+                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
+                        fut.onNodeLeft(node.id());
+
+                        if (fut.isDone())
+                            return;
+                    }
+                    else
+                        throw e;
+                }
+            }
+        }
+
+        if (locNode != null)
+            sendLocal(req);
+    }
+
+    /** Send and handle request to local node. */

Review comment:
       ```suggestion
       /** 
       * Handles request locally. 
       * @param req Query request.
       */
   ```




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r649227253



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Requests full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void requestFullPages() throws IgniteInterruptedCheckedException;
+
+    /**
+     * Checks whether cache query still runs on specified node. If a query finished (send all pages) on this node, then
+     * this method has to return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if specified node runs this query.
+     */
+    public boolean queryNode(UUID nodeId);

Review comment:
       ```suggestion
       public boolean mapNode(UUID nodeId);
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/PageStream.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Iterator over pages stream. Pages are stored in a queue. After polling a queue try load a new page instead of it.
+ */
+class PageStream<R> {
+    /** Queue of data of results pages. */
+    protected final Queue<Collection<R>> queue = new LinkedList<>();
+
+    /** Iterator over current page. */
+    private Iterator<R> iter;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    private final Collection<UUID> subgrid;
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    private final Collection<UUID> rcvd = new HashSet<>();

Review comment:
       This class has a child NodeStream, which is per-node stream.
   What semantic 'subgrid', 'queue' and 'rcvd' will have in a child class?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -324,8 +242,8 @@ public void onPage(@Nullable UUID nodeId, @Nullable Collection<?> data, @Nullabl
 
         try {
             if (err != null)
-                synchronized (this) {
-                    enqueue(Collections.emptyList());
+                synchronized (lock) {
+                    reducer().onError();

Review comment:
       Seems, synchronization no longer needed here.
   Reducer should care about synchronization in onError() by itself if needed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -786,40 +720,25 @@ private Object convert(Object obj) {
 
         long reqId = cctx.io().nextIoId();
 
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
+        final GridCacheDistributedFieldsQueryFuture fut = new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry);
+
+        initDistributedQuery(reqId, fut, nodes);
+
+        return fut;
+    }
 
+    /** Initialize distributed query: stores future, sends query requests to nodes. */
+    private void initDistributedQuery(long reqId, GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
         try {
-            qry.query().validate();
+            DistributedCacheQueryReducer reducer = createReducer(fut.qry.query().type(), reqId, fut, nodes);
 
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
+            fut.reducer(reducer);
 
-            addQueryFuture(req.id(), fut);
+            fut.qry.query().validate();
 
-            final Object topic = topic(cctx.nodeId(), req.id());
+            addQueryFuture(reqId, fut);

Review comment:
       Let's move this to outside the method or create future itself in this method.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -338,55 +256,83 @@ public void onPage(@Nullable UUID nodeId, @Nullable Collection<?> data, @Nullabl
                                 "query", qry, true),
                             err));
 
-                    onPage(nodeId, true);
-
-                    notifyAll();
+                    lock.notifyAll();
                 }
             else {
                 if (data == null)
                     data = Collections.emptyList();
 
-                data = dedupIfRequired((Collection<Object>)data);
+                data = dedupIfRequired(data);
+
+                if (qry.query().type() == GridCacheQueryType.TEXT) {
+                    ArrayList unwrapped = new ArrayList();
+
+                    for (Object o: data) {
+                        CacheEntryWithPayload e = (CacheEntryWithPayload) o;
+
+                        Object uKey = CacheObjectUtils.unwrapBinary(
+                            cctx.cacheObjectContext(), e.getKey(), qry.query().keepBinary(), true, null);
+
+                        Object uVal = CacheObjectUtils.unwrapBinary(
+                            cctx.cacheObjectContext(), e.getValue(), qry.query().keepBinary(), true, null);
 
-                data = cctx.unwrapBinariesIfNeeded((Collection<Object>)data, qry.query().keepBinary());
+                        if (uKey != e.getKey() || uVal != e.getValue())
+                            unwrapped.add(new CacheEntryWithPayload<>(uKey, uVal, e.payload()));
+                        else
+                            unwrapped.add(o);
+                    }
+
+                    data = unwrapped;
+
+                } else
+                    data = cctx.unwrapBinariesIfNeeded((Collection<Object>)data, qry.query().keepBinary());
 
-                synchronized (this) {
-                    enqueue(data);
+                synchronized (lock) {

Review comment:
       Is the lock still needed?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return Head of stream and then clean it.
+     */
+    public R get() {
+        R ret = head;
+
+        head = null;
+
+        return ret;
+    }

Review comment:
       This method looks useless.
   Al the logic could be done in next() method.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726938937



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -523,57 +488,27 @@ else if (!cancelled.contains(res.requestId()))
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
     @Override public CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, false);
+    }
+
+    /** */
+    private CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes, boolean fields) {

Review comment:
       I don't like this idea, we make all actions on future within this method, there is no need to create a future in separate method. Also it is required to create `reqId` for both cases, so code duplication will appear.
   
   Let's save this logic.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r725852050



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #nextPage()} that returns a {@link NodePage} of cache query result from
+ * single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Last delivered page from the stream. */
+    private NodePage<R> head;

Review comment:
       We need to have additional object to notify a `NodePageStream` to request next page. It could be a flag or function. IMHO, usage `head` is the most clearest approach.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704363896



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Local handler of cache query request. */
+    private final Consumer<GridCacheQueryRequest> sendLoc;
+
+    /**
+     * @param cctx Cache context.
+     * @param sendLoc Local handler of cache query request.
+     */
+    CacheQueryPageRequester(final GridCacheContext cctx, Consumer<GridCacheQueryRequest> sendLoc) {
+        this.cctx = cctx;
+        this.log = cctx.kernalContext().config().getGridLogger();
+        this.sendLoc = sendLoc;
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        sendRequest(fut, req, nodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            Collection<ClusterNode> n = new ArrayList<>();
+            for (UUID id: nodes)
+                n.add(cctx.node(id));
+
+            sendRequest(fut, req, n);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Send cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQuery(long reqId, Collection<UUID> nodes, boolean fieldsQry) {
+        final GridCacheQueryManager qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            for (UUID node : nodes) {
+                try {
+                    if (cctx.localNodeId().equals(node)) {
+                        sendLoc.accept(req);
+
+                        continue;
+                    }
+
+                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
+                }
+                catch (IgniteCheckedException e) {
+                    if (cctx.io().checkNodeLeft(node, e, false)) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to send cancel request, node failed: " + node);
+                    }
+                    else
+                        U.error(log, "Failed to send cancel request [node=" + node + ']', e);
+                }
+            }
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        final GridCacheQueryFutureAdapter fut,
+        final GridCacheQueryRequest req,
+        Collection<ClusterNode> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        final UUID locNodeId = cctx.localNodeId();
+
+        ClusterNode locNode = null;
+
+        Collection<ClusterNode> rmtNodes = null;

Review comment:
       ```suggestion
   ```




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704617195



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheEntryWithPayload.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteBiTuple;
+
+/** Represents cache key-value pair and additional payload to compare cache entity by custom rule. */
+public class CacheEntryWithPayload<K, V, P> extends IgniteBiTuple<K, V> {

Review comment:
       Let's name it ScoredCacheEntry, and replace `payload` with `score`. I will expand this class for other cases in future, it will be needed.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682501375



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       Comparator instance can be static or moved into Reducer.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r695554903



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered via callback
+ * {@link #onPage(UUID, Collection, boolean)}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Callback that invoked on receiving a new page.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.
+     */
+    public boolean onPage(UUID nodeId, Collection<T> data, boolean last);
+
+    /**
+     * Callback in case of page with error.
+     */
+    public void onError();
+
+    /**
+     * Callback that invokes after a query future is done.
+     */
+    public void onFinish();

Review comment:
       It's not possible, as it's required to release latch after `onDone()`. Actually there is a tough coupling of future and reducer. Don't think this is a problem




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704613063



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+public class UnsortedDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** */
+    private NodePage<R> page;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    public UnsortedDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes) {
+        super(fut, reqId, pageRequester, queueLock, nodes);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        if (page != null && page.hasNext())
+            return true;
+
+        Collection<NodePageStream<R>> streams = new ArrayList<>(this.streams.values());
+
+        for (NodePageStream<R> s: streams) {
+            page = s.nextPage();
+
+            if (page != null && page.hasNext())
+                return true;
+
+            this.streams.remove(s.nodeId());

Review comment:
       `streams` cleared within `cancel()` action. Actually I add this to help GC a little. But I will revert this, as cancel can be also invoked after reaching limit, and in this moment `streams` can be not finished. In current version it doesn't matter, but will be for future optimizations of algorithm.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642434101



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {
+            if (!loadAllowed)
+                return;
+
+            nodes = new ArrayList<>();
+
+            for (int i = streamOff; i < streams.length; i++) {
+                UUID nodeId = streams[i].nodeId;
+
+                // Try to contain 2 pages for every stream to avoid waits. A node has to be present in collections:
+                // 1. rcvd - as requested and already handled before, so currently no parallel request to this node.
+                // 2. subgrid - as this node still has pages to request.
+                if (streams[i].queue.size() < 1 && rcvd.remove(nodeId) && subgrid.contains(nodeId))
+                    nodes.add(nodeId);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        boolean qryLast = super.onPage(nodeId, last);
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        if (last && streamsMap.containsKey(nodeId)) {
+            streamsMap.get(nodeId).allPagesReady = true;
+
+            streamsMap.remove(nodeId);
+        }
+
+        return qryLast;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        assert Thread.holdsLock(queueLock());
+
+        // Local node.
+        if (nodeId == null) {
+            nodeId = cctx.localNodeId();
+
+            // If nodeId is NULL and query doesn't execute on local node, then it is error, notify all streams.
+            if (!streamsMap.containsKey(nodeId)) {
+                assert data.isEmpty();
+
+                for (PageStream stream: streamsMap.values())
+                    stream.addPage(data);

Review comment:
       Why do you pass 'data' to all streams, if got a page from an unknown node?




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642418871



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public abstract class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** */
+    CacheQueryPageRequester(final GridCacheContext cctx) {
+        this.cctx = cctx;
+        this.log = cctx.kernalContext().config().getGridLogger();
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.subjectId(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        sendRequest(fut, req, nodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.subjectId(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            Collection<ClusterNode> n = new ArrayList<>();
+            for (UUID id: nodes)
+                n.add(cctx.node(id));
+
+            sendRequest(fut, req, n);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQueryRequest(long reqId, Collection<ClusterNode> nodes, boolean fieldsQry) {
+        final GridCacheQueryManager qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            // Process cancel query directly (without sending) for local node.
+            sendLocal(req);
+
+            if (!nodes.isEmpty()) {
+                for (ClusterNode node : nodes) {
+                    try {
+                        cctx.io().send(node, req, cctx.ioPolicy());
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to send cancel request, node failed: " + node);
+                        }
+                        else
+                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
+                    }
+                }
+            }
+        }

Review comment:
       ```suggestion
           if (nodes.isEmpty())
               return;
               
           try {
               final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
                   reqId,
                   fieldsQry,
                   cctx.startTopologyVersion(),
                   cctx.deploymentEnabled());
   
               // Process cancel query directly (without sending) for local node.
               sendLocal(req);
   
               for (ClusterNode node : nodes) {
                   try {
                       cctx.io().send(node, req, cctx.ioPolicy());
                   }
                   catch (IgniteCheckedException e) {
                       if (cctx.io().checkNodeLeft(node.id(), e, false)) {
                           if (log.isDebugEnabled())
                               log.debug("Failed to send cancel request, node failed: " + node);
                        }
                       else
                           U.error(log, "Failed to send cancel request [node=" + node + ']', e);
                   }
               }
           }
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645795856



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream(node.id());
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {

Review comment:
       In master branch (single stream) one of purposes of the lock is to guard consistency of `queue` and collections `rcvd`, `subgrid`. As far as I undertstand it's required to avoid excess requests to nodes that already finish query. It could lead to data duplication in final result, as query runs from beginning on such node. Actually as I can see it should be an extremely rare case, but it's possible. Also I don't find any other valuable reasons for using this lock to guard all of this variables together.
   
   In implementation with multiple streams we still have shared between streams `rdc` and `subgrid` variables, that has to be in consisten state with all stream queues. So this is a reason to use shared lock 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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726525989



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -17,257 +17,172 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.util.lang.GridPlainCallable;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.processors.cache.query.reducer.MergeSortDistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.reducer.NodePageStream;
+import org.apache.ignite.internal.processors.cache.query.reducer.UnsortedDistributedCacheQueryReducer;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
+
 /**
  * Distributed query future.
  */
 public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutureAdapter<K, V, R> {
     /** */
-    private static final long serialVersionUID = 0L;
+    private final long reqId;
 
-    /** */
-    private long reqId;
+    /** Helps to send cache query requests to other nodes. */
+    private final GridCacheDistributedQueryManager<K, V> qryMgr;
 
-    /** */
-    private final Collection<UUID> subgrid = new HashSet<>();
+    /** Collection of streams. */
+    private final Map<UUID, NodePageStream<R>> streams;
 
-    /** */
-    private final Collection<UUID> rcvd = new HashSet<>();
+    /** Count of streams that finish receiving remote pages. */
+    private final AtomicInteger noRemotePagesStreamsCnt = new AtomicInteger();
 
-    /** */
-    private CountDownLatch firstPageLatch = new CountDownLatch(1);
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Set of nodes that deliver their first page. */
+    private Set<UUID> rcvdFirstPage = ConcurrentHashMap.newKeySet();
 
     /**
      * @param ctx Cache context.
      * @param reqId Request ID.
      * @param qry Query.
-     * @param nodes Nodes.
      */
-    protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId, GridCacheQueryBean qry,
-        Iterable<ClusterNode> nodes) {
+    protected GridCacheDistributedQueryFuture(
+        GridCacheContext<K, V> ctx,
+        long reqId,
+        GridCacheQueryBean qry,
+        Collection<ClusterNode> nodes
+    ) {
         super(ctx, qry, false);
 
         assert reqId > 0;
 
         this.reqId = reqId;
 
-        GridCacheQueryManager<K, V> mgr = ctx.queries();
+        qryMgr = (GridCacheDistributedQueryManager<K, V>) ctx.queries();
 
-        assert mgr != null;
+        streams = new ConcurrentHashMap<>(nodes.size());
 
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
-    }
+        for (ClusterNode node : nodes) {
+            NodePageStream<R> s = new NodePageStream<>(node.id(), () -> requestPages(node.id()), () -> cancelPages(node.id()));
 
-    /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
+            streams.put(node.id(), s);
 
-        assert qryMgr != null;
+            startQuery(node.id());
+        }
 
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
+        Map<UUID, NodePageStream<R>> streamsMap = Collections.unmodifiableMap(streams);
 
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
+        reducer = qry.query().type() == TEXT ?
+            new MergeSortDistributedCacheQueryReducer<>(streamsMap)
+            : new UnsortedDistributedCacheQueryReducer<>(streamsMap);
+    }
 
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
+    /** {@inheritDoc} */
+    @Override protected void cancelQuery(Throwable err) {
+        firstPageLatch.countDown();
 
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
+        for (NodePageStream<R> s : streams.values())
+            s.cancel(err);
 
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
     @Override protected void onNodeLeft(UUID nodeId) {

Review comment:
       Let's remote the `onNodeLeft` method and call the `onError` method instead with an appropriate `ClusterTopologyCheckedException`.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r648229840



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
+        Collection<ClusterNode> nodes;
+
+        synchronized (queueLock()) {
+            nodes = F.retain(allNodes, true,
+                new P1<ClusterNode>() {
+                    @Override public boolean apply(ClusterNode node) {
+                        return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
+                    }
+                }
+            );
+
+            rcvd.clear();
+            subgrid.clear();
+        }
+
+        pageRequester.cancelQueryRequest(reqId, nodes, fut.fields());
+
+        pageStream.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            // Loads only queue is empty to avoid memory consumption on additional pages.
+            if (!pageStream.queue.isEmpty())
+                return;
+
+            if (loadAllowed && rcvd.containsAll(subgrid)) {
+                rcvd.clear();
+
+                nodes = new ArrayList<>(subgrid);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadAll() throws IgniteInterruptedCheckedException {
+        assert !Thread.holdsLock(queueLock());
+
+        U.await(firstPageLatch);
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            if (loadAllowed && !subgrid.isEmpty())
+                nodes = new ArrayList<>(subgrid);
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        assert Thread.holdsLock(queueLock());
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        rcvd.add(nodeId);
+
+        if (!loadAllowed && rcvd.containsAll(subgrid) ) {
+            firstPageLatch.countDown();
+            loadAllowed = true;
+        }
+
+        return last && subgrid.remove(nodeId) && subgrid.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onNodeLeft(UUID nodeId) {
+        synchronized (queueLock()) {
+            return subgrid.contains(nodeId);

Review comment:
       Renamed method, now it just checks that specified node is running query.
   
   > Is there any reason to return smth to higher-level instead of releasing a latch instantly?
   
   I tried to avoid this, but found that we must release a latch (I mean CountDownLatch) after we mark future as done. It's required for correct return exception to a user in case it's hanging on `awaitInitialization()` method.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655438939



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #onPage(UUID, Collection, boolean)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Offer query result page for reduce. Note that the data collection may contain extension of type T.
+     * In such cases data item contains additional payload for custom reducer logic.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.

Review comment:
       ```suggestion
        * Callback that invoked on receiving a new page. 
        *
        * @param nodeId Node ID that sent this page.
        * @param data Page data rows.
        * @param last Whether this page is last for specified {@code nodeId}.
        * @return {@code true} if this page is final page for query and no more pages are waited, {@code false} otherwise.
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r647807863



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.

Review comment:
       Now there is single onPage method. The `last` param shows if this page is the last page from node `nodeId`. It's required to make some clean up in reducer, if we know that there wouldn't be pages from this node.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704600305



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/AbstractDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.DistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Abstract class for distributed reducer implementations.
+ */
+abstract class AbstractDistributedCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /** Collection of streams that don't deliver all pages yet. */
+    private final Map<UUID, NodePageStream<R>> remoteStreams;

Review comment:
       The `streams` collection's goals are:
   1. `hasNext()` and `next()` in a user thread use thread local variable `streams`. 
   2. When a stream is fully done (remote pages and local iterator), then we remove it from the `stream` collection.
   
   The `remoteStreams` collection's goals are different:
   1. Ignite's utility threads use it for handling new remote pages.
   2. When remote pages finish (but not local iterator), then we remove this stream from the collection. 
   3. Some background actions (cancel, onNodeLeft) bases on this collection. Then we need track set of remote node ids.
   
   Then the idea was to have `streams` thread locally, and `remoteStreams` concurrently. It simplified a sync logic between collections as they should be independent from each other, also it helps to avoid one more lock in user thread.
   
   It's possible to share single collection for all threads. But we will still have another collection for tracking remote nodes with alive pages. This tracking collection can be a Set<UUID>, and then we will get NodePageStream from the main collection (`stream`). We won't win any from this. I think it's fine to track NodePageStream in 2 independent collectios. WDYT?
   
   BUT. There was a little bug, `onError` uses `streams` and runs in non-user thread, I'll replace it with the `remoteStreams`, and this will be fine. 




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Mmuzaf commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726591836



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -776,144 +686,99 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                null,
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
+    }
 
-        return fut;
+    /**
+     * Gets topic for ordered response messages.
+     *
+     * @param nodeId Node ID.
+     * @param reqId Request ID.
+     * @return Topic.
+     */
+    private Object topic(UUID nodeId, long reqId) {
+        return TOPIC_CACHE.topic(TOPIC_PREFIX, nodeId, reqId);
     }
 
     /**
      * Sends query request.
      *
-     * @param fut Distributed future.
+     * @param fut Cache query future. {@code null} in case of cancel request.
      * @param req Request.
      * @param nodes Nodes.
      * @throws IgniteCheckedException In case of error.
      */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
+    public void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes

Review comment:
       This is always used as a single node, so I think the `Collection` is redundant.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim edited a comment on pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim edited a comment on pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#issuecomment-859468662


   Hi @AMashenkov ! I've answered on all comments (1 change is accepted, other on discussion), could you please have a look?


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r718596118



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext<?, ?> cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Local handler of cache query request. */
+    private final Consumer<GridCacheQueryRequest> sendLoc;
+
+    /**
+     * @param cctx Cache context.
+     * @param sendLoc Local handler of cache query request.
+     */
+    CacheQueryPageRequester(GridCacheContext<?, ?> cctx, Consumer<GridCacheQueryRequest> sendLoc) {
+        this.cctx = cctx;
+        this.sendLoc = sendLoc;
+
+        log = cctx.kernalContext().config().getGridLogger();
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture<?, ?, ?> fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter<?> qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.idxQryDesc(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        List<UUID> sendNodes = new ArrayList<>();
+
+        for (ClusterNode n: nodes)
+            sendNodes.add(n.id());
+
+        sendRequest(fut, req, sendNodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter<?, ?, ?> fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter<?> qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            sendRequest(fut, req, nodes);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Send cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQuery(long reqId, Collection<UUID> nodes, boolean fieldsQry) {
+        GridCacheQueryManager<?, ?> qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            sendRequest(null, req, nodes);
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param fut Cache query future. {@code null} in case of cancel request.
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        UUID locNodeId = cctx.localNodeId();
+
+        boolean loc = false;
+
+        for (UUID nodeId : nodes) {
+            if (nodeId.equals(locNodeId))
+                loc = true;
+            else {
+                if (req.cancel())
+                    sendNodeCancelRequest(nodeId, req);
+                else if (!sendNodePageRequest(nodeId, req, fut))
+                    return;
+            }
+        }
+
+        if (loc)
+            sendLoc.accept(req);
+    }
+
+    /** */
+    private void sendNodeCancelRequest(UUID nodeId, GridCacheQueryRequest req) throws IgniteCheckedException {
+        try {
+            cctx.io().send(nodeId, req, GridIoPolicy.QUERY_POOL);
+        }
+        catch (IgniteCheckedException e) {
+            if (cctx.io().checkNodeLeft(nodeId, e, false)) {

Review comment:
       I'd like to remain this check. There is a logic of node left. And this code interfere to this situation. I think this behavior should be investigated with some tests. I will create a separate ticket for that.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r648224663



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;

Review comment:
       Renamed this method to `requestFullPages`. So this method about requesting whole data at once from remote node.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r654387510



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return Head of stream and then clean it.
+     */
+    public R get() {
+        R ret = head;
+
+        head = null;
+
+        return ret;
+    }

Review comment:
       Agree, moving fetch value from next() to hasNext() resolves this. Added.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704379726



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+public class UnsortedDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** */
+    private NodePage<R> page;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    public UnsortedDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes) {
+        super(fut, reqId, pageRequester, queueLock, nodes);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        if (page != null && page.hasNext())
+            return true;
+
+        Collection<NodePageStream<R>> streams = new ArrayList<>(this.streams.values());
+
+        for (NodePageStream<R> s: streams) {
+            page = s.nextPage();
+
+            if (page != null && page.hasNext())
+                return true;
+
+            this.streams.remove(s.nodeId());

Review comment:
       `streams` is not concurrent structore.
   However it can be cleared concurrently.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r685262569



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       I'm going to use this reducer in IndexQuery later. It compares rows by column values, not by score.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704373832



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/AbstractDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.DistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Abstract class for distributed reducer implementations.
+ */
+abstract class AbstractDistributedCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /** Collection of streams that don't deliver all pages yet. */
+    private final Map<UUID, NodePageStream<R>> remoteStreams;

Review comment:
       Why we need a duplicate collection?
   Can `streams` be used instead? Or Set<UUID> or even counter?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642337612



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public abstract class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** */
+    CacheQueryPageRequester(final GridCacheContext cctx) {
+        this.cctx = cctx;
+        this.log = cctx.kernalContext().config().getGridLogger();
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.subjectId(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        sendRequest(fut, req, nodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.subjectId(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            Collection<ClusterNode> n = new ArrayList<>();
+            for (UUID id: nodes)
+                n.add(cctx.node(id));
+
+            sendRequest(fut, req, n);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQueryRequest(long reqId, Collection<ClusterNode> nodes, boolean fieldsQry) {
+        final GridCacheQueryManager qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            // Process cancel query directly (without sending) for local node.
+            sendLocal(req);
+
+            if (!nodes.isEmpty()) {
+                for (ClusterNode node : nodes) {
+                    try {
+                        cctx.io().send(node, req, cctx.ioPolicy());
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to send cancel request, node failed: " + node);
+                        }
+                        else
+                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
+                    }
+                }
+            }
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        final GridCacheQueryFutureAdapter fut,
+        final GridCacheQueryRequest req,
+        Collection<ClusterNode> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        final UUID locNodeId = cctx.localNodeId();
+
+        ClusterNode locNode = null;
+
+        Collection<ClusterNode> rmtNodes = null;
+
+        for (ClusterNode n : nodes) {
+            if (n.id().equals(locNodeId))
+                locNode = n;
+            else {
+                if (rmtNodes == null)
+                    rmtNodes = new ArrayList<>(nodes.size());
+
+                rmtNodes.add(n);
+            }
+        }
+
+        // Request should be sent to remote nodes before the query is processed on the local node.
+        // For example, a remote reducer has a state, we should not serialize and then send
+        // the reducer changed by the local node.
+        if (!F.isEmpty(rmtNodes)) {
+            for (ClusterNode node : rmtNodes) {
+                try {
+                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
+                }
+                catch (IgniteCheckedException e) {
+                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
+                        fut.onNodeLeft(node.id());
+
+                        if (fut.isDone())
+                            return;
+                    }
+                    else
+                        throw e;
+                }
+            }
+        }
+
+        if (locNode != null)
+            sendLocal(req);
+    }
+
+    /** Send and handle request to local node. */

Review comment:
       ```suggestion
       /** 
       * Send and handle request to local node. 
       */
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642952364



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {
+            if (!loadAllowed)
+                return;
+
+            nodes = new ArrayList<>();
+
+            for (int i = streamOff; i < streams.length; i++) {
+                UUID nodeId = streams[i].nodeId;
+
+                // Try to contain 2 pages for every stream to avoid waits. A node has to be present in collections:
+                // 1. rcvd - as requested and already handled before, so currently no parallel request to this node.
+                // 2. subgrid - as this node still has pages to request.
+                if (streams[i].queue.size() < 1 && rcvd.remove(nodeId) && subgrid.contains(nodeId))
+                    nodes.add(nodeId);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        boolean qryLast = super.onPage(nodeId, last);
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        if (last && streamsMap.containsKey(nodeId)) {
+            streamsMap.get(nodeId).allPagesReady = true;
+
+            streamsMap.remove(nodeId);
+        }
+
+        return qryLast;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        assert Thread.holdsLock(queueLock());

Review comment:
       You're right. I've simplified code, as nodeId == null for distributed reducer means only error. So we can remove check for sync (that was required to guard streamsMap) and replace with check for NPE.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655455509



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -306,10 +221,12 @@ else if (capacity > 0) {
     }
 
     /**
+     * Entrypoint for handling query result page from remote node.
+     *
      * @param nodeId Sender node.
      * @param data Page data.
      * @param err Error (if was).
-     * @param finished Finished or not.
+     * @param finished Whether it is the last page for sender node.

Review comment:
       Maybe 'lastPage' flag would be a better name?




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682500736



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       AFAIU, only Text queries have a 'payload'. Why is it named 'payload' and not score?
   Does 'null' score make sense for the result?  If no then no NPE possible.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655459716



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {

Review comment:
       ```suggestion
       /**
       * Peek the stream head for the next item.
       *
       * Note: requires {@link #hasNext()} to be called at first.
       * @return The item will be returned with {@link #next()}.
       */
       R head() {
   ```




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



[GitHub] [ignite] timoninmaxim commented on pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#issuecomment-902524559


   @AMashenkov, @korlov42  Hi! this is a gentle reminder. Could you please have a look on my changes?


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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r693788471



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Checks whether node with provided {@code nodeId} is a map node for the query.
+     * Note: if all pages were received from this node, then the method will return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if node with provided {@code nodeId} is a map node for the query, {@code false} otherwise.
+     */
+    public boolean mapNode(UUID nodeId);

Review comment:
       This is bad codestyle =)
   The one have to read javadoc to understand what the method doing. 
   It is unclear whether the method checks "the node is map-node" or method "make the node a map-node".
   Maybe name "mappedToNode" be better? WDYT?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Mmuzaf commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726527273



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -776,144 +686,99 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                null,
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
+    }
 
-        return fut;
+    /**
+     * Gets topic for ordered response messages.
+     *
+     * @param nodeId Node ID.
+     * @param reqId Request ID.
+     * @return Topic.
+     */
+    private Object topic(UUID nodeId, long reqId) {
+        return TOPIC_CACHE.topic(TOPIC_PREFIX, nodeId, reqId);
     }
 
     /**
      * Sends query request.
      *
-     * @param fut Distributed future.
+     * @param fut Cache query future. {@code null} in case of cancel request.
      * @param req Request.
      * @param nodes Nodes.
      * @throws IgniteCheckedException In case of error.
      */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
+    public void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes
     ) throws IgniteCheckedException {
-        assert fut != null;
         assert req != null;
         assert nodes != null;
 
-        final UUID locNodeId = cctx.localNodeId();
+        UUID locNodeId = cctx.localNodeId();
 
-        ClusterNode locNode = null;
+        boolean loc = false;
 
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
+        for (UUID nodeId : nodes) {
+            if (nodeId.equals(locNodeId))
+                loc = true;
             else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
-
-                rmtNodes.add(n);
-            }
-        }
-
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
-
-                        if (fut.isDone())
-                            return;
-                    }
-                    else
-                        throw e;
-                }
+                if (req.cancel())
+                    sendNodeCancelRequest(nodeId, req);
+                else if (!sendNodePageRequest(nodeId, req, fut))
+                    return;
             }
         }
 
-        if (locNode != null) {
+        if (loc) {
             cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
                 @Override public Object call() throws Exception {
                     req.beforeLocalExecution(cctx);
 
-                    processQueryRequest(locNodeId, req);
+                    processQueryRequest(cctx.localNodeId(), req);
 
                     return null;
                 }
             }, GridIoPolicy.QUERY_POOL);
         }
     }
 
+    /** */
+    private void sendNodeCancelRequest(UUID nodeId, GridCacheQueryRequest req) throws IgniteCheckedException {
+        try {
+            cctx.io().send(nodeId, req, GridIoPolicy.QUERY_POOL);
+        }
+        catch (IgniteCheckedException e) {
+            if (cctx.io().checkNodeLeft(nodeId, e, false)) {

Review comment:
       Let's remove `checkNodeLeft` usages? This case is covered by discovery message guarantees.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726929733



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -17,257 +17,172 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.util.lang.GridPlainCallable;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.processors.cache.query.reducer.MergeSortDistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.reducer.NodePageStream;
+import org.apache.ignite.internal.processors.cache.query.reducer.UnsortedDistributedCacheQueryReducer;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
+
 /**
  * Distributed query future.
  */
 public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutureAdapter<K, V, R> {
     /** */
-    private static final long serialVersionUID = 0L;
+    private final long reqId;
 
-    /** */
-    private long reqId;
+    /** Helps to send cache query requests to other nodes. */
+    private final GridCacheDistributedQueryManager<K, V> qryMgr;
 
-    /** */
-    private final Collection<UUID> subgrid = new HashSet<>();
+    /** Collection of streams. */
+    private final Map<UUID, NodePageStream<R>> streams;
 
-    /** */
-    private final Collection<UUID> rcvd = new HashSet<>();
+    /** Count of streams that finish receiving remote pages. */
+    private final AtomicInteger noRemotePagesStreamsCnt = new AtomicInteger();
 
-    /** */
-    private CountDownLatch firstPageLatch = new CountDownLatch(1);
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Set of nodes that deliver their first page. */
+    private Set<UUID> rcvdFirstPage = ConcurrentHashMap.newKeySet();
 
     /**
      * @param ctx Cache context.
      * @param reqId Request ID.
      * @param qry Query.
-     * @param nodes Nodes.
      */
-    protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId, GridCacheQueryBean qry,
-        Iterable<ClusterNode> nodes) {
+    protected GridCacheDistributedQueryFuture(
+        GridCacheContext<K, V> ctx,
+        long reqId,
+        GridCacheQueryBean qry,
+        Collection<ClusterNode> nodes
+    ) {
         super(ctx, qry, false);
 
         assert reqId > 0;
 
         this.reqId = reqId;
 
-        GridCacheQueryManager<K, V> mgr = ctx.queries();
+        qryMgr = (GridCacheDistributedQueryManager<K, V>) ctx.queries();
 
-        assert mgr != null;
+        streams = new ConcurrentHashMap<>(nodes.size());
 
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
-    }
+        for (ClusterNode node : nodes) {
+            NodePageStream<R> s = new NodePageStream<>(node.id(), () -> requestPages(node.id()), () -> cancelPages(node.id()));
 
-    /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
+            streams.put(node.id(), s);
 
-        assert qryMgr != null;
+            startQuery(node.id());

Review comment:
       Reverted.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682611383



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered via callback
+ * {@link #onPage(UUID, Collection, boolean)}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Callback that invoked on receiving a new page.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.
+     */
+    public boolean onPage(UUID nodeId, Collection<T> data, boolean last);
+
+    /**
+     * Callback in case of page with error.
+     */
+    public void onError();
+
+    /**
+     * Callback that invokes after a query future is done.
+     */
+    public void onFinish();

Review comment:
       There is a comment about it, see invokation [1]. We should release latch after onDone() invokation. Also I've described this in previous conversation: https://github.com/apache/ignite/pull/9081#discussion_r649824494
   
   [1] [GridCacheQueryFutureAdapter:332](https://github.com/apache/ignite/blob/0add93b488ebe7265b2a0f951fcdfd55adbcc01f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java#L332)




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655443239



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #onPage(UUID, Collection, boolean)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.

Review comment:
       Actually, interface don't imply any coupling with the  query future.
   If you think that this worth to be mentioned, then abstract class or concrete implementation would be a better place.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655451676



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -146,40 +137,50 @@ public GridCacheQueryBean query() {
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
 
-            cnt.decrementAndGet();
+            checkError();
+
+            R next = null;
+
+            if (reducer().hasNext()) {
+                next = unmaskNull(reducer().next());
+
+                if (!limitDisabled) {
+                    cnt++;
+
+                    // Exceed limit, stop page loading and cancel queries.
+                    if (cnt == capacity)
+                        cancel();
+                }
+            }
+
+            checkError();
 
             return next;
         }
-        catch (NoSuchElementException ignored) {
-            return null;
-        }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
         }
     }
 
+    /** @return Cache query results reducer. */

Review comment:
       ```suggestion
       /** 
       * @return Cache query results reducer. 
       */
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642618926



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;

Review comment:
       Check this code:
   https://github.com/apache/ignite/pull/9081/files#diff-068da2d055341b4686f2dc28d687b5c58678a255cde712e4062588bf0fc4f46fR135




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



[GitHub] [ignite] timoninmaxim commented on pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#issuecomment-857642340


   Hi @AMashenkov ! Thanks for comments, I refactored code and make it more clear:
   1. Move all reducers to separate package, make PageStream, NodePageStream top level classes;
   2. Remove custom logic from NodePageStream (now it just wrapper over PageStream with single node) and leverage on PageStream only. It also helps to simplify logic of MergeSortReducer;
   3. Reworked CacheQueryReducer interface: added callback methods (onError, onCancel, onFinish), removed/renamed and fixed doc for unclear methods, moving some methods to top level hierarchy class.
   
   Could you please review PR again? Thanks!


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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642339704



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/AbstractCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Base abstract class for all Reducer descenders. It declares dependencies on cache query, and base logic for handling
+ * remote pages.
+ */
+public abstract class AbstractCacheQueryReducer<R> implements CacheQueryReducer<R> {
+    /** Query info. */
+    private final GridCacheQueryAdapter qry;
+
+    /** Flag shows whether all pages are ready. */
+    protected volatile boolean allPagesReady;
+
+    /** Timestamp when a query fails with timeout. */
+    private final long timeoutTime;
+
+    /** Lock shared between this reducer and future. */
+    private final Object sharedLock;
+
+    /**
+     * @param fut Cache query future relates to this query. Future is done when last page is delivered to reducer.
+     */
+    AbstractCacheQueryReducer(GridCacheQueryFutureAdapter fut) {
+        qry = fut.qry.query();
+        timeoutTime = fut.endTime();
+        // The only reason to use lock in 2 places is the deduplication mechanism.
+        sharedLock = fut.lock;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object sharedLock() {
+        return sharedLock;
+    }

Review comment:
       This looks like abstraction leakage antipattern.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655460576



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {

Review comment:
       Let's make the class package visible if possible.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642412156



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/AbstractCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Base abstract class for all Reducer descenders. It declares dependencies on cache query, and base logic for handling
+ * remote pages.
+ */
+public abstract class AbstractCacheQueryReducer<R> implements CacheQueryReducer<R> {
+    /** Query info. */
+    private final GridCacheQueryAdapter qry;
+
+    /** Flag shows whether all pages are ready. */
+    protected volatile boolean allPagesReady;
+
+    /** Timestamp when a query fails with timeout. */
+    private final long timeoutTime;
+
+    /** Lock shared between this reducer and future. */
+    private final Object sharedLock;
+
+    /**
+     * @param fut Cache query future relates to this query. Future is done when last page is delivered to reducer.
+     */
+    AbstractCacheQueryReducer(GridCacheQueryFutureAdapter fut) {
+        qry = fut.qry.query();
+        timeoutTime = fut.endTime();
+        // The only reason to use lock in 2 places is the deduplication mechanism.
+        sharedLock = fut.lock;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object sharedLock() {
+        return sharedLock;
+    }

Review comment:
       @AMashenkov hi! Thanks for review. I've removed this method from interface, and make protected.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r695512207



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Checks whether node with provided {@code nodeId} is a map node for the query.
+     * Note: if all pages were received from this node, then the method will return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if node with provided {@code nodeId} is a map node for the query, {@code false} otherwise.
+     */
+    public boolean mapNode(UUID nodeId);

Review comment:
       Renamed to `remoteQueryNode`




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Mmuzaf commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r721285189



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #nextPage()} that returns a {@link NodePage} of cache query result from
+ * single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Last delivered page from the stream. */
+    private NodePage<R> head;
+
+    /** Promise to notify the stream consumer about delivering new page. */
+    private CompletableFuture<UUID> pageReady = new CompletableFuture<>();
+
+    /** This lock syncs {@link #head} and {@link #noRemotePages}. */
+    private final Object pagesLock = new Object();

Review comment:
       Let's do synchronization on `this`.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Base class for distributed cache query reducers.
+ */
+public abstract class DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Page streams collection. */
+    protected final Map<UUID, NodePageStream<T>> pageStreams;
+
+    /** Timestamp of query timeout. */
+    protected final long endTime;

Review comment:
       Let's move the endTime-related logic out from the distributed reducer to the `NodeStreams` class. When the `endTime` is reached the `NodeStreams#nextPage` should throw an `IgniteTimeoutException`.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #nextPage()} that returns a {@link NodePage} of cache query result from
+ * single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Last delivered page from the stream. */
+    private NodePage<R> head;

Review comment:
       Do we need it in case of using the ` CompletableFuture<Collection<R>>`?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #nextPage()} that returns a {@link NodePage} of cache query result from
+ * single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Last delivered page from the stream. */
+    private NodePage<R> head;
+
+    /** Promise to notify the stream consumer about delivering new page. */
+    private CompletableFuture<UUID> pageReady = new CompletableFuture<>();
+
+    /** This lock syncs {@link #head} and {@link #noRemotePages}. */
+    private final Object pagesLock = new Object();
+
+    /** Request pages action. */
+    private final Runnable reqPages;
+
+    /** */
+    public NodePageStream(UUID nodeId, Runnable reqPages) {
+        this.nodeId = nodeId;
+        this.reqPages = reqPages;
+    }
+
+    /** */
+    public UUID nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * @return Future that will be completed when a new page delivered.
+     */
+    public CompletableFuture<UUID> pageReady() {
+        return pageReady;
+    }
+
+    /**
+     * Returns a last delivered page from this stream. Note, that this method has to be invoked after getting
+     * the future provided with {@link #pageReady()}.
+     *
+     * @return Query result page.
+     */
+    public NodePage<R> nextPage() {

Review comment:
       I suggest keeping CompletableFuture<NodePage<R>> the method return type here.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #nextPage()} that returns a {@link NodePage} of cache query result from
+ * single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Last delivered page from the stream. */
+    private NodePage<R> head;
+
+    /** Promise to notify the stream consumer about delivering new page. */
+    private CompletableFuture<UUID> pageReady = new CompletableFuture<>();

Review comment:
       We should keep not the nodeId promise, but `CompletableFuture<Collection<R>>`.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #nextPage()} that returns a {@link NodePage} of cache query result from
+ * single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Last delivered page from the stream. */
+    private NodePage<R> head;
+
+    /** Promise to notify the stream consumer about delivering new page. */
+    private CompletableFuture<UUID> pageReady = new CompletableFuture<>();
+
+    /** This lock syncs {@link #head} and {@link #noRemotePages}. */
+    private final Object pagesLock = new Object();
+
+    /** Request pages action. */
+    private final Runnable reqPages;
+
+    /** */
+    public NodePageStream(UUID nodeId, Runnable reqPages) {
+        this.nodeId = nodeId;
+        this.reqPages = reqPages;
+    }
+
+    /** */
+    public UUID nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * @return Future that will be completed when a new page delivered.
+     */
+    public CompletableFuture<UUID> pageReady() {
+        return pageReady;
+    }
+
+    /**
+     * Returns a last delivered page from this stream. Note, that this method has to be invoked after getting
+     * the future provided with {@link #pageReady()}.
+     *
+     * @return Query result page.
+     */
+    public NodePage<R> nextPage() {
+        boolean loadPage = false;
+
+        NodePage<R> page;
+
+        synchronized (pagesLock) {
+            assert head != null;
+
+            page = head;
+
+            head = null;
+
+            if (!noRemotePages)
+                loadPage = true;
+        }
+
+        if (loadPage) {
+            pageReady = new CompletableFuture<>();
+
+            reqPages.run();
+        }
+
+        return page;
+    }
+
+    /**
+     * Add new query result page of data.
+     *
+     * @param data Collection of query result items.
+     * @param last Whether it is the last page from this node.
+     */
+    public void addPage(Collection<R> data, boolean last) {
+        synchronized (pagesLock) {
+            head = new NodePage<>(nodeId, data);
+
+            if (last)
+                noRemotePages = true;
+        }
+
+        pageReady.complete(nodeId);
+    }
+
+    /**
+     * Cancel query on all nodes.
+     */
+    public void cancel() {
+        synchronized (pagesLock) {
+            head = new NodePage<>(nodeId, Collections.emptyList());

Review comment:
       I suggest completing the future with the `IgniteCancelledException` in case of any errors and change the `cancel` to the `complete(IgniteException e)` or `error(IgniteException e)`.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r648226199



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #addPage(UUID, Collection)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Offer query result page for reduce. Note that the data collection may contain extension of type T.
+     * In such cases it stores additional payload for custom reducer logic.
+     *
+     * @param nodeId Node ID that sent this page. {@code null} means local node or error page.
+     * @param data Page data rows.
+     */
+    public void addPage(@Nullable UUID nodeId, Collection<T> data);
+
+    /**
+     * Callback that invokes after reducer get last query result page.
+     * Also invokes for failed queries to let reducer know that there won't be new pages.

Review comment:
       Added them.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r647804741



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
+        Collection<ClusterNode> nodes;
+
+        synchronized (queueLock()) {
+            nodes = F.retain(allNodes, true,
+                new P1<ClusterNode>() {
+                    @Override public boolean apply(ClusterNode node) {
+                        return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
+                    }
+                }
+            );
+
+            rcvd.clear();
+            subgrid.clear();
+        }
+
+        pageRequester.cancelQueryRequest(reqId, nodes, fut.fields());
+
+        pageStream.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            // Loads only queue is empty to avoid memory consumption on additional pages.
+            if (!pageStream.queue.isEmpty())
+                return;
+
+            if (loadAllowed && rcvd.containsAll(subgrid)) {
+                rcvd.clear();
+
+                nodes = new ArrayList<>(subgrid);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadAll() throws IgniteInterruptedCheckedException {
+        assert !Thread.holdsLock(queueLock());
+
+        U.await(firstPageLatch);
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            if (loadAllowed && !subgrid.isEmpty())
+                nodes = new ArrayList<>(subgrid);
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {

Review comment:
       Merged them to single method.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645795856



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream(node.id());
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {

Review comment:
       In master branch (single stream) one of purposes of the lock is to guard consistency of `queue` and collections `rcvd`, `subgrid`. As far as I undertstand it's required to avoid excess requests to nodes that already finish query. It could lead to data duplication in final result, as query runs from beginning . Actually as I can see it should be an extremely rare case, but it's possible. Also I don't find any other valuable reasons for this lock.
   
   In implementation with multiple streams we still have shared between streams `rdc` and `subgrid` variables, that has to be in consisten state with all stream queues. So this is a reason to use shared lock 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



[GitHub] [ignite] Mmuzaf commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726437686



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #headPage()} that returns a future will be completed with {@link NodePage}
+ * of cache query result from single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Promise to notify the stream consumer about delivering new page. */
+    private CompletableFuture<NodePage<R>> head = new CompletableFuture<>();
+
+    /** Request pages action. */
+    private final Runnable reqPages;

Review comment:
       This is not so critical, however, it's better to move all the `final` fields upper rather than others.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #headPage()} that returns a future will be completed with {@link NodePage}
+ * of cache query result from single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Promise to notify the stream consumer about delivering new page. */
+    private CompletableFuture<NodePage<R>> head = new CompletableFuture<>();
+
+    /** Request pages action. */
+    private final Runnable reqPages;
+
+    /** Cancel remote pages action. */
+    private final Runnable cancelPages;
+
+    /** */
+    public NodePageStream(UUID nodeId, Runnable reqPages, Runnable cancelPages) {
+        this.nodeId = nodeId;
+        this.reqPages = reqPages;
+        this.cancelPages = cancelPages;
+    }
+
+    /** */
+    public UUID nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * Returns a last delivered page from this stream.
+     *
+     * @return Future that will be completed with query result page.
+     */
+    public CompletableFuture<NodePage<R>> headPage() {

Review comment:
       Let's make all the methods of this class `synchronized` instead of using the synchronized block, it will look much better.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #headPage()} that returns a future will be completed with {@link NodePage}
+ * of cache query result from single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;

Review comment:
       Let's rename it to `hasRemotePages` to avoid negotiation each time.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #headPage()} that returns a future will be completed with {@link NodePage}
+ * of cache query result from single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */

Review comment:
       ```suggestion
       /** Node ID to stream pages. */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #headPage()} that returns a future will be completed with {@link NodePage}
+ * of cache query result from single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Promise to notify the stream consumer about delivering new page. */
+    private CompletableFuture<NodePage<R>> head = new CompletableFuture<>();
+
+    /** Request pages action. */
+    private final Runnable reqPages;
+
+    /** Cancel remote pages action. */
+    private final Runnable cancelPages;
+
+    /** */
+    public NodePageStream(UUID nodeId, Runnable reqPages, Runnable cancelPages) {
+        this.nodeId = nodeId;
+        this.reqPages = reqPages;
+        this.cancelPages = cancelPages;
+    }
+
+    /** */
+    public UUID nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * Returns a last delivered page from this stream.
+     *
+     * @return Future that will be completed with query result page.
+     */
+    public CompletableFuture<NodePage<R>> headPage() {
+        synchronized (this) {
+            return head;
+        }
+    }
+
+    /**
+     * Add new query result page of data.
+     *
+     * @param data Collection of query result items.
+     * @param last Whether it is the last page from this node.
+     */
+    public void addPage(Collection<R> data, boolean last) {
+        synchronized (this) {
+            head.complete(new NodePage<R>(nodeId, data) {
+                /** Flag shows whether the request for new page was triggered. */
+                private boolean reqNext;
+
+                /** {@inheritDoc} */
+                @Override public boolean hasNext() {
+                    if (!reqNext) {
+                        synchronized (NodePageStream.this) {
+                            if (!noRemotePages) {
+                                head = new CompletableFuture<>();
+
+                                reqPages.run();
+                            } else
+                                head = null;
+                        }
+
+                        reqNext = true;
+                    }
+
+                    return super.hasNext();
+                }
+            });
+
+            if (last)
+                noRemotePages = true;
+        }
+    }
+
+    /**
+     * Cancel query on all nodes.
+     */
+    public void cancel(Throwable err) {
+        synchronized (this) {
+            if (!closed()) {
+                head.completeExceptionally(err);
+
+                cancelPages.run();
+
+                noRemotePages = true;
+            }
+        }
+    }

Review comment:
       Suggest exiting from the main path earlier:
   
   ```suggestion
       public synchronized void cancel(Throwable err) {
           if (closed())
               return;
   
           head.completeExceptionally(err);
           cancelPages.run();
           noRemotePages = true;
       }
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -284,4 +199,62 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
     long requestId() {
         return reqId;
     }
+
+    /**
+     * Send initial query request to specified node.
+     *
+     * @param nodeId Node to send request.
+     */
+    private void startQuery(UUID nodeId) {
+        try {
+            GridCacheQueryRequest req = GridCacheQueryRequest.startQueryRequest(cctx, reqId, this);
+
+            qryMgr.sendRequest(this, req, Collections.singletonList(nodeId));
+        }
+        catch (IgniteCheckedException e) {
+            onError(e);
+        }
+    }
+
+    /**
+     * Send request to fetch new pages.
+     *
+     * @param nodeId Node to send request.
+     */
+    private void requestPages(UUID nodeId) {
+        try {
+            GridCacheQueryRequest req = GridCacheQueryRequest.pageRequest(cctx, reqId, query().query(), fields());
+
+            qryMgr.sendRequest(null, req, Collections.singletonList(nodeId));
+        }
+        catch (IgniteCheckedException e) {
+            onError(e);
+        }
+    }
+
+    /**
+     * Cancels remote pages.
+     *
+     * @param nodeId Node to send request.
+     */
+    private void cancelPages(UUID nodeId) {
+        try {
+            GridCacheQueryRequest req = GridCacheQueryRequest.cancelRequest(cctx, reqId, fields());
+
+            qryMgr.sendRequest(null, req, Collections.singletonList(nodeId));
+        }
+        catch (IgniteCheckedException e) {
+            if (logger() != null)
+                U.error(logger(), "Failed to send cancel request (will cancel query in any case).", e);

Review comment:
       Let's do not swallow exceptions in case of logger not exist.
   
   ```suggestion
               U.error(logger(), "Failed to send cancel request (will cancel query in any case).", e);
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -776,144 +686,99 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                null,
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
+    }
 
-        return fut;
+    /**
+     * Gets topic for ordered response messages.
+     *
+     * @param nodeId Node ID.
+     * @param reqId Request ID.
+     * @return Topic.
+     */
+    private Object topic(UUID nodeId, long reqId) {
+        return TOPIC_CACHE.topic(TOPIC_PREFIX, nodeId, reqId);
     }
 
     /**
      * Sends query request.
      *
-     * @param fut Distributed future.
+     * @param fut Cache query future. {@code null} in case of cancel request.
      * @param req Request.
      * @param nodes Nodes.
      * @throws IgniteCheckedException In case of error.
      */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
+    public void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes
     ) throws IgniteCheckedException {
-        assert fut != null;
         assert req != null;
         assert nodes != null;
 
-        final UUID locNodeId = cctx.localNodeId();
+        UUID locNodeId = cctx.localNodeId();
 
-        ClusterNode locNode = null;
+        boolean loc = false;
 
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
+        for (UUID nodeId : nodes) {
+            if (nodeId.equals(locNodeId))
+                loc = true;
             else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
-
-                rmtNodes.add(n);
-            }
-        }
-
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
-
-                        if (fut.isDone())
-                            return;
-                    }
-                    else
-                        throw e;
-                }
+                if (req.cancel())
+                    sendNodeCancelRequest(nodeId, req);
+                else if (!sendNodePageRequest(nodeId, req, fut))
+                    return;
             }
         }
 
-        if (locNode != null) {
+        if (loc) {
             cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
                 @Override public Object call() throws Exception {
                     req.beforeLocalExecution(cctx);
 
-                    processQueryRequest(locNodeId, req);
+                    processQueryRequest(cctx.localNodeId(), req);
 
                     return null;
                 }
             }, GridIoPolicy.QUERY_POOL);
         }
     }
 
+    /** */
+    private void sendNodeCancelRequest(UUID nodeId, GridCacheQueryRequest req) throws IgniteCheckedException {

Review comment:
       Why do we need this change? I don't see any reason having both of methods `sendNodePageRequest` and `sendNodeCancelRequest`. The previous implementations seem to be good. More over the `checkNodeLeft` seems to be redundant since handling the discovery messages do the same thing.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -17,257 +17,172 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.util.lang.GridPlainCallable;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.processors.cache.query.reducer.MergeSortDistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.reducer.NodePageStream;
+import org.apache.ignite.internal.processors.cache.query.reducer.UnsortedDistributedCacheQueryReducer;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
+
 /**
  * Distributed query future.
  */
 public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutureAdapter<K, V, R> {
     /** */
-    private static final long serialVersionUID = 0L;
+    private final long reqId;
 
-    /** */
-    private long reqId;
+    /** Helps to send cache query requests to other nodes. */
+    private final GridCacheDistributedQueryManager<K, V> qryMgr;
 
-    /** */
-    private final Collection<UUID> subgrid = new HashSet<>();
+    /** Collection of streams. */
+    private final Map<UUID, NodePageStream<R>> streams;
 
-    /** */
-    private final Collection<UUID> rcvd = new HashSet<>();
+    /** Count of streams that finish receiving remote pages. */
+    private final AtomicInteger noRemotePagesStreamsCnt = new AtomicInteger();
 
-    /** */
-    private CountDownLatch firstPageLatch = new CountDownLatch(1);
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Set of nodes that deliver their first page. */
+    private Set<UUID> rcvdFirstPage = ConcurrentHashMap.newKeySet();
 
     /**
      * @param ctx Cache context.
      * @param reqId Request ID.
      * @param qry Query.
-     * @param nodes Nodes.
      */
-    protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId, GridCacheQueryBean qry,
-        Iterable<ClusterNode> nodes) {
+    protected GridCacheDistributedQueryFuture(
+        GridCacheContext<K, V> ctx,
+        long reqId,
+        GridCacheQueryBean qry,
+        Collection<ClusterNode> nodes
+    ) {
         super(ctx, qry, false);
 
         assert reqId > 0;
 
         this.reqId = reqId;
 
-        GridCacheQueryManager<K, V> mgr = ctx.queries();
+        qryMgr = (GridCacheDistributedQueryManager<K, V>) ctx.queries();
 
-        assert mgr != null;
+        streams = new ConcurrentHashMap<>(nodes.size());
 
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
-    }
+        for (ClusterNode node : nodes) {
+            NodePageStream<R> s = new NodePageStream<>(node.id(), () -> requestPages(node.id()), () -> cancelPages(node.id()));

Review comment:
       ```suggestion
               streams.computeIfAbsent(node.id(), k -> new NodePageStream<>(k, () -> requestPages(k), () -> cancelPages(k)));
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
##########
@@ -1219,12 +1199,13 @@ protected void runQuery(GridCacheQueryInfo qryInfo) {
                     if (type == SCAN || type == INDEX)
                         // Scan iterator may return already transformed entry
                         data.add(row0);
+

Review comment:
       Redundant newline. 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -143,43 +126,62 @@ public GridCacheQueryBean query() {
         return qry;
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteLogger logger() {
+        return log;
+    }
+
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
+
+            checkError();
+
+            R next = null;
 
-            cnt.decrementAndGet();
+            if (reducer.hasNext()) {
+                next = unmaskNull(reducer.next());
+
+                if (!limitDisabled) {
+                    cnt++;
+
+                    // Exceed limit, stop page loading and cancel queries.
+                    if (cnt == capacity)
+                        cancel();
+                }
+            }
 
             return next;
         }
-        catch (NoSuchElementException ignored) {
-            return null;
-        }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
         }
     }
 
     /**
-     * Waits for the first page to be received from remote node(s), if any.
+     * @return Cache context.
+     */
+    public GridCacheContext<K, V> cacheContext() {
+        return cctx;
+    }
+
+    /**
+     * TODO: remote after IGNITE-15728.

Review comment:
       ```suggestion
        * TODO: IGNITE-15728 Provide custom reducer for ScanQueryFallbackClosableIterator.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -583,17 +518,25 @@ else if (!cancelled.contains(res.requestId()))
                 }
             });
 
-            sendRequest(fut, req, nodes);
+            futs.put(reqId, fut);
+
+            if (cctx.kernalContext().clientDisconnected()) {
+                IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(
+                    cctx.kernalContext().cluster().clientReconnectFuture(),
+                    "Query was cancelled, client node disconnected.");
+
+                fut.onError(err);
+            }

Review comment:
       ```suggestion
               if (cctx.kernalContext().clientDisconnected()) {
                   throw new IgniteClientDisconnectedCheckedException(cctx.kernalContext().cluster().clientReconnectFuture(),
                       "Query was cancelled, client node disconnected.");
               }
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -17,257 +17,172 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.util.lang.GridPlainCallable;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.processors.cache.query.reducer.MergeSortDistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.reducer.NodePageStream;
+import org.apache.ignite.internal.processors.cache.query.reducer.UnsortedDistributedCacheQueryReducer;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
+
 /**
  * Distributed query future.
  */
 public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutureAdapter<K, V, R> {
     /** */
-    private static final long serialVersionUID = 0L;
+    private final long reqId;
 
-    /** */
-    private long reqId;
+    /** Helps to send cache query requests to other nodes. */
+    private final GridCacheDistributedQueryManager<K, V> qryMgr;
 
-    /** */
-    private final Collection<UUID> subgrid = new HashSet<>();
+    /** Collection of streams. */
+    private final Map<UUID, NodePageStream<R>> streams;
 
-    /** */
-    private final Collection<UUID> rcvd = new HashSet<>();
+    /** Count of streams that finish receiving remote pages. */
+    private final AtomicInteger noRemotePagesStreamsCnt = new AtomicInteger();
 
-    /** */
-    private CountDownLatch firstPageLatch = new CountDownLatch(1);
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Set of nodes that deliver their first page. */
+    private Set<UUID> rcvdFirstPage = ConcurrentHashMap.newKeySet();
 
     /**
      * @param ctx Cache context.
      * @param reqId Request ID.
      * @param qry Query.
-     * @param nodes Nodes.
      */
-    protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId, GridCacheQueryBean qry,
-        Iterable<ClusterNode> nodes) {
+    protected GridCacheDistributedQueryFuture(
+        GridCacheContext<K, V> ctx,
+        long reqId,
+        GridCacheQueryBean qry,
+        Collection<ClusterNode> nodes
+    ) {
         super(ctx, qry, false);
 
         assert reqId > 0;
 
         this.reqId = reqId;
 
-        GridCacheQueryManager<K, V> mgr = ctx.queries();
+        qryMgr = (GridCacheDistributedQueryManager<K, V>) ctx.queries();
 
-        assert mgr != null;
+        streams = new ConcurrentHashMap<>(nodes.size());
 
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
-    }
+        for (ClusterNode node : nodes) {
+            NodePageStream<R> s = new NodePageStream<>(node.id(), () -> requestPages(node.id()), () -> cancelPages(node.id()));
 
-    /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
+            streams.put(node.id(), s);
 
-        assert qryMgr != null;
+            startQuery(node.id());

Review comment:
       Can we avoid starting anything from the class constructor? I don't think this is truly matters here but looks like a bad design.
   
   Probably you should call the `execute` method after the future instantiation like it for the `GridCacheLocalQueryFuture#execute` happens.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
##########
@@ -59,29 +67,38 @@ void execute() {
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
+    @Override protected void cancelQuery(Throwable err) throws IgniteCheckedException {
         if (fut != null)
             fut.cancel();
-    }
 
-    /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        return last;
+        stream.cancel(err);
     }
 
     /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        // No-op.
+    @Override public void awaitFirstItemAvailable() throws IgniteCheckedException {
+        try {
+            stream.headPage().get();

Review comment:
       Let's use the static `CacheQueryReducer#get` method as I suggested earlier.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -523,57 +488,27 @@ else if (!cancelled.contains(res.requestId()))
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
     @Override public CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, false);
+    }
+
+    /** */
+    private CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes, boolean fields) {

Review comment:
       Let's change the signature and method name to this one? This won't require complex future instantiation depending on incoming arguments.
   
   ```suggestion
       private CacheQueryFuture<?> registerAndStartDistributedQuery(GridCacheDistributedQueryFuture fut) {
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/CacheQueryReducer.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridIteratorAdapter;
+
+/**
+ * This abstract class is base class for cache query reducers. They are responsible for reducing results of cache query.
+ *
+ * <T> is a type of cache query result item.
+ */
+public abstract class CacheQueryReducer<T> extends GridIteratorAdapter<T> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Page streams collection. */
+    protected final Map<UUID, NodePageStream<T>> pageStreams;
+
+    /** */
+    protected CacheQueryReducer(final Map<UUID, NodePageStream<T>> pageStreams) {
+        this.pageStreams = pageStreams;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeX() throws IgniteCheckedException {
+        throw new UnsupportedOperationException("CacheQueryReducer doesn't support removing items.");
+    }
+
+    /**
+     * @return Page with query results data from specified stream.
+     */
+    protected NodePage<T> page(CompletableFuture<?> pageFut) throws IgniteCheckedException {

Review comment:
       Suggets changing the method signature to:
   
   ```suggestion
       public static <N> NodePage<N> get(CompletableFuture<NodePage<N>> pageFut) throws IgniteCheckedException {
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -143,43 +126,62 @@ public GridCacheQueryBean query() {
         return qry;
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteLogger logger() {
+        return log;
+    }
+
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
+
+            checkError();

Review comment:
       Let's inline this method.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -523,57 +488,27 @@ else if (!cancelled.contains(res.requestId()))
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
     @Override public CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, false);
+    }
+
+    /** */
+    private CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes, boolean fields) {
         assert cctx.config().getCacheMode() != LOCAL;
 
         if (log.isDebugEnabled())
             log.debug("Executing distributed query: " + qry);
 
         long reqId = cctx.io().nextIoId();
 
-        final GridCacheDistributedQueryFuture<K, V, ?> fut =
-            new GridCacheDistributedQueryFuture<>(cctx, reqId, qry, nodes);
+        final GridCacheDistributedQueryFuture fut = fields ?
+            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes) :
+            new GridCacheDistributedQueryFuture(cctx, reqId, qry, nodes);
 
         try {
-            qry.query().validate();
-
-            String clsName = qry.query().queryClassName();
-            Boolean dataPageScanEnabled = qry.query().isDataPageScanEnabled();
-            MvccSnapshot mvccSnapshot = qry.query().mvccSnapshot();
-
-            boolean deployFilterOrTransformer = (qry.query().scanFilter() != null || qry.query().transform() != null
-                || qry.query().idxQryDesc() != null) && cctx.gridDeploy().enabled();
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                false,
-                qry.query().clause(),
-                qry.query().idxQryDesc(),
-                qry.query().limit(),
-                clsName,
-                qry.query().scanFilter(),
-                qry.query().partition(),
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                false,
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                mvccSnapshot,
-                // Force deployment anyway if scan query is used.
-                cctx.deploymentEnabled() || deployFilterOrTransformer,
-                dataPageScanEnabled);
+            fut.qry.query().validate();

Review comment:
       Should we also call the `fut.qry.query().execute();` method here? (the same way as it for the `GridCacheLocalQueryFuture` happens?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
##########
@@ -59,29 +67,38 @@ void execute() {
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
+    @Override protected void cancelQuery(Throwable err) throws IgniteCheckedException {
         if (fut != null)
             fut.cancel();
-    }
 
-    /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        return last;
+        stream.cancel(err);
     }
 
     /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        // No-op.
+    @Override public void awaitFirstItemAvailable() throws IgniteCheckedException {
+        try {
+            stream.headPage().get();
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteCheckedException("Query was interrupted.", e);
+        }
+        catch (ExecutionException e) {
+            throw new IgniteCheckedException(e);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void loadAllPages() {
-        // No-op.
+    @Override protected void onError(Throwable err) {
+        onDone(err);
+
+        stream.cancel(err);

Review comment:
       ```suggestion
           if (onDone(err))
               stream.cancel(err);
   ```




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642433260



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {
+            if (!loadAllowed)
+                return;
+
+            nodes = new ArrayList<>();
+
+            for (int i = streamOff; i < streams.length; i++) {
+                UUID nodeId = streams[i].nodeId;
+
+                // Try to contain 2 pages for every stream to avoid waits. A node has to be present in collections:
+                // 1. rcvd - as requested and already handled before, so currently no parallel request to this node.
+                // 2. subgrid - as this node still has pages to request.
+                if (streams[i].queue.size() < 1 && rcvd.remove(nodeId) && subgrid.contains(nodeId))
+                    nodes.add(nodeId);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        boolean qryLast = super.onPage(nodeId, last);
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        if (last && streamsMap.containsKey(nodeId)) {
+            streamsMap.get(nodeId).allPagesReady = true;
+
+            streamsMap.remove(nodeId);
+        }
+
+        return qryLast;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        assert Thread.holdsLock(queueLock());

Review comment:
       Why do we care about external synchronization?
   Seems,  it is enough to avoid NPE after streamsMap.get(nodeId).
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682455470



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered via callback
+ * {@link #onPage(UUID, Collection, boolean)}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Callback that invoked on receiving a new page.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.
+     */
+    public boolean onPage(UUID nodeId, Collection<T> data, boolean last);
+
+    /**
+     * Callback in case of page with error.
+     */
+    public void onError();
+
+    /**
+     * Callback that invokes after a query future is done.
+     */
+    public void onFinish();
+
+    /**
+     * Callback is invoked on the query cancellation.
+     */
+    public void onCancel();

Review comment:
       It is not actually callback.
   ```suggestion
       public void cancel();
   ```




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682484307



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)
+            );
+
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head(), o2.head());
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId());
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId());
+                streams[i] = null;
+                streamOff++;
+            }
+        }
+
+        if (streamOff == streams.length)
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(UUID nodeId, Collection<R> data, boolean last) {
+        NodePageStream<R> stream = streamsMap.get(nodeId);
+
+        if (stream == null)
+            return streamsMap.isEmpty();
+
+        stream.addPage(nodeId, data, last);
+
+        return last && (streamsMap.remove(nodeId) != null) && streamsMap.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.onError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCancel() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.cancel(ns -> pageRequester.cancelQueryRequest(reqId, ns, fut.fields()));
+
+        streamsMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean mapNode(UUID nodeId) {
+        return streamsMap.containsKey(nodeId);
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param cmp Comparator.
+     */
+    private static <Z> void bubbleUp(Z[] arr, int off, Comparator<Z> cmp) {

Review comment:
       Why don't use logN algorithm here?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642463364



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;

Review comment:
       AFAIK, at the very first call, first pages will be requested from all data nodes.
   The next data will be got from the head.
   When the first page data will be consumed, then the next page will be requested synchronously.
   
   Thus, prefetch only works for the first nodes pages only.
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645101172



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream(node.id());
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {

Review comment:
       You use the same lock to synchronize access to streams[] and all queues in all NodeStrems objects.
   What is the purpose to use a shared lock?




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r633572949



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryResultFetcher.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public abstract class CacheQueryResultFetcher {
+    /**
+     * Map (requestId -> query future) where request id is unique for all requests per query.
+     * This map is populated by query manager.
+     */
+    private final ConcurrentMap<Long, GridCacheDistributedQueryFuture<?, ?, ?>> qryFuts;
+
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** */
+    CacheQueryResultFetcher(
+        final GridCacheContext cctx,
+        final ConcurrentMap<Long, GridCacheDistributedQueryFuture<?, ?, ?>> qryFuts) {

Review comment:
       CacheQueryResultFetcher doesn't look like good class name.
   1. Actually, it is not fetcher for particular query, it is a holder for object-fetchers for multiple queries.
   2. Why it depends on qryFuts?
   3. The functionality of the class is limited to create-and-send requests, but any response processing is out of scope...
   So, it is a 'requester', but not 'fetcher'




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r633556166



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -169,103 +97,30 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    @Override protected boolean onPage(@Nullable UUID nodeId, boolean last) {
+        return reducer.onPage(nodeId, last);
     }
 
     /** {@inheritDoc} */
     @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+        reducer.loadAll();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override protected Reducer<R> reducer() {
+        return reducer;
     }
 
     /** {@inheritDoc} */
     @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         return super.onCancelled();
     }
 
     /** {@inheritDoc} */
     @Override public void onTimeout() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         super.onTimeout();
     }

Review comment:
       Do we need all these callbacks in QueryFutureAdapter just to delegate calls to reducer
   and having reducer() at same time?
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r654284588



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return Head of stream and then clean it.
+     */
+    public R get() {
+        R ret = head;
+
+        head = null;
+
+        return ret;
+    }

Review comment:
       `
   @Override public boolean hasNext() throws IgniteCheckedException {
           if (head != null)
               return true;
          else {
               if (!super.hasNext())
                   return false;
               else
                   head = super.next();   
          }
   }
   `

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return Head of stream and then clean it.
+     */
+    public R get() {
+        R ret = head;
+
+        head = null;
+
+        return ret;
+    }

Review comment:
       ```
   @Override public boolean hasNext() throws IgniteCheckedException {
           if (head != null)
               return true;
          else {
               if (!super.hasNext())
                   return false;
               else
                   head = super.next();   
          }
   }
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return Head of stream and then clean it.
+     */
+    public R get() {
+        R ret = head;
+
+        head = null;
+
+        return ret;
+    }

Review comment:
       ```
   @Override public boolean hasNext() throws IgniteCheckedException {
           if (head != null)
               return true;
          else {
               if (!super.hasNext())
                   return false;
               else {
                   head = super.next();
                   
                   assert head != null;
                   
                   return truel
               }
          }
   }
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return Head of stream and then clean it.
+     */
+    public R get() {
+        R ret = head;
+
+        head = null;
+
+        return ret;
+    }

Review comment:
       ```
   @Override public boolean hasNext() throws IgniteCheckedException {
           if (head != null)
               return true;
          else {
               if (!super.hasNext())
                   return false;
               else
                   return (head = super.next()) != null;                           
          }
   }
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r695556485



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered via callback
+ * {@link #onPage(UUID, Collection, boolean)}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Callback that invoked on receiving a new page.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.
+     */
+    public boolean onPage(UUID nodeId, Collection<T> data, boolean last);
+
+    /**
+     * Callback in case of page with error.
+     */
+    public void onError();
+
+    /**
+     * Callback that invokes after a query future is done.
+     */
+    public void onFinish();

Review comment:
       We can move `onFinish()` from `onDone()` and explicity invoke it, but there are about 5-6 places to invoke this method - after every `onDone()`. Looks wrong to use this method such way




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#issuecomment-859468662


   Hi @AMashenkov ! I've answered on all comments (2 changes are accepted, other on discussion), could you please have a look?


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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r637832326



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -169,103 +97,30 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    @Override protected boolean onPage(@Nullable UUID nodeId, boolean last) {
+        return reducer.onPage(nodeId, last);
     }
 
     /** {@inheritDoc} */
     @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+        reducer.loadAll();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override protected Reducer<R> reducer() {
+        return reducer;
     }
 
     /** {@inheritDoc} */
     @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         return super.onCancelled();
     }
 
     /** {@inheritDoc} */
     @Override public void onTimeout() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         super.onTimeout();
     }

Review comment:
       @AMashenkov hi! Thanks for have a look this PR. Agree, I'll move it to the base class (`GridCacheQueryFutureAdapter`), there are additional functionality on this callbacks methods (onCancelled, onTimeout)
   
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655449087



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -101,12 +104,30 @@
     /** Event listener. */
     private GridLocalEventListener lsnr;
 
+    /** Requester of cache query result pages. */
+    private CacheQueryPageRequester pageRequester;
+
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() != LOCAL;
 
+        pageRequester = new CacheQueryPageRequester(cctx) {
+            /** {@inheritDoc} */

Review comment:
       ```suggestion
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682624157



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -69,205 +53,69 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
         GridCacheQueryManager<K, V> mgr = ctx.queries();
 
         assert mgr != null;
-
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
-
-        assert qryMgr != null;
-
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
-
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
+    @Override protected void cancelQuery() {
+        reducer.onCancel();
 
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
-
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
     @Override protected void onNodeLeft(UUID nodeId) {
-        boolean callOnPage;
-
-        synchronized (this) {
-            callOnPage = !loc && subgrid.contains(nodeId);
-        }
+        boolean qryNode = reducer.mapNode(nodeId);
 
-        if (callOnPage)
-            onPage(nodeId, Collections.emptyList(),
+        if (qryNode)
+            onPage(nodeId, null,
                 new ClusterTopologyCheckedException("Remote node has left topology: " + nodeId), true);
     }
 
     /** {@inheritDoc} */
-    @Override public void awaitFirstPage() throws IgniteCheckedException {
-        try {
-            firstPageLatch.await();
+    @Override public void awaitFirstItemAvailable() throws IgniteCheckedException {
+        reducer.awaitInitialization();
 
-            if (isDone() && error() != null)
-                // Throw the exception if future failed.
-                get();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
+        if (isDone() && error() != null)
+            // Throw the exception if future failed.
+            get();
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
+    @Override protected CacheQueryReducer<R> reducer() {
+        return reducer;
     }
 
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    /** Set reducer. */
+    void reducer(DistributedCacheQueryReducer<R> reducer) {
+        this.reducer = reducer;
     }
 
     /** {@inheritDoc} */
-    @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+    @Override public Collection<R> get() throws IgniteCheckedException {
+        return get0();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteCheckedException {
+        return get0();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
-
-        return super.onCancelled();
+    @Override public Collection<R> getUninterruptibly() throws IgniteCheckedException {
+        return get0();
     }
 
-    /** {@inheritDoc} */
-    @Override public void onTimeout() {
-        firstPageLatch.countDown();
+    /**
+     * Completion of distributed query future depends on user that iterates over query result with lazy page loading.
+     * Then {@link #get()} can lock on unpredictably long period of time. So we should avoid call it.
+     */
+    private Collection<R> get0() throws IgniteCheckedException {
+        if (!isDone())
+            throw new IgniteIllegalStateException("Unexpected lock on iterator over distributed cache query result.");

Review comment:
       The only place we use get() it's in this class, some lines above.
   
   https://github.com/apache/ignite/blob/0add93b488ebe7265b2a0f951fcdfd55adbcc01f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java#L82
   
   But I agree, that it a little bit misleading. So I will replace this line of code with `super.get()` and then we can forbid `get()` at all.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726931325



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -17,257 +17,172 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.util.lang.GridPlainCallable;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.processors.cache.query.reducer.MergeSortDistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.reducer.NodePageStream;
+import org.apache.ignite.internal.processors.cache.query.reducer.UnsortedDistributedCacheQueryReducer;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
+
 /**
  * Distributed query future.
  */
 public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutureAdapter<K, V, R> {
     /** */
-    private static final long serialVersionUID = 0L;
+    private final long reqId;
 
-    /** */
-    private long reqId;
+    /** Helps to send cache query requests to other nodes. */
+    private final GridCacheDistributedQueryManager<K, V> qryMgr;
 
-    /** */
-    private final Collection<UUID> subgrid = new HashSet<>();
+    /** Collection of streams. */
+    private final Map<UUID, NodePageStream<R>> streams;
 
-    /** */
-    private final Collection<UUID> rcvd = new HashSet<>();
+    /** Count of streams that finish receiving remote pages. */
+    private final AtomicInteger noRemotePagesStreamsCnt = new AtomicInteger();
 
-    /** */
-    private CountDownLatch firstPageLatch = new CountDownLatch(1);
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Set of nodes that deliver their first page. */
+    private Set<UUID> rcvdFirstPage = ConcurrentHashMap.newKeySet();
 
     /**
      * @param ctx Cache context.
      * @param reqId Request ID.
      * @param qry Query.
-     * @param nodes Nodes.
      */
-    protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId, GridCacheQueryBean qry,
-        Iterable<ClusterNode> nodes) {
+    protected GridCacheDistributedQueryFuture(
+        GridCacheContext<K, V> ctx,
+        long reqId,
+        GridCacheQueryBean qry,
+        Collection<ClusterNode> nodes
+    ) {
         super(ctx, qry, false);
 
         assert reqId > 0;
 
         this.reqId = reqId;
 
-        GridCacheQueryManager<K, V> mgr = ctx.queries();
+        qryMgr = (GridCacheDistributedQueryManager<K, V>) ctx.queries();
 
-        assert mgr != null;
+        streams = new ConcurrentHashMap<>(nodes.size());
 
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
-    }
+        for (ClusterNode node : nodes) {
+            NodePageStream<R> s = new NodePageStream<>(node.id(), () -> requestPages(node.id()), () -> cancelPages(node.id()));
 
-    /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
+            streams.put(node.id(), s);
 
-        assert qryMgr != null;
+            startQuery(node.id());

Review comment:
       Use `#startQuery` for start query. `#execute` is OK for local future as it naturally executes full query locally. For distributed query we can just start query on remote nodes.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726947639



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -776,144 +686,99 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                null,
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
+    }
 
-        return fut;
+    /**
+     * Gets topic for ordered response messages.
+     *
+     * @param nodeId Node ID.
+     * @param reqId Request ID.
+     * @return Topic.
+     */
+    private Object topic(UUID nodeId, long reqId) {
+        return TOPIC_CACHE.topic(TOPIC_PREFIX, nodeId, reqId);
     }
 
     /**
      * Sends query request.
      *
-     * @param fut Distributed future.
+     * @param fut Cache query future. {@code null} in case of cancel request.
      * @param req Request.
      * @param nodes Nodes.
      * @throws IgniteCheckedException In case of error.
      */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
+    public void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes
     ) throws IgniteCheckedException {
-        assert fut != null;
         assert req != null;
         assert nodes != null;
 
-        final UUID locNodeId = cctx.localNodeId();
+        UUID locNodeId = cctx.localNodeId();
 
-        ClusterNode locNode = null;
+        boolean loc = false;
 
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
+        for (UUID nodeId : nodes) {
+            if (nodeId.equals(locNodeId))
+                loc = true;
             else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
-
-                rmtNodes.add(n);
-            }
-        }
-
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
-
-                        if (fut.isDone())
-                            return;
-                    }
-                    else
-                        throw e;
-                }
+                if (req.cancel())
+                    sendNodeCancelRequest(nodeId, req);
+                else if (!sendNodePageRequest(nodeId, req, fut))
+                    return;
             }
         }
 
-        if (locNode != null) {
+        if (loc) {
             cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
                 @Override public Object call() throws Exception {
                     req.beforeLocalExecution(cctx);
 
-                    processQueryRequest(locNodeId, req);
+                    processQueryRequest(cctx.localNodeId(), req);
 
                     return null;
                 }
             }, GridIoPolicy.QUERY_POOL);
         }
     }
 
+    /** */
+    private void sendNodeCancelRequest(UUID nodeId, GridCacheQueryRequest req) throws IgniteCheckedException {
+        try {
+            cctx.io().send(nodeId, req, GridIoPolicy.QUERY_POOL);
+        }
+        catch (IgniteCheckedException e) {
+            if (cctx.io().checkNodeLeft(nodeId, e, false)) {

Review comment:
       Let's save this, it's old behavior. There could be cases, when one thread tries to request page, another thread concurrently accepts info about node left. And this affects what we should do, log error or raise an exception. Let's make this in separate ticket, WDYT?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r725850692



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * This class provides an interface {@link #nextPage()} that returns a {@link NodePage} of cache query result from
+ * single node. A new page requests when previous page was fetched by class consumer.
+ */
+public class NodePageStream<R> {
+    /** Node ID to stream pages */
+    private final UUID nodeId;
+
+    /** Flags shows whether there are no available pages on a query node. */
+    private boolean noRemotePages;
+
+    /** Last delivered page from the stream. */
+    private NodePage<R> head;
+
+    /** Promise to notify the stream consumer about delivering new page. */
+    private CompletableFuture<UUID> pageReady = new CompletableFuture<>();
+
+    /** This lock syncs {@link #head} and {@link #noRemotePages}. */
+    private final Object pagesLock = new Object();
+
+    /** Request pages action. */
+    private final Runnable reqPages;
+
+    /** */
+    public NodePageStream(UUID nodeId, Runnable reqPages) {
+        this.nodeId = nodeId;
+        this.reqPages = reqPages;
+    }
+
+    /** */
+    public UUID nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * @return Future that will be completed when a new page delivered.
+     */
+    public CompletableFuture<UUID> pageReady() {
+        return pageReady;
+    }
+
+    /**
+     * Returns a last delivered page from this stream. Note, that this method has to be invoked after getting
+     * the future provided with {@link #pageReady()}.
+     *
+     * @return Query result page.
+     */
+    public NodePage<R> nextPage() {
+        boolean loadPage = false;
+
+        NodePage<R> page;
+
+        synchronized (pagesLock) {
+            assert head != null;
+
+            page = head;
+
+            head = null;
+
+            if (!noRemotePages)
+                loadPage = true;
+        }
+
+        if (loadPage) {
+            pageReady = new CompletableFuture<>();
+
+            reqPages.run();
+        }
+
+        return page;
+    }
+
+    /**
+     * Add new query result page of data.
+     *
+     * @param data Collection of query result items.
+     * @param last Whether it is the last page from this node.
+     */
+    public void addPage(Collection<R> data, boolean last) {
+        synchronized (pagesLock) {
+            head = new NodePage<>(nodeId, data);
+
+            if (last)
+                noRemotePages = true;
+        }
+
+        pageReady.complete(nodeId);
+    }
+
+    /**
+     * Cancel query on all nodes.
+     */
+    public void cancel() {
+        synchronized (pagesLock) {
+            head = new NodePage<>(nodeId, Collections.emptyList());

Review comment:
       Future already completed there. This is a callback for clearing owned resources.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r729910269



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryLimitTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/** */
+@RunWith(Parameterized.class)
+public class GridCacheFullTextQueryLimitTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_PER_NODE_COUNT = 100;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Number of nodes. */
+    @Parameterized.Parameter(0)
+    public int nodesCnt;
+
+    /** */
+    @Parameterized.Parameters(name = "nodesCnt={0}")
+    public static Iterable<Object[]> params() {
+        List<Object[]> params = new ArrayList<>();
+
+        for (int i = 1; i <= 8; i++) {
+            Object[] p = new Object[1];
+            p[0] = i;
+
+            params.add(p);
+        }
+
+        return params;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName(PERSON_CACHE)
+            .setCacheMode(PARTITIONED)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testResultOrderedByScore() throws Exception {
+        startGrids(nodesCnt);
+
+        int items = MAX_ITEM_PER_NODE_COUNT * nodesCnt;
+
+        final IgniteEx ignite = grid(0);
+
+        IgniteCache<Integer, Person> cache = ignite.cache(PERSON_CACHE);
+
+        int helloPivot = new Random().nextInt(items);
+
+        for (int i = 0; i < items; i++) {
+            String name = i == helloPivot ? "hello" : String.format("hello%02d", i);
+
+            cache.put(i, new Person(name));
+        }
+
+        // Extract all data that matches even little.
+        for (int limit = 1; limit <= items; limit++) {
+            TextQuery qry = new TextQuery<>(Person.class, "hello~")
+                .setLimit(limit);
+
+            List<Cache.Entry<Integer, Person>> result = cache.query(qry).getAll();
+
+            // Lucene returns top 50 results by query from single node even if limit is higher.
+            // Number of docs depends on amount of data on every node.
+            if (limit <= 50)
+                assertEquals(limit, result.size());
+            else
+                assertTrue(limit >= result.size());
+
+            // hello has to be on the top.
+            assertEquals("Limit=" + limit, "hello", result.get(0).getValue().name);
+        }
+    }
+
+    /**
+     * Test model class.
+     */
+    public static class Person implements Serializable {

Review comment:
       It's will be only the Person class and `getConfiguration()` method. 

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryLimitTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/** */
+@RunWith(Parameterized.class)
+public class GridCacheFullTextQueryLimitTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_PER_NODE_COUNT = 100;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Number of nodes. */
+    @Parameterized.Parameter(0)
+    public int nodesCnt;
+
+    /** */
+    @Parameterized.Parameters(name = "nodesCnt={0}")
+    public static Iterable<Object[]> params() {
+        List<Object[]> params = new ArrayList<>();
+
+        for (int i = 1; i <= 8; i++) {
+            Object[] p = new Object[1];
+            p[0] = i;
+
+            params.add(p);
+        }
+
+        return params;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName(PERSON_CACHE)
+            .setCacheMode(PARTITIONED)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testResultOrderedByScore() throws Exception {
+        startGrids(nodesCnt);
+
+        int items = MAX_ITEM_PER_NODE_COUNT * nodesCnt;
+
+        final IgniteEx ignite = grid(0);
+
+        IgniteCache<Integer, Person> cache = ignite.cache(PERSON_CACHE);
+
+        int helloPivot = new Random().nextInt(items);
+
+        for (int i = 0; i < items; i++) {
+            String name = i == helloPivot ? "hello" : String.format("hello%02d", i);
+
+            cache.put(i, new Person(name));
+        }
+
+        // Extract all data that matches even little.
+        for (int limit = 1; limit <= items; limit++) {
+            TextQuery qry = new TextQuery<>(Person.class, "hello~")
+                .setLimit(limit);
+
+            List<Cache.Entry<Integer, Person>> result = cache.query(qry).getAll();
+
+            // Lucene returns top 50 results by query from single node even if limit is higher.
+            // Number of docs depends on amount of data on every node.
+            if (limit <= 50)
+                assertEquals(limit, result.size());
+            else
+                assertTrue(limit >= result.size());
+
+            // hello has to be on the top.
+            assertEquals("Limit=" + limit, "hello", result.get(0).getValue().name);

Review comment:
       No, as it's pretty unpredictable. For string "hello" fully match the query "hello~" and it will have the highest score. Other strings match somehow, we don't have a function to calculate this score within a test.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryLimitTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/** */
+@RunWith(Parameterized.class)
+public class GridCacheFullTextQueryLimitTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_PER_NODE_COUNT = 100;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Number of nodes. */
+    @Parameterized.Parameter(0)
+    public int nodesCnt;
+
+    /** */
+    @Parameterized.Parameters(name = "nodesCnt={0}")
+    public static Iterable<Object[]> params() {
+        List<Object[]> params = new ArrayList<>();
+
+        for (int i = 1; i <= 8; i++) {
+            Object[] p = new Object[1];
+            p[0] = i;
+
+            params.add(p);
+        }
+
+        return params;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName(PERSON_CACHE)
+            .setCacheMode(PARTITIONED)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testResultOrderedByScore() throws Exception {
+        startGrids(nodesCnt);
+
+        int items = MAX_ITEM_PER_NODE_COUNT * nodesCnt;
+
+        final IgniteEx ignite = grid(0);
+
+        IgniteCache<Integer, Person> cache = ignite.cache(PERSON_CACHE);
+
+        int helloPivot = new Random().nextInt(items);
+
+        for (int i = 0; i < items; i++) {
+            String name = i == helloPivot ? "hello" : String.format("hello%02d", i);
+
+            cache.put(i, new Person(name));
+        }
+
+        // Extract all data that matches even little.
+        for (int limit = 1; limit <= items; limit++) {
+            TextQuery qry = new TextQuery<>(Person.class, "hello~")
+                .setLimit(limit);
+
+            List<Cache.Entry<Integer, Person>> result = cache.query(qry).getAll();
+
+            // Lucene returns top 50 results by query from single node even if limit is higher.
+            // Number of docs depends on amount of data on every node.
+            if (limit <= 50)

Review comment:
       Limit works correctly, as we control limit in a reducer. But lucene make a decision about number of strings to return by self (it's an internal configuration of Lucene). Lucene does it because there is no need to return to user results with low score, as they are irrelevant. 




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726948205



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -776,144 +686,99 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                null,
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
+    }
 
-        return fut;
+    /**
+     * Gets topic for ordered response messages.
+     *
+     * @param nodeId Node ID.
+     * @param reqId Request ID.
+     * @return Topic.
+     */
+    private Object topic(UUID nodeId, long reqId) {
+        return TOPIC_CACHE.topic(TOPIC_PREFIX, nodeId, reqId);
     }
 
     /**
      * Sends query request.
      *
-     * @param fut Distributed future.
+     * @param fut Cache query future. {@code null} in case of cancel request.
      * @param req Request.
      * @param nodes Nodes.
      * @throws IgniteCheckedException In case of error.
      */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
+    public void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes

Review comment:
       Code that uses collections was reverted. Then this suggestion isn't actual anymore.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726932080



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -143,43 +126,62 @@ public GridCacheQueryBean query() {
         return qry;
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteLogger logger() {
+        return log;
+    }
+
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
+
+            checkError();

Review comment:
       Let's save `checkError()` method, it's more friendly and it's already was before




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r725853902



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -143,43 +126,62 @@ public GridCacheQueryBean query() {
         return qry;
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteLogger logger() {
+        return log;
+    }
+
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
+
+            checkError();
+
+            R next = null;
 
-            cnt.decrementAndGet();
+            if (reducer.hasNext()) {
+                next = unmaskNull(reducer.next());
+
+                if (!limitDisabled) {
+                    cnt++;
+
+                    // Exceed limit, stop page loading and cancel queries.
+                    if (cnt == capacity)
+                        cancel();
+                }
+            }
+
+            checkError();

Review comment:
       In case of error we have in `checkError()` all page streams are already closed, as they have `cancel()` method that invokes automatically when future failed.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655461165



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return {@code true} If this stream has next row, {@code false} otherwise.
+     */

Review comment:
       ```suggestion
       /** {@inheritDoc} */
   ```




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642455784



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
+        Collection<ClusterNode> nodes;
+
+        synchronized (queueLock()) {
+            nodes = F.retain(allNodes, true,
+                new P1<ClusterNode>() {
+                    @Override public boolean apply(ClusterNode node) {
+                        return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
+                    }
+                }
+            );
+
+            rcvd.clear();
+            subgrid.clear();
+        }
+
+        pageRequester.cancelQueryRequest(reqId, nodes, fut.fields());
+
+        pageStream.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            // Loads only queue is empty to avoid memory consumption on additional pages.
+            if (!pageStream.queue.isEmpty())
+                return;
+
+            if (loadAllowed && rcvd.containsAll(subgrid)) {
+                rcvd.clear();
+
+                nodes = new ArrayList<>(subgrid);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadAll() throws IgniteInterruptedCheckedException {
+        assert !Thread.holdsLock(queueLock());
+
+        U.await(firstPageLatch);
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            if (loadAllowed && !subgrid.isEmpty())
+                nodes = new ArrayList<>(subgrid);
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        assert Thread.holdsLock(queueLock());
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        rcvd.add(nodeId);
+
+        if (!loadAllowed && rcvd.containsAll(subgrid) ) {
+            firstPageLatch.countDown();
+            loadAllowed = true;
+        }
+
+        return last && subgrid.remove(nodeId) && subgrid.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onNodeLeft(UUID nodeId) {
+        synchronized (queueLock()) {
+            return subgrid.contains(nodeId);

Review comment:
       Here we know the node was left.
   Is there any reason to return smth to higher-level instead of releasing a latch instantly?
   It is non-obvious if onPage will ever be called.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655446801



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Checks whether cache query still runs on specified node. If a query finished (send all pages) on this node, then
+     * this method has to return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if specified node runs this query.

Review comment:
       ```suggestion
        * Checks whether node with provided {@code nodeId} is a map node for the query.
        * Note: if all the pages were received this node, then the method will return {@code false}.
        *
        * @param nodeId Node ID.
        * @return {@code true} if specified node is a map node for the query, {@code false} otherwise.
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r685276411



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)
+            );
+
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head(), o2.head());
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId());
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId());
+                streams[i] = null;
+                streamOff++;
+            }
+        }
+
+        if (streamOff == streams.length)
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(UUID nodeId, Collection<R> data, boolean last) {
+        NodePageStream<R> stream = streamsMap.get(nodeId);
+
+        if (stream == null)
+            return streamsMap.isEmpty();
+
+        stream.addPage(nodeId, data, last);
+
+        return last && (streamsMap.remove(nodeId) != null) && streamsMap.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.onError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCancel() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.cancel(ns -> pageRequester.cancelQueryRequest(reqId, ns, fut.fields()));
+
+        streamsMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean mapNode(UUID nodeId) {
+        return streamsMap.containsKey(nodeId);
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param cmp Comparator.
+     */
+    private static <Z> void bubbleUp(Z[] arr, int off, Comparator<Z> cmp) {

Review comment:
       Now use PriorityQueue instead.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] korlov42 commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682463592



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)

Review comment:
       do we really need to create lambda for every node?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+public class UnsortedDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Single page stream. */
+    private final PageStream<R> pageStream;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    public UnsortedDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes) {
+

Review comment:
       empty line

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;

Review comment:
       array should be replaced with PriorityQueue

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/PageStream.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Iterator over pages stream. Pages are stored in a queue. After polling a queue try load a new page instead of it.
+ */
+class PageStream<R> {
+    /** Queue of data of results pages. */
+    protected final Queue<Collection<R>> queue = new LinkedList<>();
+
+    /** Iterator over current page. */
+    private Iterator<R> iter;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    private final Collection<UUID> subgrid;
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    private final Collection<UUID> rcvd = new HashSet<>();

Review comment:
       why do you decide to split reducer's logic between actual reducer class and this one?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] korlov42 commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682463592



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)

Review comment:
       do we really need to create lambda for every node?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+public class UnsortedDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Single page stream. */
+    private final PageStream<R> pageStream;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    public UnsortedDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes) {
+

Review comment:
       empty line

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;

Review comment:
       array should be replaced with PriorityQueue




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] korlov42 commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
korlov42 commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682484934



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/PageStream.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Iterator over pages stream. Pages are stored in a queue. After polling a queue try load a new page instead of it.
+ */
+class PageStream<R> {
+    /** Queue of data of results pages. */
+    protected final Queue<Collection<R>> queue = new LinkedList<>();
+
+    /** Iterator over current page. */
+    private Iterator<R> iter;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    private final Collection<UUID> subgrid;
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    private final Collection<UUID> rcvd = new HashSet<>();

Review comment:
       why do you decide to split reducer's logic between actual reducer class and this one?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645161916



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
+        Collection<ClusterNode> nodes;
+
+        synchronized (queueLock()) {
+            nodes = F.retain(allNodes, true,
+                new P1<ClusterNode>() {
+                    @Override public boolean apply(ClusterNode node) {
+                        return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
+                    }
+                }
+            );
+
+            rcvd.clear();
+            subgrid.clear();
+        }
+
+        pageRequester.cancelQueryRequest(reqId, nodes, fut.fields());
+
+        pageStream.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            // Loads only queue is empty to avoid memory consumption on additional pages.
+            if (!pageStream.queue.isEmpty())
+                return;
+
+            if (loadAllowed && rcvd.containsAll(subgrid)) {
+                rcvd.clear();
+
+                nodes = new ArrayList<>(subgrid);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadAll() throws IgniteInterruptedCheckedException {
+        assert !Thread.holdsLock(queueLock());
+
+        U.await(firstPageLatch);
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            if (loadAllowed && !subgrid.isEmpty())
+                nodes = new ArrayList<>(subgrid);
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        assert Thread.holdsLock(queueLock());
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        rcvd.add(nodeId);
+
+        if (!loadAllowed && rcvd.containsAll(subgrid) ) {
+            firstPageLatch.countDown();
+            loadAllowed = true;
+        }
+
+        return last && subgrid.remove(nodeId) && subgrid.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onNodeLeft(UUID nodeId) {
+        synchronized (queueLock()) {
+            return subgrid.contains(nodeId);

Review comment:
       You can have 2 separate calls instead.
   fut.onNodeLeft -> rdc.onNodeLeft 
   fut.onNodeLeft -> rdc.isFailed/isFinished




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645143648



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;
+
+    /**
+     * Callback to handle node left.
+     *
+     * @param nodeId Node ID that left a cluster.
+     * @return {@code true} if specified node runs this query.
+     */
+    public boolean onNodeLeft(UUID nodeId);
+
+    /** Blocks while reducer doesn't get first result item for this query. */
+    public void awaitFirstItem() throws InterruptedException;
+
+    /** Callback that invokes when this query is cancelled. */
+    public void cancel();

Review comment:
       ```suggestion
       /** 
       * Callback is invoked on the query cancellation. 
       */
       public void onCancel();
   ```




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645140194



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;

Review comment:
       Ok. One can forcibly load all the pages, but where will the data be returned or saved?




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726531132



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -776,144 +686,99 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                null,
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
+    }
 
-        return fut;
+    /**
+     * Gets topic for ordered response messages.
+     *
+     * @param nodeId Node ID.
+     * @param reqId Request ID.
+     * @return Topic.
+     */
+    private Object topic(UUID nodeId, long reqId) {
+        return TOPIC_CACHE.topic(TOPIC_PREFIX, nodeId, reqId);
     }
 
     /**
      * Sends query request.
      *
-     * @param fut Distributed future.
+     * @param fut Cache query future. {@code null} in case of cancel request.
      * @param req Request.
      * @param nodes Nodes.
      * @throws IgniteCheckedException In case of error.
      */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
+    public void sendRequest(
+        @Nullable GridCacheQueryFutureAdapter<?, ?, ?> fut,
+        GridCacheQueryRequest req,
+        Collection<UUID> nodes
     ) throws IgniteCheckedException {
-        assert fut != null;
         assert req != null;
         assert nodes != null;
 
-        final UUID locNodeId = cctx.localNodeId();
+        UUID locNodeId = cctx.localNodeId();
 
-        ClusterNode locNode = null;
+        boolean loc = false;
 
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
+        for (UUID nodeId : nodes) {
+            if (nodeId.equals(locNodeId))
+                loc = true;
             else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
-
-                rmtNodes.add(n);
-            }
-        }
-
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
-
-                        if (fut.isDone())
-                            return;
-                    }
-                    else
-                        throw e;
-                }
+                if (req.cancel())
+                    sendNodeCancelRequest(nodeId, req);
+                else if (!sendNodePageRequest(nodeId, req, fut))
+                    return;
             }
         }
 
-        if (locNode != null) {
+        if (loc) {
             cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
                 @Override public Object call() throws Exception {
                     req.beforeLocalExecution(cctx);
 
-                    processQueryRequest(locNodeId, req);
+                    processQueryRequest(cctx.localNodeId(), req);
 
                     return null;
                 }
             }, GridIoPolicy.QUERY_POOL);
         }
     }
 
+    /** */
+    private void sendNodeCancelRequest(UUID nodeId, GridCacheQueryRequest req) throws IgniteCheckedException {

Review comment:
       Difference is in error handling. For cancelling query we skip throwing an error and just log about it.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r648234854



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream(node.id());
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId);
+                streams[i] = null;
+                streamOff++;
+            }
+            else {
+                // Prefetch head value.
+                s.next();
+            }
+        }
+
+        if (finished())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId);
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        List<UUID> nodes;
+
+        synchronized (queueLock()) {

Review comment:
       I simplified a code, now NodePageStream doesn't have own logic for control nodes or page loading, it inherited from PageStream. So, reducers doesn't use shared lock now.
   
   But all streams shared the same lock, as it guards consistency between CacheQueryFuture state, queues and rcvd. In one moment only single stream requesting a pages (others do have `head` to use), so it doesn't affect performance, and work the same way as UnorderedReducer does.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655438939



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #onPage(UUID, Collection, boolean)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Offer query result page for reduce. Note that the data collection may contain extension of type T.
+     * In such cases data item contains additional payload for custom reducer logic.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.

Review comment:
       ```suggestion
        * Callback that invoked on receiving a new page. 
        *
        * @param nodeId Node ID that sent this page.
        * @param data Page data rows.
        * @param last Whether this page is last for specified {@code nodeId}.
        * @return {@code true} if this page is final page for query and no more pages are waited, {@code false} otherwise.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #onPage(UUID, Collection, boolean)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.

Review comment:
       ```suggestion
    * This class is responsible for reducing results of cache query. 
    *
    * Query results are delivered via callback {@link #onPage(UUID, Collection, boolean)}.
    * @see GridCacheQueryFutureAdapter.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #onPage(UUID, Collection, boolean)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.

Review comment:
       Actually, interface don't imply any coupling with the  query future.
   If you think that this worth to be mentioned, then abstract class or concrete implementation would be a better place.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Checks whether cache query still runs on specified node. If a query finished (send all pages) on this node, then
+     * this method has to return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if specified node runs this query.

Review comment:
       ```suggestion
        * Checks whether node with provided {@code nodeId} is a map node for the query.
        * Note: if all the pages were received this node, then the method will return {@code false}.
        *
        * @param nodeId Node ID.
        * @return {@code true} if specified node is a map node for the query, {@code false} otherwise.
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -69,205 +50,43 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
         GridCacheQueryManager<K, V> mgr = ctx.queries();
 
         assert mgr != null;
-
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
-
-        assert qryMgr != null;
-
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
-
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
-
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
+    @Override protected void cancelQuery() {
+        reducer.onCancel();
 
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
-    /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
+    /** Fail if a node runs this query left cluster. */

Review comment:
       ```suggestion
       /** {@inheritDoc} */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -101,12 +104,30 @@
     /** Event listener. */
     private GridLocalEventListener lsnr;
 
+    /** Requester of cache query result pages. */
+    private CacheQueryPageRequester pageRequester;
+
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() != LOCAL;
 
+        pageRequester = new CacheQueryPageRequester(cctx) {
+            /** {@inheritDoc} */

Review comment:
       ```suggestion
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -523,57 +544,31 @@ else if (!cancelled.contains(res.requestId()))
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
     @Override public CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, false);
+    }
+
+    /** */
+    private CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes, boolean fields) {
         assert cctx.config().getCacheMode() != LOCAL;
 
         if (log.isDebugEnabled())
             log.debug("Executing distributed query: " + qry);
 
         long reqId = cctx.io().nextIoId();
 
-        final GridCacheDistributedQueryFuture<K, V, ?> fut =
-            new GridCacheDistributedQueryFuture<>(cctx, reqId, qry, nodes);
+        final GridCacheDistributedQueryFuture fut = fields ?
+            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry)
+            : new GridCacheDistributedQueryFuture(cctx, reqId, qry);

Review comment:
       ```suggestion
               new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry) :
               new GridCacheDistributedQueryFuture(cctx, reqId, qry);
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -146,40 +137,50 @@ public GridCacheQueryBean query() {
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
 
-            cnt.decrementAndGet();
+            checkError();
+
+            R next = null;
+
+            if (reducer().hasNext()) {
+                next = unmaskNull(reducer().next());
+
+                if (!limitDisabled) {
+                    cnt++;
+
+                    // Exceed limit, stop page loading and cancel queries.
+                    if (cnt == capacity)
+                        cancel();
+                }
+            }
+
+            checkError();
 
             return next;
         }
-        catch (NoSuchElementException ignored) {
-            return null;
-        }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
         }
     }
 
+    /** @return Cache query results reducer. */

Review comment:
       ```suggestion
       /** 
       * @return Cache query results reducer. 
       */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -146,40 +137,50 @@ public GridCacheQueryBean query() {
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
 
-            cnt.decrementAndGet();
+            checkError();
+
+            R next = null;
+
+            if (reducer().hasNext()) {
+                next = unmaskNull(reducer().next());
+
+                if (!limitDisabled) {
+                    cnt++;
+
+                    // Exceed limit, stop page loading and cancel queries.
+                    if (cnt == capacity)
+                        cancel();
+                }
+            }
+
+            checkError();
 
             return next;
         }
-        catch (NoSuchElementException ignored) {
-            return null;
-        }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
         }
     }
 
+    /** @return Cache query results reducer. */
+    protected abstract CacheQueryReducer<R> reducer();
+
     /**
-     * Waits for the first page to be received from remote node(s), if any.
+     * Waits for the first item to be received from remote node(s), if any.
      *
      * @throws IgniteCheckedException If query execution failed with an error.
      */
-    public abstract void awaitFirstPage() throws IgniteCheckedException;
+    public abstract void awaitFirstItem() throws IgniteCheckedException;

Review comment:
       Maybe awaitFirstItemAvailable() will be better.
   Actually, we wait here not for the first item/page from remote, but the first item available after the reduction applied.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -306,10 +221,12 @@ else if (capacity > 0) {
     }
 
     /**
+     * Entrypoint for handling query result page from remote node.
+     *
      * @param nodeId Sender node.
      * @param data Page data.
      * @param err Error (if was).
-     * @param finished Finished or not.
+     * @param finished Whether it is the last page for sender node.

Review comment:
       Maybe 'lastPage' flag would be a better name?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {

Review comment:
       ```suggestion
       /**
       * Peek the stream head for the next item.
       *
       * Note: requires {@link #hasNext()} to be called at first.
       * @return The item will be returned with {@link #next()}.
       */
       R head() {
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {

Review comment:
       Let's make the class package visible if possible.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return {@code true} If this stream has next row, {@code false} otherwise.
+     */

Review comment:
       ```suggestion
       /** {@inheritDoc} */
   ```

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return {@code true} If this stream has next row, {@code false} otherwise.
+     */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        if (head != null)
+            return true;
+        else {
+            if (!super.hasNext())
+                return false;
+            else
+                return (head = super.next()) != null;
+        }
+    }
+
+    /**
+     * @return Next item from this stream.
+     */

Review comment:
       ```suggestion
       /** {@inheritDoc} */
   ```




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



[GitHub] [ignite] Mmuzaf merged pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf merged pull request #9081:
URL: https://github.com/apache/ignite/pull/9081


   


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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642607727



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
+        Collection<ClusterNode> nodes;
+
+        synchronized (queueLock()) {
+            nodes = F.retain(allNodes, true,
+                new P1<ClusterNode>() {
+                    @Override public boolean apply(ClusterNode node) {
+                        return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
+                    }
+                }
+            );
+
+            rcvd.clear();
+            subgrid.clear();
+        }
+
+        pageRequester.cancelQueryRequest(reqId, nodes, fut.fields());
+
+        pageStream.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void loadPage() {
+        assert !Thread.holdsLock(queueLock());
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            // Loads only queue is empty to avoid memory consumption on additional pages.
+            if (!pageStream.queue.isEmpty())
+                return;
+
+            if (loadAllowed && rcvd.containsAll(subgrid)) {
+                rcvd.clear();
+
+                nodes = new ArrayList<>(subgrid);
+            }
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadAll() throws IgniteInterruptedCheckedException {
+        assert !Thread.holdsLock(queueLock());
+
+        U.await(firstPageLatch);
+
+        Collection<UUID> nodes = null;
+
+        synchronized (queueLock()) {
+            if (loadAllowed && !subgrid.isEmpty())
+                nodes = new ArrayList<>(subgrid);
+        }
+
+        if (nodes != null)
+            pageRequester.requestPages(reqId, fut, nodes, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(@Nullable UUID nodeId, boolean last) {
+        assert Thread.holdsLock(queueLock());
+
+        if (nodeId == null)
+            nodeId = cctx.localNodeId();
+
+        rcvd.add(nodeId);
+
+        if (!loadAllowed && rcvd.containsAll(subgrid) ) {
+            firstPageLatch.countDown();
+            loadAllowed = true;
+        }
+
+        return last && subgrid.remove(nodeId) && subgrid.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onNodeLeft(UUID nodeId) {
+        synchronized (queueLock()) {
+            return subgrid.contains(nodeId);

Review comment:
       You're correct, it fut.onPage(error) can be invoked in this method. But it looks like as spaghetti. Stack of calls is:
   fut.onNodeLeft -> rdc.onNodeLeft -> fut.onPage(error) -> rdc.onPage. 
   
   It's not too much, but looks strange. 
   
   > It is non-obvious if onPage will ever be called.
   
   Why do you think? JavaDocs clearly covers return value of rdc.onNodeLeft.
   
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704364265



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryPageRequester.java
##########
@@ -0,0 +1,236 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public class CacheQueryPageRequester {
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** Local handler of cache query request. */
+    private final Consumer<GridCacheQueryRequest> sendLoc;
+
+    /**
+     * @param cctx Cache context.
+     * @param sendLoc Local handler of cache query request.
+     */
+    CacheQueryPageRequester(final GridCacheContext cctx, Consumer<GridCacheQueryRequest> sendLoc) {
+        this.cctx = cctx;
+        this.log = cctx.kernalContext().config().getGridLogger();
+        this.sendLoc = sendLoc;
+    }
+
+    /**
+     * Send initial query request to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param fut Cache query future, contains query info.
+     * @param nodes Collection of nodes to send a request.
+     */
+    public void initRequestPages(long reqId, GridCacheDistributedQueryFuture fut,
+        Collection<ClusterNode> nodes) throws IgniteCheckedException {
+        GridCacheQueryBean bean = fut.query();
+        GridCacheQueryAdapter qry = bean.query();
+
+        boolean deployFilterOrTransformer = (qry.scanFilter() != null || qry.transform() != null)
+            && cctx.gridDeploy().enabled();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.type(),
+            fut.fields(),
+            qry.clause(),
+            qry.limit(),
+            qry.queryClassName(),
+            qry.scanFilter(),
+            qry.partition(),
+            bean.reducer(),
+            qry.transform(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            bean.arguments(),
+            qry.includeMetadata(),
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            qry.mvccSnapshot(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || deployFilterOrTransformer,
+            qry.isDataPageScanEnabled());
+
+        sendRequest(fut, req, nodes);
+    }
+
+    /**
+     * Send request for fetching query result pages to specified nodes.
+     *
+     * @param reqId Request (cache query) ID.
+     * @param nodes Collection of nodes to send a request.
+     * @param all If {@code true} then request for all pages, otherwise for single only.
+     */
+    public void requestPages(long reqId, GridCacheQueryFutureAdapter fut, Collection<UUID> nodes, boolean all) {
+        GridCacheQueryAdapter qry = fut.query().query();
+
+        GridCacheQueryRequest req = new GridCacheQueryRequest(
+            cctx.cacheId(),
+            reqId,
+            cctx.name(),
+            qry.pageSize(),
+            qry.includeBackups(),
+            fut.fields(),
+            all,
+            qry.keepBinary(),
+            qry.taskHash(),
+            cctx.startTopologyVersion(),
+            // Force deployment anyway if scan query is used.
+            cctx.deploymentEnabled() || (qry.scanFilter() != null && cctx.gridDeploy().enabled()),
+            qry.isDataPageScanEnabled());
+
+        try {
+            Collection<ClusterNode> n = new ArrayList<>();
+            for (UUID id: nodes)
+                n.add(cctx.node(id));
+
+            sendRequest(fut, req, n);
+
+        } catch (IgniteCheckedException e) {
+            fut.onDone(e);
+        }
+    }
+
+    /**
+     * Send cancel query request, so no new pages will be sent.
+     *
+     * @param reqId Query request ID.
+     * @param nodes Collection of nodes to send the cancel request.
+     * @param fieldsQry Whether query is a fields query.
+     *
+     */
+    public void cancelQuery(long reqId, Collection<UUID> nodes, boolean fieldsQry) {
+        final GridCacheQueryManager qryMgr = cctx.queries();
+
+        assert qryMgr != null;
+
+        try {
+            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
+                reqId,
+                fieldsQry,
+                cctx.startTopologyVersion(),
+                cctx.deploymentEnabled());
+
+            for (UUID node : nodes) {
+                try {
+                    if (cctx.localNodeId().equals(node)) {
+                        sendLoc.accept(req);
+
+                        continue;
+                    }
+
+                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
+                }
+                catch (IgniteCheckedException e) {
+                    if (cctx.io().checkNodeLeft(node, e, false)) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to send cancel request, node failed: " + node);
+                    }
+                    else
+                        U.error(log, "Failed to send cancel request [node=" + node + ']', e);
+                }
+            }
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
+        }
+    }
+
+    /**
+     * Sends query request.
+     *
+     * @param req Request.
+     * @param nodes Nodes.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sendRequest(
+        final GridCacheQueryFutureAdapter fut,
+        final GridCacheQueryRequest req,
+        Collection<ClusterNode> nodes
+    ) throws IgniteCheckedException {
+        assert req != null;
+        assert nodes != null;
+
+        final UUID locNodeId = cctx.localNodeId();
+
+        ClusterNode locNode = null;
+
+        Collection<ClusterNode> rmtNodes = null;
+
+        for (ClusterNode n : nodes) {
+            if (n.id().equals(locNodeId))
+                locNode = n;
+            else {
+                if (rmtNodes == null)
+                    rmtNodes = new ArrayList<>(nodes.size());
+
+                rmtNodes.add(n);
+            }
+        }
+
+        // Request should be sent to remote nodes before the query is processed on the local node.
+        // For example, a remote reducer has a state, we should not serialize and then send
+        // the reducer changed by the local node.
+        if (!F.isEmpty(rmtNodes)) {
+            for (ClusterNode node : rmtNodes) {

Review comment:
       ```suggestion            
   ```




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642339704



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/AbstractCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Base abstract class for all Reducer descenders. It declares dependencies on cache query, and base logic for handling
+ * remote pages.
+ */
+public abstract class AbstractCacheQueryReducer<R> implements CacheQueryReducer<R> {
+    /** Query info. */
+    private final GridCacheQueryAdapter qry;
+
+    /** Flag shows whether all pages are ready. */
+    protected volatile boolean allPagesReady;
+
+    /** Timestamp when a query fails with timeout. */
+    private final long timeoutTime;
+
+    /** Lock shared between this reducer and future. */
+    private final Object sharedLock;
+
+    /**
+     * @param fut Cache query future relates to this query. Future is done when last page is delivered to reducer.
+     */
+    AbstractCacheQueryReducer(GridCacheQueryFutureAdapter fut) {
+        qry = fut.qry.query();
+        timeoutTime = fut.endTime();
+        // The only reason to use lock in 2 places is the deduplication mechanism.
+        sharedLock = fut.lock;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object sharedLock() {
+        return sharedLock;
+    }

Review comment:
       This looks like an abstraction leakage antipattern.
   Can it be protected?
   




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642425152



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {

Review comment:
       Should this method countdown firstPageLatch?




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704373832



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/AbstractDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.DistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Abstract class for distributed reducer implementations.
+ */
+abstract class AbstractDistributedCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /** Collection of streams that don't deliver all pages yet. */
+    private final Map<UUID, NodePageStream<R>> remoteStreams;

Review comment:
       Why we need a duplicate collection?
   Can `streams` be used instead?




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r705178320



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+public class UnsortedDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** */
+    private NodePage<R> page;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    public UnsortedDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes) {
+        super(fut, reqId, pageRequester, queueLock, nodes);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        if (page != null && page.hasNext())
+            return true;
+
+        Collection<NodePageStream<R>> streams = new ArrayList<>(this.streams.values());
+
+        for (NodePageStream<R> s: streams) {
+            page = s.nextPage();
+
+            if (page != null && page.hasNext())
+                return true;
+
+            this.streams.remove(s.nodeId());

Review comment:
       Remove clearing `streams` in cancel().




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r642922363



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/UnsortedDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distributed query, fetch pages from remote nodes. All pages go in single page stream so no ordering is provided.
+ */
+class UnsortedDistributedCacheQueryReducer<R> extends AbstractCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /**
+     * Whether it is allowed to send cache query result requests to nodes.
+     * It is set to {@code false} if a query finished or failed.
+     */
+    protected volatile boolean loadAllowed;
+
+    /** Query request ID. */
+    protected final long reqId;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    protected final Collection<UUID> subgrid = new HashSet<>();
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    protected final Collection<UUID> rcvd = new HashSet<>();
+
+    /** Requester of cache query result pages. */
+    protected final CacheQueryPageRequester pageRequester;
+
+    /** Cache context. */
+    protected final GridCacheContext cctx;
+
+    /** Count down this latch when every node responses on initial cache query request. */
+    private final CountDownLatch firstPageLatch = new CountDownLatch(1);
+
+    /** Single page stream. */
+    private final PageStream pageStream;
+
+    /** Query future. */
+    protected final GridCacheQueryFutureAdapter fut;
+
+    /**
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param nodes Collection of nodes this query applies to.
+     */
+    UnsortedDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Collection<ClusterNode> nodes) {
+        super(fut);
+
+        this.reqId = reqId;
+        this.pageRequester = pageRequester;
+
+        synchronized (queueLock()) {
+            for (ClusterNode node : nodes)
+                subgrid.add(node.id());
+        }
+
+        cctx = fut.cctx;
+
+        pageStream = new PageStream();
+
+        this.fut = fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        return pageStream.hasNext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        return pageStream.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addPage(@Nullable UUID nodeId, Collection<R> data) {
+        pageStream.addPage(data);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onLastPage() {
+        super.onLastPage();
+
+        loadAllowed = false;
+
+        firstPageLatch.countDown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {

Review comment:
       No, as `cancel()` always prefaced with `onLastPage()` call. 
   
   Also, I checked that actually the `awaitFirstItem()` used only for ScanQuery only in case user specified partitions to enable retry this query if first page loading failed. Looks too narrow case, and I think there should be a separate activity to investigate this functionality.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655461447



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/NodePageStream.java
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Stream over single node.
+ */
+public class NodePageStream<R> extends PageStream<R> {
+    /** */
+    private final UUID nodeId;
+
+    /** */
+    private R head;
+
+    /** */
+    protected NodePageStream(GridCacheQueryAdapter qry, Object queueLock, long timeoutTime,
+        UUID nodeId, BiConsumer<Collection<UUID>, Boolean> reqPages) {
+        super(qry, queueLock, timeoutTime, new HashSet<>(F.asList(nodeId)), reqPages);
+
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return Head of stream, that is last item returned with {@link #next()}.
+     */
+    public R head() {
+        return head;
+    }
+
+    /**
+     * @return {@code true} If this stream has next row, {@code false} otherwise.
+     */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        if (head != null)
+            return true;
+        else {
+            if (!super.hasNext())
+                return false;
+            else
+                return (head = super.next()) != null;
+        }
+    }
+
+    /**
+     * @return Next item from this stream.
+     */

Review comment:
       ```suggestion
       /** {@inheritDoc} */
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r704600305



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/AbstractDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.DistributedCacheQueryReducer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Abstract class for distributed reducer implementations.
+ */
+abstract class AbstractDistributedCacheQueryReducer<R> implements DistributedCacheQueryReducer<R> {
+    /** Collection of streams that don't deliver all pages yet. */
+    private final Map<UUID, NodePageStream<R>> remoteStreams;

Review comment:
       The `streams` collection's goals are:
   1. `hasNext()` and `next()` in a user thread use thread local variable `streams`. 
   2. When a stream is fully done (remote pages and local iterator), then we remove it from the `stream` collection.
   
   The `remoteStreams` collection's goals are different:
   1. Ignite's utility threads use it for handling new remote pages.
   2. When remote pages finish (but not local iterator), then we remove this stream from the collection. 
   3. Some background actions (cancel, onNodeLeft) bases on this collection. Then we need track set of remote node ids.
   
   Then the idea was to have `streams` thread locally, and `remoteStreams` concurrently. It simplified a sync logic between collections as they should be independent from each other, also it helps to avoid one more lock in user thread.
   
   It's possible to share single collection for all threads. But we will still have another collection for tracking remote nodes with alive pages. This tracking collection can be a Set<UUID>, and then we will get NodePageStream from the main collection (`stream`), and the both collections will be concurrent. We won't win any from this. I think it's fine to track NodePageStream in 2 independent collectios. WDYT?
   
   BUT. There was a little bug, `onError` uses `streams` and runs in non-user thread, I'll replace it with the `remoteStreams`, and this will be fine. 




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r648231282



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,298 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream> streamCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(GridCacheQueryFutureAdapter fut, long reqId,
+        CacheQueryPageRequester fetcher, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, fetcher, nodes);
+
+        synchronized (queueLock()) {
+            streamsMap = new ConcurrentHashMap<>(nodes.size());
+            streams = (NodePageStream[])Array.newInstance(NodePageStream.class, nodes.size());
+
+            int i = 0;
+
+            for (ClusterNode node : nodes) {
+                streams[i] = new NodePageStream(node.id());
+                streamsMap.put(node.id(), streams[i++]);
+            }
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head, o2.head);
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            // Skip if stream has head: on previous next() head extracts from stream[streamOff], others streams do
+            // have head or it is the first run.
+            if (s.head() != null)
+                break;

Review comment:
       Reworked this, now there is a single logic for loading pages for PageStream and NodePageStream. After moving last item from queue to iterator, we trigger load page.




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r685278567



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       Make comparator static.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682454634



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {

Review comment:
       CacheQueryReducer.onPage requires nodeId as a param. 
   Was it made just because it is required for DistributedCacheQueryReducer?
   Why do we need DistributedCacheQueryReducer at all?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Checks whether node with provided {@code nodeId} is a map node for the query.
+     * Note: if all pages were received from this node, then the method will return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if node with provided {@code nodeId} is a map node for the query, {@code false} otherwise.
+     */
+    public boolean mapNode(UUID nodeId);

Review comment:
       ```suggestion
       public boolean isMapNode(UUID nodeId);
   ```




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r647804632



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.cache.query;
+
+import java.lang.reflect.Array;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends UnsortedDistributedCacheQueryReducer<R> {

Review comment:
       Done.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682464590



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -69,205 +53,69 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
         GridCacheQueryManager<K, V> mgr = ctx.queries();
 
         assert mgr != null;
-
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
-
-        assert qryMgr != null;
-
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
-
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
+    @Override protected void cancelQuery() {
+        reducer.onCancel();
 
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
-
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
     @Override protected void onNodeLeft(UUID nodeId) {
-        boolean callOnPage;
-
-        synchronized (this) {
-            callOnPage = !loc && subgrid.contains(nodeId);
-        }
+        boolean qryNode = reducer.mapNode(nodeId);
 
-        if (callOnPage)
-            onPage(nodeId, Collections.emptyList(),
+        if (qryNode)
+            onPage(nodeId, null,
                 new ClusterTopologyCheckedException("Remote node has left topology: " + nodeId), true);
     }
 
     /** {@inheritDoc} */
-    @Override public void awaitFirstPage() throws IgniteCheckedException {
-        try {
-            firstPageLatch.await();
+    @Override public void awaitFirstItemAvailable() throws IgniteCheckedException {
+        reducer.awaitInitialization();
 
-            if (isDone() && error() != null)
-                // Throw the exception if future failed.
-                get();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
+        if (isDone() && error() != null)
+            // Throw the exception if future failed.
+            get();
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
+    @Override protected CacheQueryReducer<R> reducer() {
+        return reducer;
     }
 
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    /** Set reducer. */
+    void reducer(DistributedCacheQueryReducer<R> reducer) {
+        this.reducer = reducer;
     }
 
     /** {@inheritDoc} */
-    @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+    @Override public Collection<R> get() throws IgniteCheckedException {
+        return get0();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteCheckedException {
+        return get0();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
-
-        return super.onCancelled();
+    @Override public Collection<R> getUninterruptibly() throws IgniteCheckedException {
+        return get0();
     }
 
-    /** {@inheritDoc} */
-    @Override public void onTimeout() {
-        firstPageLatch.countDown();
+    /**
+     * Completion of distributed query future depends on user that iterates over query result with lazy page loading.
+     * Then {@link #get()} can lock on unpredictably long period of time. So we should avoid call it.
+     */
+    private Collection<R> get0() throws IgniteCheckedException {
+        if (!isDone())
+            throw new IgniteIllegalStateException("Unexpected lock on iterator over distributed cache query result.");

Review comment:
       Why do you allow to get result via get() method, but forbid to wait?
   If noone must call get() then it should throws exception in anyway.
   If the method is never called and you don't care then overriding is useless.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)
+            );
+
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head(), o2.head());
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId());
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId());
+                streams[i] = null;
+                streamOff++;
+            }
+        }
+
+        if (streamOff == streams.length)
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(UUID nodeId, Collection<R> data, boolean last) {
+        NodePageStream<R> stream = streamsMap.get(nodeId);
+
+        if (stream == null)
+            return streamsMap.isEmpty();
+
+        stream.addPage(nodeId, data, last);
+
+        return last && (streamsMap.remove(nodeId) != null) && streamsMap.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.onError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCancel() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.cancel(ns -> pageRequester.cancelQueryRequest(reqId, ns, fut.fields()));
+
+        streamsMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean mapNode(UUID nodeId) {
+        return streamsMap.containsKey(nodeId);
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param cmp Comparator.
+     */
+    private static <Z> void bubbleUp(Z[] arr, int off, Comparator<Z> cmp) {

Review comment:
       Why don't use logN algorithm here?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/PageStream.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Iterator over pages stream. Pages are stored in a queue. After polling a queue try load a new page instead of it.
+ */
+class PageStream<R> {
+    /** Queue of data of results pages. */
+    protected final Queue<Collection<R>> queue = new LinkedList<>();
+
+    /** Iterator over current page. */
+    private Iterator<R> iter;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    private final Collection<UUID> subgrid;
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    private final Collection<UUID> rcvd = new HashSet<>();

Review comment:
       Why don't use NodeStream only and merge results at higher level?
   Maybe NodeStreams can even share same queue for the results.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -101,12 +104,30 @@
     /** Event listener. */
     private GridLocalEventListener lsnr;
 
+    /** Requester of cache query result pages. */
+    private CacheQueryPageRequester pageRequester;
+
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() != LOCAL;
 
+        pageRequester = new CacheQueryPageRequester(cctx) {
+            /** {@inheritDoc} */
+            @Override protected void sendLocal(GridCacheQueryRequest req) {

Review comment:
       This looks ambiguous.
   Why don't have just a factory for the requests instead of Requester?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       AFAIU, only Text queries have a 'payload'. Why is it named 'payload' and not score?
   Does 'null' score make sense for the result? 

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       AFAIU, only Text queries have a 'payload'. Why is it named 'payload' and not score?
   Does 'null' score make sense for the result?  If no then no NPE possible.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       Comparator instance can be static.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)
+            );
+
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head(), o2.head());
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId());
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId());
+                streams[i] = null;
+                streamOff++;
+            }
+        }
+
+        if (streamOff == streams.length)
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(UUID nodeId, Collection<R> data, boolean last) {
+        NodePageStream<R> stream = streamsMap.get(nodeId);
+
+        if (stream == null)
+            return streamsMap.isEmpty();
+
+        stream.addPage(nodeId, data, last);
+
+        return last && (streamsMap.remove(nodeId) != null) && streamsMap.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.onError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCancel() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.cancel(ns -> pageRequester.cancelQueryRequest(reqId, ns, fut.fields()));
+
+        streamsMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean mapNode(UUID nodeId) {
+        return streamsMap.containsKey(nodeId);
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param cmp Comparator.
+     */
+    private static <Z> void bubbleUp(Z[] arr, int off, Comparator<Z> cmp) {

Review comment:
       Why don't use logN algorithm here?
   E.g. use MinHeap instead.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.lang.reflect.Array;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /** Map of Node ID to stream. Used for inserting new pages. */
+    private final Map<UUID, NodePageStream<R>> streamsMap;
+
+    /** Array of streams. Used for iteration over result pages. */
+    private final NodePageStream<R>[] streams;
+
+    /** If {@code true} then there wasn't call {@link #hasNext()} on this reducer. */
+    private boolean first = true;
+
+    /**
+     * Offset of current stream to get next value. This offset only increments if a stream is done. The array {@link
+     * #streams} is sorted to put stream with lowest value on this offset.
+     */
+    private int streamOff;
+
+    /** Compares head of streams to get lowest value at the moment. */
+    private final Comparator<NodePageStream<R>> streamCmp;
+
+    /**
+     * @param fut Cache query future.
+     * @param reqId Cache query request ID.
+     * @param pageRequester Provides a functionality to request pages from remote nodes.
+     * @param queueLock Lock object that is shared between GridCacheQueryFuture and reducer.
+     * @param nodes Collection of nodes this query applies to.
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester);
+
+        streamsMap = new ConcurrentHashMap<>(nodes.size());
+        streams = (NodePageStream<R>[])Array.newInstance(NodePageStream.class, nodes.size());
+
+        int i = 0;
+
+        for (ClusterNode node : nodes) {
+            streams[i] = new NodePageStream<>(fut.query().query(), queueLock, fut.endTime(), node.id(), (ns, all) ->
+                pageRequester.requestPages(reqId, fut, ns, all)
+            );
+
+            streamsMap.put(node.id(), streams[i++]);
+        }
+
+        streamCmp = (o1, o2) -> {
+            if (o1 == o2)
+                return 0;
+
+            if (o1 == null)
+                return -1;
+
+            if (o2 == null)
+                return 1;
+
+            return rowCmp.compare(o1.head(), o2.head());
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            if (streams[i].hasNext())
+                return true;
+
+            // Nullify obsolete stream, move left bound.
+            streamsMap.remove(streams[i].nodeId());
+            streams[i] = null;
+            streamOff++;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        for (int i = streamOff; i < streams.length; i++) {
+            NodePageStream s = streams[i];
+
+            if (!s.hasNext()) {
+                // Nullify obsolete stream, move left bound.
+                streamsMap.remove(s.nodeId());
+                streams[i] = null;
+                streamOff++;
+            }
+        }
+
+        if (streamOff == streams.length)
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        if (first) {
+            first = false;
+
+            Arrays.sort(streams, streamCmp);
+        } else
+            bubbleUp(streams, streamOff, streamCmp);
+
+        return streams[streamOff].next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onPage(UUID nodeId, Collection<R> data, boolean last) {
+        NodePageStream<R> stream = streamsMap.get(nodeId);
+
+        if (stream == null)
+            return streamsMap.isEmpty();
+
+        stream.addPage(nodeId, data, last);
+
+        return last && (streamsMap.remove(nodeId) != null) && streamsMap.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.onError();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onCancel() {
+        for (NodePageStream<R> s: streamsMap.values())
+            s.cancel(ns -> pageRequester.cancelQueryRequest(reqId, ns, fut.fields()));
+
+        streamsMap.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean mapNode(UUID nodeId) {
+        return streamsMap.containsKey(nodeId);
+    }
+
+    /**
+     * @param arr Array.
+     * @param off Offset.
+     * @param cmp Comparator.
+     */
+    private static <Z> void bubbleUp(Z[] arr, int off, Comparator<Z> cmp) {

Review comment:
       Why don't use logN algorithm here?
   E.g. use MinHeap structure instead.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -777,133 +724,32 @@ private Object convert(Object obj) {
     }
 
     /** {@inheritDoc} */
-    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry,
-        Collection<ClusterNode> nodes) {
-        assert cctx.config().getCacheMode() != LOCAL;
-
-        if (log.isDebugEnabled())
-            log.debug("Executing distributed query: " + qry);
-
-        long reqId = cctx.io().nextIoId();
-
-        final GridCacheDistributedFieldsQueryFuture fut =
-            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes);
-
-        try {
-            qry.query().validate();
-
-            GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                true,
-                qry.query().clause(),
-                qry.query().limit(),
-                null,
-                null,
-                null,
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                qry.query().includeMetadata(),
-                qry.query().keepBinary(),
-                qry.query().subjectId(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                null,
-                cctx.deploymentEnabled(),
-                qry.query().isDataPageScanEnabled());
-
-            addQueryFuture(req.id(), fut);
-
-            final Object topic = topic(cctx.nodeId(), req.id());
-
-            cctx.io().addOrderedCacheHandler(cctx.shared(), topic, resHnd);
-
-            fut.listen(new CI1<IgniteInternalFuture<?>>() {
-                @Override public void apply(IgniteInternalFuture<?> fut) {
-                    cctx.io().removeOrderedHandler(false, topic);
-                }
-            });
-
-            sendRequest(fut, req, nodes);
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-        }
-
-        return fut;
+    @Override public CacheQueryFuture<?> queryFieldsDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, true);
     }
 
-    /**
-     * Sends query request.
-     *
-     * @param fut Distributed future.
-     * @param req Request.
-     * @param nodes Nodes.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private void sendRequest(
-        final GridCacheDistributedQueryFuture<?, ?, ?> fut,
-        final GridCacheQueryRequest req,
-        Collection<ClusterNode> nodes
-    ) throws IgniteCheckedException {
-        assert fut != null;
-        assert req != null;
-        assert nodes != null;
-
-        final UUID locNodeId = cctx.localNodeId();
-
-        ClusterNode locNode = null;
-
-        Collection<ClusterNode> rmtNodes = null;
-
-        for (ClusterNode n : nodes) {
-            if (n.id().equals(locNodeId))
-                locNode = n;
-            else {
-                if (rmtNodes == null)
-                    rmtNodes = new ArrayList<>(nodes.size());
+    /** Creates a reducer depends on query type. */
+    private DistributedCacheQueryReducer createReducer(GridCacheQueryType qryType, long reqId,
+        GridCacheDistributedQueryFuture fut, Collection<ClusterNode> nodes) {
 
-                rmtNodes.add(n);
-            }
-        }
+        if (qryType == TEXT) {
+            Comparator<CacheEntryWithPayload<K, V, Float>> cmp = (c1, c2) -> {
+                if (c1.payload() == c2.payload())
+                    return 0;
 
-        // Request should be sent to remote nodes before the query is processed on the local node.
-        // For example, a remote reducer has a state, we should not serialize and then send
-        // the reducer changed by the local node.
-        if (!F.isEmpty(rmtNodes)) {
-            for (ClusterNode node : rmtNodes) {
-                try {
-                    cctx.io().send(node, req, GridIoPolicy.QUERY_POOL);
-                }
-                catch (IgniteCheckedException e) {
-                    if (cctx.io().checkNodeLeft(node.id(), e, true)) {
-                        fut.onNodeLeft(node.id());
+                if (c1.payload() == null)

Review comment:
       Comparator instance can be static or moved into Reducer.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r726932960



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -523,57 +488,27 @@ else if (!cancelled.contains(res.requestId()))
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
     @Override public CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes) {
+        return queryDistributed(qry, nodes, false);
+    }
+
+    /** */
+    private CacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, final Collection<ClusterNode> nodes, boolean fields) {
         assert cctx.config().getCacheMode() != LOCAL;
 
         if (log.isDebugEnabled())
             log.debug("Executing distributed query: " + qry);
 
         long reqId = cctx.io().nextIoId();
 
-        final GridCacheDistributedQueryFuture<K, V, ?> fut =
-            new GridCacheDistributedQueryFuture<>(cctx, reqId, qry, nodes);
+        final GridCacheDistributedQueryFuture fut = fields ?
+            new GridCacheDistributedFieldsQueryFuture(cctx, reqId, qry, nodes) :
+            new GridCacheDistributedQueryFuture(cctx, reqId, qry, nodes);
 
         try {
-            qry.query().validate();
-
-            String clsName = qry.query().queryClassName();
-            Boolean dataPageScanEnabled = qry.query().isDataPageScanEnabled();
-            MvccSnapshot mvccSnapshot = qry.query().mvccSnapshot();
-
-            boolean deployFilterOrTransformer = (qry.query().scanFilter() != null || qry.query().transform() != null
-                || qry.query().idxQryDesc() != null) && cctx.gridDeploy().enabled();
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(
-                cctx.cacheId(),
-                reqId,
-                cctx.name(),
-                qry.query().type(),
-                false,
-                qry.query().clause(),
-                qry.query().idxQryDesc(),
-                qry.query().limit(),
-                clsName,
-                qry.query().scanFilter(),
-                qry.query().partition(),
-                qry.reducer(),
-                qry.transform(),
-                qry.query().pageSize(),
-                qry.query().includeBackups(),
-                qry.arguments(),
-                false,
-                qry.query().keepBinary(),
-                qry.query().taskHash(),
-                queryTopologyVersion(),
-                mvccSnapshot,
-                // Force deployment anyway if scan query is used.
-                cctx.deploymentEnabled() || deployFilterOrTransformer,
-                dataPageScanEnabled);
+            fut.qry.query().validate();

Review comment:
       Use #startQuery for start distributed query. #execute is OK for local future as it naturally executes full query locally. For distributed query we can just start query on remote nodes. And then we just put future to collection of futures and return iterator to user. 




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] Mmuzaf commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
Mmuzaf commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r729809348



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryFailoverTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.cache;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/** */
+public class GridCacheFullTextQueryFailoverTest extends GridCommonAbstractTest {
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        cache = startGrids(2).cache(PERSON_CACHE);
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, new Person("str" + i));
+    }
+
+    /** */
+    @Test
+    public void testStopNodeDuringQuery() throws Exception {
+        TextQuery<Integer, Person> qry = new TextQuery<Integer, Person>(Person.class, "str~")
+            .setPageSize(10);
+
+        Iterator<Cache.Entry<Integer, Person>> iter = cache.query(qry).iterator();
+
+        // Initialize internal structures.
+        iter.next();
+
+        stopGrid(1);
+
+        awaitPartitionMapExchange();
+
+        GridTestUtils.assertThrows(null, () -> {
+            iter.hasNext();
+
+            return null;
+        }, CacheException.class, "Remote node has left topology");
+    }
+
+    /** */
+    @Test
+    public void testCancelQuery() {
+        TextQuery<Integer, Person> qry = new TextQuery<Integer, Person>(Person.class, "str~")
+            .setPageSize(10);
+
+        QueryCursor<Cache.Entry<Integer, Person>> cursor = cache.query(qry);
+
+        Iterator<Cache.Entry<Integer, Person>> iter = cursor.iterator();
+
+        // Initialize internal structures.
+        iter.next();
+
+        cursor.close();
+
+        assertFalse(iter.hasNext());
+
+        GridTestUtils.assertThrows(null, () -> {
+            iter.next();
+
+            return null;
+        }, NoSuchElementException.class, null);

Review comment:
       ```suggestion
           GridTestUtils.assertThrows(log, iter::next, NoSuchElementException.class, null);
   ```

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryFailoverTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.cache;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/** */
+public class GridCacheFullTextQueryFailoverTest extends GridCommonAbstractTest {
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        cache = startGrids(2).cache(PERSON_CACHE);
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, new Person("str" + i));
+    }
+
+    /** */
+    @Test
+    public void testStopNodeDuringQuery() throws Exception {
+        TextQuery<Integer, Person> qry = new TextQuery<Integer, Person>(Person.class, "str~")
+            .setPageSize(10);
+
+        Iterator<Cache.Entry<Integer, Person>> iter = cache.query(qry).iterator();
+
+        // Initialize internal structures.
+        iter.next();
+
+        stopGrid(1);
+
+        awaitPartitionMapExchange();
+
+        GridTestUtils.assertThrows(null, () -> {
+            iter.hasNext();
+
+            return null;
+        }, CacheException.class, "Remote node has left topology");
+    }
+
+    /** */
+    @Test
+    public void testCancelQuery() {
+        TextQuery<Integer, Person> qry = new TextQuery<Integer, Person>(Person.class, "str~")
+            .setPageSize(10);
+
+        QueryCursor<Cache.Entry<Integer, Person>> cursor = cache.query(qry);
+
+        Iterator<Cache.Entry<Integer, Person>> iter = cursor.iterator();
+
+        // Initialize internal structures.
+        iter.next();
+
+        cursor.close();
+
+        assertFalse(iter.hasNext());
+
+        GridTestUtils.assertThrows(null, () -> {
+            iter.next();
+
+            return null;
+        }, NoSuchElementException.class, null);
+    }
+
+    /** */
+    public static class Person {

Review comment:
       ```suggestion
       private static class Person {
   ```

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryLimitTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/** */
+@RunWith(Parameterized.class)
+public class GridCacheFullTextQueryLimitTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_PER_NODE_COUNT = 100;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Number of nodes. */
+    @Parameterized.Parameter(0)
+    public int nodesCnt;
+
+    /** */
+    @Parameterized.Parameters(name = "nodesCnt={0}")
+    public static Iterable<Object[]> params() {
+        List<Object[]> params = new ArrayList<>();
+
+        for (int i = 1; i <= 8; i++) {
+            Object[] p = new Object[1];
+            p[0] = i;
+
+            params.add(p);
+        }
+
+        return params;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName(PERSON_CACHE)
+            .setCacheMode(PARTITIONED)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testResultOrderedByScore() throws Exception {
+        startGrids(nodesCnt);
+
+        int items = MAX_ITEM_PER_NODE_COUNT * nodesCnt;
+
+        final IgniteEx ignite = grid(0);
+
+        IgniteCache<Integer, Person> cache = ignite.cache(PERSON_CACHE);
+
+        int helloPivot = new Random().nextInt(items);
+
+        for (int i = 0; i < items; i++) {
+            String name = i == helloPivot ? "hello" : String.format("hello%02d", i);
+
+            cache.put(i, new Person(name));
+        }
+
+        // Extract all data that matches even little.
+        for (int limit = 1; limit <= items; limit++) {
+            TextQuery qry = new TextQuery<>(Person.class, "hello~")
+                .setLimit(limit);
+
+            List<Cache.Entry<Integer, Person>> result = cache.query(qry).getAll();
+
+            // Lucene returns top 50 results by query from single node even if limit is higher.
+            // Number of docs depends on amount of data on every node.
+            if (limit <= 50)

Review comment:
       Is it true that the limit doesn't work correctly? Do we have an issue to fix it?

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest;
+import org.apache.ignite.internal.util.typedef.CI2;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test pages loading for text queries tests.
+ */
+public class GridCacheFullTextQueryPagesTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_COUNT = 10_000;
+
+    /** Nodes count. */
+    private static final int NODES = 4;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Query page size. */
+    private static final int PAGE_SIZE = 100;
+
+    /** Limitation to query response size */
+    private static final int QUERY_LIMIT = 100;
+
+    /** Client node to start query. */
+    private static IgniteEx client;
+
+    /** */
+    private static TestStats testStats;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = new CacheConfiguration<Integer, Person>()
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(NODES);
+
+        client = startClientGrid();
+
+        testStats = new TestStats();
+
+        populateCache(client, MAX_ITEM_COUNT, (IgnitePredicate<Integer>)x -> String.valueOf(x).startsWith("1"));
+    }
+
+    /** Test that there is no cancel queries and number of requests corresponds to count of data rows on remote nodes. */
+    @Test
+    public void testTextQueryMultiplePagesNoLimit() throws Exception {
+        CountDownLatch latch = new CountDownLatch(testStats.nodesPagesCnt);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", 0, PAGE_SIZE);
+
+        latch.await(1, TimeUnit.SECONDS);
+
+        assertEquals(testStats.nodesPagesCnt, stats.totalReq());
+    }
+
+    /** Test that do not send cache page request after limit exceeded. */
+    @Test
+    public void testTextQueryLimitedMultiplePages() throws Exception {
+        CountDownLatch latch = new CountDownLatch(14);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", QUERY_LIMIT, 30);
+
+        checkPages(latch, stats, 4, 7, 3);
+    }
+
+    /** Test that rerequest some pages but then send a cancel query after limit exceeded. */
+    @Test
+    public void testTextQueryHighLimitedMultiplePages() throws Exception {
+        CountDownLatch latch = new CountDownLatch(15);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", QUERY_LIMIT, 20);
+
+        checkPages(latch, stats, 4, 8, 3);
+    }
+
+    /** */
+    private void checkPages(CountDownLatch pagesLatch, PageStats stats,
+        int expInitCnt, int expLoadCnt, int expCancelCnt) throws Exception {
+
+        pagesLatch.await(1, TimeUnit.SECONDS);
+
+        assertEquals(expInitCnt, stats.initCnt.get());
+        assertEquals(expLoadCnt, stats.loadCnt.get());
+        assertEquals(expCancelCnt, stats.cancelCnt.get());
+    }
+
+    /** */
+    private PageStats cacheQueryRequestsCount(CountDownLatch allReqLatch) {
+        PageStats stats = new PageStats();
+
+        for (int i = 0; i < NODES; i++) {
+            IgniteEx node = grid(i);
+
+            GridCacheContext cctx = node.cachex(PERSON_CACHE).context();
+
+            cctx.io().removeCacheHandlers(cctx.cacheId());
+
+            cctx.io().addCacheHandler(cctx.cacheId(), GridCacheQueryRequest.class, new CI2<UUID, GridCacheQueryRequest>() {
+                @Override public void apply(UUID nodeId, GridCacheQueryRequest req) {
+                    if (req.cancel())
+                        stats.cancelCnt.incrementAndGet();
+                    else if (req.limit() != 0)
+                        stats.initCnt.incrementAndGet();
+                    else
+                        stats.loadCnt.incrementAndGet();
+
+                    allReqLatch.countDown();
+
+                    ((GridCacheDistributedQueryManager) cctx.queries()).processQueryRequest(nodeId, req);
+                }
+            });
+        }
+
+        return stats;
+    }
+
+    /**
+     * Fill cache.
+     *
+     * @throws IgniteCheckedException if failed.
+     */
+    private void populateCache(IgniteEx ignite, int cnt, IgnitePredicate<Integer> expectedEntryFilter) throws IgniteCheckedException {
+        IgniteInternalCache<Integer, Person> cache = ignite.cachex(PERSON_CACHE);
+
+        Affinity<Integer> aff = cache.affinity();
+
+        Map<UUID, Integer> nodeToCnt = new HashMap<>();
+
+        Set<String> vals = new HashSet<>();
+
+        for (int i = 0; i < cnt; i++) {
+            cache.put(i, new Person(String.valueOf(i)));
+
+            if (expectedEntryFilter.apply(i)) {
+                vals.add(String.valueOf(i));
+
+                UUID nodeId = aff.mapKeyToNode(i).id();
+
+                if (nodeId.equals(ignite.localNode().id()))
+                    continue;
+
+                nodeToCnt.putIfAbsent(nodeId, 0);
+
+                nodeToCnt.compute(nodeId, (k, v) -> v + 1);
+            }
+        }
+
+        for (UUID nodeId: nodeToCnt.keySet()) {
+            int rowsCnt = nodeToCnt.get(nodeId);
+
+            int pagesCnt = rowsCnt / PAGE_SIZE + (rowsCnt % PAGE_SIZE == 0 ? 0 : 1);
+
+            testStats.nodesPagesCnt += pagesCnt;
+        }
+
+        testStats.dataCnt = vals.size();
+    }
+
+    /**
+     * @param clause Query clause.
+     * @param limit limits response size.
+     */
+    private void checkTextQuery(String clause, int limit, int pageSize) {
+        TextQuery<Integer, Person> qry = new TextQuery<Integer, Person>(Person.class, clause)
+            .setLimit(limit).setPageSize(pageSize);
+
+        validateQueryResults(qry);
+    }
+
+    /**
+     * Check query results.
+     */
+    private void validateQueryResults(TextQuery<Integer, Person> qry) {
+        IgniteCache<Integer, Person> cache = client.cache(PERSON_CACHE);
+
+        List<Cache.Entry<Integer, Person>> result = cache.query(qry).getAll();
+
+        int expRes = qry.getLimit() == 0 ? testStats.dataCnt : qry.getLimit();
+
+        assertEquals(expRes, result.size());
+    }
+
+    /**
+     * Test model class.
+     */
+    public static class Person implements Serializable {
+        /** */
+        @QueryTextField
+        private final String name;
+
+        /**
+         * Constructor
+         */
+        public Person(String name) {
+            this.name = name;
+        }
+    }
+
+    /** */
+    private static class PageStats {
+        /** Counter of init requests. */
+        private final AtomicInteger initCnt = new AtomicInteger();
+
+        /** Counter of load page requests. */
+        private final AtomicInteger loadCnt = new AtomicInteger();
+
+        /** Counter of cancel query requests. */
+        private final AtomicInteger cancelCnt = new AtomicInteger();
+
+        /** */
+        private int totalReq() {
+            return initCnt.get() + loadCnt.get() + cancelCnt.get();
+        }
+    }
+
+    /** */
+    private static class TestStats {

Review comment:
       You can use all page counters as a tests class field. There is no need for a dedicated pojo here.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest;
+import org.apache.ignite.internal.util.typedef.CI2;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test pages loading for text queries tests.
+ */
+public class GridCacheFullTextQueryPagesTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_COUNT = 10_000;
+
+    /** Nodes count. */
+    private static final int NODES = 4;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Query page size. */
+    private static final int PAGE_SIZE = 100;
+
+    /** Limitation to query response size */
+    private static final int QUERY_LIMIT = 100;
+
+    /** Client node to start query. */
+    private static IgniteEx client;
+
+    /** */
+    private static TestStats testStats;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = new CacheConfiguration<Integer, Person>()
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(NODES);
+
+        client = startClientGrid();
+
+        testStats = new TestStats();
+
+        populateCache(client, MAX_ITEM_COUNT, (IgnitePredicate<Integer>)x -> String.valueOf(x).startsWith("1"));
+    }
+
+    /** Test that there is no cancel queries and number of requests corresponds to count of data rows on remote nodes. */
+    @Test
+    public void testTextQueryMultiplePagesNoLimit() throws Exception {
+        CountDownLatch latch = new CountDownLatch(testStats.nodesPagesCnt);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", 0, PAGE_SIZE);
+
+        latch.await(1, TimeUnit.SECONDS);

Review comment:
       Should we wait for a more specific condition here instead of the latch?

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryFailoverTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.cache;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/** */
+public class GridCacheFullTextQueryFailoverTest extends GridCommonAbstractTest {
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);

Review comment:
       ```suggestion
           cacheCfg.setName(PERSON_CACHE)
               .setIndexedTypes(Integer.class, Person.class);
   ```

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryFailoverTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.cache;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/** */
+public class GridCacheFullTextQueryFailoverTest extends GridCommonAbstractTest {
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** */
+    private static IgniteCache<Integer, Person> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        cache = startGrids(2).cache(PERSON_CACHE);
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, new Person("str" + i));
+    }
+
+    /** */
+    @Test
+    public void testStopNodeDuringQuery() throws Exception {
+        TextQuery<Integer, Person> qry = new TextQuery<Integer, Person>(Person.class, "str~")
+            .setPageSize(10);
+
+        Iterator<Cache.Entry<Integer, Person>> iter = cache.query(qry).iterator();
+
+        // Initialize internal structures.
+        iter.next();
+
+        stopGrid(1);
+
+        awaitPartitionMapExchange();
+
+        GridTestUtils.assertThrows(null, () -> {
+            iter.hasNext();
+
+            return null;
+        }, CacheException.class, "Remote node has left topology");

Review comment:
       ```suggestion
           GridTestUtils.assertThrows(log, 
               iter::hasNext, 
               CacheException.class, 
               "Remote node has left topology");
   ```

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest;
+import org.apache.ignite.internal.util.typedef.CI2;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test pages loading for text queries tests.
+ */
+public class GridCacheFullTextQueryPagesTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_COUNT = 10_000;
+
+    /** Nodes count. */
+    private static final int NODES = 4;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Query page size. */
+    private static final int PAGE_SIZE = 100;
+
+    /** Limitation to query response size */
+    private static final int QUERY_LIMIT = 100;
+
+    /** Client node to start query. */
+    private static IgniteEx client;
+
+    /** */
+    private static TestStats testStats;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = new CacheConfiguration<Integer, Person>()
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(NODES);
+
+        client = startClientGrid();
+
+        testStats = new TestStats();
+
+        populateCache(client, MAX_ITEM_COUNT, (IgnitePredicate<Integer>)x -> String.valueOf(x).startsWith("1"));
+    }
+
+    /** Test that there is no cancel queries and number of requests corresponds to count of data rows on remote nodes. */
+    @Test
+    public void testTextQueryMultiplePagesNoLimit() throws Exception {
+        CountDownLatch latch = new CountDownLatch(testStats.nodesPagesCnt);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", 0, PAGE_SIZE);
+
+        latch.await(1, TimeUnit.SECONDS);
+
+        assertEquals(testStats.nodesPagesCnt, stats.totalReq());
+    }
+
+    /** Test that do not send cache page request after limit exceeded. */
+    @Test
+    public void testTextQueryLimitedMultiplePages() throws Exception {
+        CountDownLatch latch = new CountDownLatch(14);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", QUERY_LIMIT, 30);
+
+        checkPages(latch, stats, 4, 7, 3);
+    }
+
+    /** Test that rerequest some pages but then send a cancel query after limit exceeded. */
+    @Test
+    public void testTextQueryHighLimitedMultiplePages() throws Exception {
+        CountDownLatch latch = new CountDownLatch(15);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", QUERY_LIMIT, 20);
+
+        checkPages(latch, stats, 4, 8, 3);
+    }
+
+    /** */
+    private void checkPages(CountDownLatch pagesLatch, PageStats stats,
+        int expInitCnt, int expLoadCnt, int expCancelCnt) throws Exception {
+
+        pagesLatch.await(1, TimeUnit.SECONDS);
+
+        assertEquals(expInitCnt, stats.initCnt.get());
+        assertEquals(expLoadCnt, stats.loadCnt.get());
+        assertEquals(expCancelCnt, stats.cancelCnt.get());
+    }
+
+    /** */
+    private PageStats cacheQueryRequestsCount(CountDownLatch allReqLatch) {

Review comment:
       Use the `TestRecordingCommunicationSpi` instead to count all the necessary data you require.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryLimitTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/** */
+@RunWith(Parameterized.class)
+public class GridCacheFullTextQueryLimitTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_PER_NODE_COUNT = 100;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Number of nodes. */
+    @Parameterized.Parameter(0)
+    public int nodesCnt;
+
+    /** */
+    @Parameterized.Parameters(name = "nodesCnt={0}")
+    public static Iterable<Object[]> params() {
+        List<Object[]> params = new ArrayList<>();
+
+        for (int i = 1; i <= 8; i++) {
+            Object[] p = new Object[1];
+            p[0] = i;
+
+            params.add(p);
+        }
+
+        return params;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName(PERSON_CACHE)
+            .setCacheMode(PARTITIONED)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testResultOrderedByScore() throws Exception {
+        startGrids(nodesCnt);
+
+        int items = MAX_ITEM_PER_NODE_COUNT * nodesCnt;
+
+        final IgniteEx ignite = grid(0);
+
+        IgniteCache<Integer, Person> cache = ignite.cache(PERSON_CACHE);
+
+        int helloPivot = new Random().nextInt(items);
+
+        for (int i = 0; i < items; i++) {
+            String name = i == helloPivot ? "hello" : String.format("hello%02d", i);
+
+            cache.put(i, new Person(name));
+        }
+
+        // Extract all data that matches even little.
+        for (int limit = 1; limit <= items; limit++) {
+            TextQuery qry = new TextQuery<>(Person.class, "hello~")
+                .setLimit(limit);
+
+            List<Cache.Entry<Integer, Person>> result = cache.query(qry).getAll();
+
+            // Lucene returns top 50 results by query from single node even if limit is higher.
+            // Number of docs depends on amount of data on every node.
+            if (limit <= 50)
+                assertEquals(limit, result.size());
+            else
+                assertTrue(limit >= result.size());
+
+            // hello has to be on the top.
+            assertEquals("Limit=" + limit, "hello", result.get(0).getValue().name);

Review comment:
       Should we check the order of the rest of the data?

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest;
+import org.apache.ignite.internal.util.typedef.CI2;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test pages loading for text queries tests.
+ */
+public class GridCacheFullTextQueryPagesTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_COUNT = 10_000;
+
+    /** Nodes count. */
+    private static final int NODES = 4;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Query page size. */
+    private static final int PAGE_SIZE = 100;
+
+    /** Limitation to query response size */
+    private static final int QUERY_LIMIT = 100;
+
+    /** Client node to start query. */
+    private static IgniteEx client;
+
+    /** */
+    private static TestStats testStats;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = new CacheConfiguration<Integer, Person>()
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(NODES);
+
+        client = startClientGrid();
+
+        testStats = new TestStats();
+
+        populateCache(client, MAX_ITEM_COUNT, (IgnitePredicate<Integer>)x -> String.valueOf(x).startsWith("1"));
+    }
+
+    /** Test that there is no cancel queries and number of requests corresponds to count of data rows on remote nodes. */
+    @Test
+    public void testTextQueryMultiplePagesNoLimit() throws Exception {
+        CountDownLatch latch = new CountDownLatch(testStats.nodesPagesCnt);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", 0, PAGE_SIZE);
+
+        latch.await(1, TimeUnit.SECONDS);
+
+        assertEquals(testStats.nodesPagesCnt, stats.totalReq());
+    }
+
+    /** Test that do not send cache page request after limit exceeded. */
+    @Test
+    public void testTextQueryLimitedMultiplePages() throws Exception {
+        CountDownLatch latch = new CountDownLatch(14);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", QUERY_LIMIT, 30);
+
+        checkPages(latch, stats, 4, 7, 3);
+    }
+
+    /** Test that rerequest some pages but then send a cancel query after limit exceeded. */
+    @Test
+    public void testTextQueryHighLimitedMultiplePages() throws Exception {
+        CountDownLatch latch = new CountDownLatch(15);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", QUERY_LIMIT, 20);
+
+        checkPages(latch, stats, 4, 8, 3);
+    }
+
+    /** */
+    private void checkPages(CountDownLatch pagesLatch, PageStats stats,
+        int expInitCnt, int expLoadCnt, int expCancelCnt) throws Exception {
+
+        pagesLatch.await(1, TimeUnit.SECONDS);
+
+        assertEquals(expInitCnt, stats.initCnt.get());
+        assertEquals(expLoadCnt, stats.loadCnt.get());
+        assertEquals(expCancelCnt, stats.cancelCnt.get());
+    }
+
+    /** */
+    private PageStats cacheQueryRequestsCount(CountDownLatch allReqLatch) {
+        PageStats stats = new PageStats();
+
+        for (int i = 0; i < NODES; i++) {
+            IgniteEx node = grid(i);
+
+            GridCacheContext cctx = node.cachex(PERSON_CACHE).context();
+
+            cctx.io().removeCacheHandlers(cctx.cacheId());
+
+            cctx.io().addCacheHandler(cctx.cacheId(), GridCacheQueryRequest.class, new CI2<UUID, GridCacheQueryRequest>() {
+                @Override public void apply(UUID nodeId, GridCacheQueryRequest req) {
+                    if (req.cancel())
+                        stats.cancelCnt.incrementAndGet();
+                    else if (req.limit() != 0)
+                        stats.initCnt.incrementAndGet();
+                    else
+                        stats.loadCnt.incrementAndGet();
+
+                    allReqLatch.countDown();
+
+                    ((GridCacheDistributedQueryManager) cctx.queries()).processQueryRequest(nodeId, req);
+                }
+            });
+        }
+
+        return stats;
+    }
+
+    /**
+     * Fill cache.
+     *
+     * @throws IgniteCheckedException if failed.
+     */
+    private void populateCache(IgniteEx ignite, int cnt, IgnitePredicate<Integer> expectedEntryFilter) throws IgniteCheckedException {
+        IgniteInternalCache<Integer, Person> cache = ignite.cachex(PERSON_CACHE);
+
+        Affinity<Integer> aff = cache.affinity();
+
+        Map<UUID, Integer> nodeToCnt = new HashMap<>();
+
+        Set<String> vals = new HashSet<>();
+
+        for (int i = 0; i < cnt; i++) {
+            cache.put(i, new Person(String.valueOf(i)));
+
+            if (expectedEntryFilter.apply(i)) {
+                vals.add(String.valueOf(i));
+
+                UUID nodeId = aff.mapKeyToNode(i).id();
+
+                if (nodeId.equals(ignite.localNode().id()))
+                    continue;
+
+                nodeToCnt.putIfAbsent(nodeId, 0);
+
+                nodeToCnt.compute(nodeId, (k, v) -> v + 1);
+            }
+        }
+
+        for (UUID nodeId: nodeToCnt.keySet()) {
+            int rowsCnt = nodeToCnt.get(nodeId);
+
+            int pagesCnt = rowsCnt / PAGE_SIZE + (rowsCnt % PAGE_SIZE == 0 ? 0 : 1);
+
+            testStats.nodesPagesCnt += pagesCnt;
+        }
+
+        testStats.dataCnt = vals.size();
+    }
+
+    /**
+     * @param clause Query clause.
+     * @param limit limits response size.
+     */
+    private void checkTextQuery(String clause, int limit, int pageSize) {
+        TextQuery<Integer, Person> qry = new TextQuery<Integer, Person>(Person.class, clause)
+            .setLimit(limit).setPageSize(pageSize);
+
+        validateQueryResults(qry);
+    }
+
+    /**
+     * Check query results.
+     */
+    private void validateQueryResults(TextQuery<Integer, Person> qry) {

Review comment:
       You can inline this method.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryLimitTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/** */
+@RunWith(Parameterized.class)
+public class GridCacheFullTextQueryLimitTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_PER_NODE_COUNT = 100;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Number of nodes. */
+    @Parameterized.Parameter(0)
+    public int nodesCnt;
+
+    /** */
+    @Parameterized.Parameters(name = "nodesCnt={0}")
+    public static Iterable<Object[]> params() {
+        List<Object[]> params = new ArrayList<>();
+
+        for (int i = 1; i <= 8; i++) {
+            Object[] p = new Object[1];
+            p[0] = i;
+
+            params.add(p);
+        }
+
+        return params;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName(PERSON_CACHE)
+            .setCacheMode(PARTITIONED)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** */
+    @Test
+    public void testResultOrderedByScore() throws Exception {
+        startGrids(nodesCnt);
+
+        int items = MAX_ITEM_PER_NODE_COUNT * nodesCnt;
+
+        final IgniteEx ignite = grid(0);
+
+        IgniteCache<Integer, Person> cache = ignite.cache(PERSON_CACHE);
+
+        int helloPivot = new Random().nextInt(items);
+
+        for (int i = 0; i < items; i++) {
+            String name = i == helloPivot ? "hello" : String.format("hello%02d", i);
+
+            cache.put(i, new Person(name));
+        }
+
+        // Extract all data that matches even little.
+        for (int limit = 1; limit <= items; limit++) {
+            TextQuery qry = new TextQuery<>(Person.class, "hello~")
+                .setLimit(limit);
+
+            List<Cache.Entry<Integer, Person>> result = cache.query(qry).getAll();
+
+            // Lucene returns top 50 results by query from single node even if limit is higher.
+            // Number of docs depends on amount of data on every node.
+            if (limit <= 50)
+                assertEquals(limit, result.size());
+            else
+                assertTrue(limit >= result.size());
+
+            // hello has to be on the top.
+            assertEquals("Limit=" + limit, "hello", result.get(0).getValue().name);
+        }
+    }
+
+    /**
+     * Test model class.
+     */
+    public static class Person implements Serializable {

Review comment:
       Can we create an abstract test with all utils and data for text query testing? 
   e.g. `AbstractCacheFullTextTest extends GridCommonAbstractTest`

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryPagesTest.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.TextQuery;
+import org.apache.ignite.cache.query.annotations.QueryTextField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest;
+import org.apache.ignite.internal.util.typedef.CI2;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test pages loading for text queries tests.
+ */
+public class GridCacheFullTextQueryPagesTest extends GridCommonAbstractTest {
+    /** Cache size. */
+    private static final int MAX_ITEM_COUNT = 10_000;
+
+    /** Nodes count. */
+    private static final int NODES = 4;
+
+    /** Cache name */
+    private static final String PERSON_CACHE = "Person";
+
+    /** Query page size. */
+    private static final int PAGE_SIZE = 100;
+
+    /** Limitation to query response size */
+    private static final int QUERY_LIMIT = 100;
+
+    /** Client node to start query. */
+    private static IgniteEx client;
+
+    /** */
+    private static TestStats testStats;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Person> cacheCfg = new CacheConfiguration<Integer, Person>()
+            .setName(PERSON_CACHE)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(NODES);
+
+        client = startClientGrid();
+
+        testStats = new TestStats();
+
+        populateCache(client, MAX_ITEM_COUNT, (IgnitePredicate<Integer>)x -> String.valueOf(x).startsWith("1"));
+    }
+
+    /** Test that there is no cancel queries and number of requests corresponds to count of data rows on remote nodes. */
+    @Test
+    public void testTextQueryMultiplePagesNoLimit() throws Exception {
+        CountDownLatch latch = new CountDownLatch(testStats.nodesPagesCnt);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", 0, PAGE_SIZE);
+
+        latch.await(1, TimeUnit.SECONDS);
+
+        assertEquals(testStats.nodesPagesCnt, stats.totalReq());
+    }
+
+    /** Test that do not send cache page request after limit exceeded. */
+    @Test
+    public void testTextQueryLimitedMultiplePages() throws Exception {
+        CountDownLatch latch = new CountDownLatch(14);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", QUERY_LIMIT, 30);
+
+        checkPages(latch, stats, 4, 7, 3);
+    }
+
+    /** Test that rerequest some pages but then send a cancel query after limit exceeded. */
+    @Test
+    public void testTextQueryHighLimitedMultiplePages() throws Exception {
+        CountDownLatch latch = new CountDownLatch(15);
+
+        PageStats stats = cacheQueryRequestsCount(latch);
+
+        checkTextQuery("1*", QUERY_LIMIT, 20);
+
+        checkPages(latch, stats, 4, 8, 3);
+    }
+
+    /** */
+    private void checkPages(CountDownLatch pagesLatch, PageStats stats,
+        int expInitCnt, int expLoadCnt, int expCancelCnt) throws Exception {
+
+        pagesLatch.await(1, TimeUnit.SECONDS);
+
+        assertEquals(expInitCnt, stats.initCnt.get());
+        assertEquals(expLoadCnt, stats.loadCnt.get());
+        assertEquals(expCancelCnt, stats.cancelCnt.get());
+    }
+
+    /** */
+    private PageStats cacheQueryRequestsCount(CountDownLatch allReqLatch) {
+        PageStats stats = new PageStats();
+
+        for (int i = 0; i < NODES; i++) {
+            IgniteEx node = grid(i);
+
+            GridCacheContext cctx = node.cachex(PERSON_CACHE).context();
+
+            cctx.io().removeCacheHandlers(cctx.cacheId());
+
+            cctx.io().addCacheHandler(cctx.cacheId(), GridCacheQueryRequest.class, new CI2<UUID, GridCacheQueryRequest>() {
+                @Override public void apply(UUID nodeId, GridCacheQueryRequest req) {
+                    if (req.cancel())
+                        stats.cancelCnt.incrementAndGet();
+                    else if (req.limit() != 0)
+                        stats.initCnt.incrementAndGet();
+                    else
+                        stats.loadCnt.incrementAndGet();
+
+                    allReqLatch.countDown();
+
+                    ((GridCacheDistributedQueryManager) cctx.queries()).processQueryRequest(nodeId, req);
+                }
+            });
+        }
+
+        return stats;
+    }
+
+    /**
+     * Fill cache.
+     *
+     * @throws IgniteCheckedException if failed.
+     */
+    private void populateCache(IgniteEx ignite, int cnt, IgnitePredicate<Integer> expectedEntryFilter) throws IgniteCheckedException {
+        IgniteInternalCache<Integer, Person> cache = ignite.cachex(PERSON_CACHE);
+
+        Affinity<Integer> aff = cache.affinity();
+
+        Map<UUID, Integer> nodeToCnt = new HashMap<>();
+
+        Set<String> vals = new HashSet<>();
+
+        for (int i = 0; i < cnt; i++) {
+            cache.put(i, new Person(String.valueOf(i)));
+
+            if (expectedEntryFilter.apply(i)) {
+                vals.add(String.valueOf(i));
+
+                UUID nodeId = aff.mapKeyToNode(i).id();
+
+                if (nodeId.equals(ignite.localNode().id()))
+                    continue;
+
+                nodeToCnt.putIfAbsent(nodeId, 0);
+
+                nodeToCnt.compute(nodeId, (k, v) -> v + 1);
+            }
+        }
+
+        for (UUID nodeId: nodeToCnt.keySet()) {
+            int rowsCnt = nodeToCnt.get(nodeId);
+
+            int pagesCnt = rowsCnt / PAGE_SIZE + (rowsCnt % PAGE_SIZE == 0 ? 0 : 1);
+
+            testStats.nodesPagesCnt += pagesCnt;
+        }
+
+        testStats.dataCnt = vals.size();
+    }
+
+    /**
+     * @param clause Query clause.
+     * @param limit limits response size.
+     */
+    private void checkTextQuery(String clause, int limit, int pageSize) {
+        TextQuery<Integer, Person> qry = new TextQuery<Integer, Person>(Person.class, clause)
+            .setLimit(limit).setPageSize(pageSize);
+
+        validateQueryResults(qry);
+    }
+
+    /**
+     * Check query results.
+     */
+    private void validateQueryResults(TextQuery<Integer, Person> qry) {
+        IgniteCache<Integer, Person> cache = client.cache(PERSON_CACHE);
+
+        List<Cache.Entry<Integer, Person>> result = cache.query(qry).getAll();
+
+        int expRes = qry.getLimit() == 0 ? testStats.dataCnt : qry.getLimit();
+
+        assertEquals(expRes, result.size());
+    }
+
+    /**
+     * Test model class.
+     */
+    public static class Person implements Serializable {
+        /** */
+        @QueryTextField
+        private final String name;
+
+        /**
+         * Constructor
+         */
+        public Person(String name) {
+            this.name = name;
+        }
+    }
+
+    /** */
+    private static class PageStats {

Review comment:
       I don't think you need a dedicated class here, all these things can be done by recording communication spi.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655448376



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -69,205 +50,43 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
         GridCacheQueryManager<K, V> mgr = ctx.queries();
 
         assert mgr != null;
-
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
-
-        assert qryMgr != null;
-
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
-
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
-
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
+    @Override protected void cancelQuery() {
+        reducer.onCancel();
 
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
-    /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
+    /** Fail if a node runs this query left cluster. */

Review comment:
       ```suggestion
       /** {@inheritDoc} */
   ```




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682491318



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/PageStream.java
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Iterator over pages stream. Pages are stored in a queue. After polling a queue try load a new page instead of it.
+ */
+class PageStream<R> {
+    /** Queue of data of results pages. */
+    protected final Queue<Collection<R>> queue = new LinkedList<>();
+
+    /** Iterator over current page. */
+    private Iterator<R> iter;
+
+    /**
+     * Dynamic collection of nodes that run this query. If a node finishes this query then remove it from this colleciton.
+     */
+    private final Collection<UUID> subgrid;
+
+    /**
+     * List of nodes that respons with cache query result pages. This collection should be cleaned before sending new
+     * cache query request.
+     */
+    private final Collection<UUID> rcvd = new HashSet<>();

Review comment:
       Why don't use NodeStream only and merge results at higher level?
   Maybe NodeStreams can even share same queue for the results.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r633556166



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -169,103 +97,30 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    @Override protected boolean onPage(@Nullable UUID nodeId, boolean last) {
+        return reducer.onPage(nodeId, last);
     }
 
     /** {@inheritDoc} */
     @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+        reducer.loadAll();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override protected Reducer<R> reducer() {
+        return reducer;
     }
 
     /** {@inheritDoc} */
     @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         return super.onCancelled();
     }
 
     /** {@inheritDoc} */
     @Override public void onTimeout() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         super.onTimeout();
     }

Review comment:
       Do we need all these callbacks in QueryFutureAdapter just to delegate calls to reducer
   and having reducer() at same time?
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -169,103 +97,30 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    @Override protected boolean onPage(@Nullable UUID nodeId, boolean last) {
+        return reducer.onPage(nodeId, last);
     }
 
     /** {@inheritDoc} */
     @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+        reducer.loadAll();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override protected Reducer<R> reducer() {
+        return reducer;
     }
 
     /** {@inheritDoc} */
     @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         return super.onCancelled();
     }
 
     /** {@inheritDoc} */
     @Override public void onTimeout() {
-        firstPageLatch.countDown();
+        reducer.onLastPage();
 
         super.onTimeout();
     }

Review comment:
       Do we need all these callbacks in QueryFutureAdapter just to delegate calls to reducer
   and having reducer() at same time?
   I see a patter you alresy use: reducer().onPage() and similar.
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryResultFetcher.java
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.cache.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * This class is responsible for sending request for query result pages to remote nodes.
+ */
+public abstract class CacheQueryResultFetcher {
+    /**
+     * Map (requestId -> query future) where request id is unique for all requests per query.
+     * This map is populated by query manager.
+     */
+    private final ConcurrentMap<Long, GridCacheDistributedQueryFuture<?, ?, ?>> qryFuts;
+
+    /** Cache context. */
+    private final GridCacheContext cctx;
+
+    /** Ignite logger. */
+    private final IgniteLogger log;
+
+    /** */
+    CacheQueryResultFetcher(
+        final GridCacheContext cctx,
+        final ConcurrentMap<Long, GridCacheDistributedQueryFuture<?, ?, ?>> qryFuts) {

Review comment:
       CacheQueryResultFetcher doesn't look like good class name.
   1. Actually, it is not fetcher for particular query, it is a holder for object-fetchers for multiple queries.
   2. Why it depends on qryFuts?
   3. The functionality of the class is limited to create-and-send requests, but any response processing is out of scope...
   So, it is a 'requester', but not 'fetcher'




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r648213313



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {

Review comment:
       Reworked methods within hierarchy, rename some methods. So, now it looks better. Don't think that separation is good idea, as `reducer(): CacheQueryReducer` used within GridCacheQueryFutureAdapter and shared between all types of queries.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682464590



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -69,205 +53,69 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
         GridCacheQueryManager<K, V> mgr = ctx.queries();
 
         assert mgr != null;
-
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
-
-        assert qryMgr != null;
-
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
-
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
+    @Override protected void cancelQuery() {
+        reducer.onCancel();
 
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
-
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
     @Override protected void onNodeLeft(UUID nodeId) {
-        boolean callOnPage;
-
-        synchronized (this) {
-            callOnPage = !loc && subgrid.contains(nodeId);
-        }
+        boolean qryNode = reducer.mapNode(nodeId);
 
-        if (callOnPage)
-            onPage(nodeId, Collections.emptyList(),
+        if (qryNode)
+            onPage(nodeId, null,
                 new ClusterTopologyCheckedException("Remote node has left topology: " + nodeId), true);
     }
 
     /** {@inheritDoc} */
-    @Override public void awaitFirstPage() throws IgniteCheckedException {
-        try {
-            firstPageLatch.await();
+    @Override public void awaitFirstItemAvailable() throws IgniteCheckedException {
+        reducer.awaitInitialization();
 
-            if (isDone() && error() != null)
-                // Throw the exception if future failed.
-                get();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
+        if (isDone() && error() != null)
+            // Throw the exception if future failed.
+            get();
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
+    @Override protected CacheQueryReducer<R> reducer() {
+        return reducer;
     }
 
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    /** Set reducer. */
+    void reducer(DistributedCacheQueryReducer<R> reducer) {
+        this.reducer = reducer;
     }
 
     /** {@inheritDoc} */
-    @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+    @Override public Collection<R> get() throws IgniteCheckedException {
+        return get0();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteCheckedException {
+        return get0();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
-
-        return super.onCancelled();
+    @Override public Collection<R> getUninterruptibly() throws IgniteCheckedException {
+        return get0();
     }
 
-    /** {@inheritDoc} */
-    @Override public void onTimeout() {
-        firstPageLatch.countDown();
+    /**
+     * Completion of distributed query future depends on user that iterates over query result with lazy page loading.
+     * Then {@link #get()} can lock on unpredictably long period of time. So we should avoid call it.
+     */
+    private Collection<R> get0() throws IgniteCheckedException {
+        if (!isDone())
+            throw new IgniteIllegalStateException("Unexpected lock on iterator over distributed cache query result.");

Review comment:
       Why do you allow to get result via get() method, but forbid to wait?
   If noone must call get() then it should throws exception in anyway.
   If the method is never called and you don't care then overriding is useless.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r655454330



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
##########
@@ -146,40 +137,50 @@ public GridCacheQueryBean query() {
     /**
      * @return If fields query.
      */
-    boolean fields() {
+    public boolean fields() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        cctx.time().removeTimeoutObject(this);
-
-        return super.onDone(res, err);
-    }
-
     /** {@inheritDoc} */
     @Override public R next() {
         try {
-            R next = unmaskNull(internalIterator().next());
+            if (!limitDisabled && cnt == capacity)
+                return null;
 
-            cnt.decrementAndGet();
+            checkError();
+
+            R next = null;
+
+            if (reducer().hasNext()) {
+                next = unmaskNull(reducer().next());
+
+                if (!limitDisabled) {
+                    cnt++;
+
+                    // Exceed limit, stop page loading and cancel queries.
+                    if (cnt == capacity)
+                        cancel();
+                }
+            }
+
+            checkError();
 
             return next;
         }
-        catch (NoSuchElementException ignored) {
-            return null;
-        }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
         }
     }
 
+    /** @return Cache query results reducer. */
+    protected abstract CacheQueryReducer<R> reducer();
+
     /**
-     * Waits for the first page to be received from remote node(s), if any.
+     * Waits for the first item to be received from remote node(s), if any.
      *
      * @throws IgniteCheckedException If query execution failed with an error.
      */
-    public abstract void awaitFirstPage() throws IgniteCheckedException;
+    public abstract void awaitFirstItem() throws IgniteCheckedException;

Review comment:
       Maybe awaitFirstItemAvailable() will be better.
   Actually, we wait here not for the first item/page from remote, but the first item available after the reduction applied.




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645128765



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Callback that invoked after getting a page from remote node. Checks whether it is the last page for query or not.
+     *
+     * @param nodeId Node ID of remote page.
+     * @param last Whether page is last for specified node.
+     * @return Whether page is last for a query.
+     */
+    public boolean onPage(@Nullable UUID nodeId, boolean last);
+
+    /**
+     * Loads full cache query result pages from remote nodes. It can be done for speedup operation if user invokes
+     * get() on {@link GridCacheQueryFutureAdapter} instead of using it as iterator.
+     *
+     * @throws IgniteInterruptedCheckedException If thread is interrupted.
+     */
+    public void loadAll() throws IgniteInterruptedCheckedException;
+
+    /**
+     * Callback to handle node left.
+     *
+     * @param nodeId Node ID that left a cluster.
+     * @return {@code true} if specified node runs this query.
+     */
+    public boolean onNodeLeft(UUID nodeId);
+
+    /** Blocks while reducer doesn't get first result item for this query. */
+    public void awaitFirstItem() throws InterruptedException;

Review comment:
       ```suggestion
       /** 
       * Blocks current thread until reducer will be ready to return the very first result item for the query. 
       */
       public void awaitInitialization() throws InterruptedException;
   ```




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



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682466010



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Checks whether node with provided {@code nodeId} is a map node for the query.
+     * Note: if all pages were received from this node, then the method will return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if node with provided {@code nodeId} is a map node for the query, {@code false} otherwise.
+     */
+    public boolean mapNode(UUID nodeId);

Review comment:
       According to code style [1] we should not use "is" as prefix for internal API.
   
   [1] https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-Gettersandsetters
   
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered via callback
+ * {@link #onPage(UUID, Collection, boolean)}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Callback that invoked on receiving a new page.
+     *
+     * @param nodeId Node ID that sent this page.
+     * @param data Page data rows.
+     * @param last Whether this page is last for specified {@code nodeId}.
+     * @return {@code true} if this page is final page for query and no more pages are waited, otherwise {@code false}.
+     */
+    public boolean onPage(UUID nodeId, Collection<T> data, boolean last);
+
+    /**
+     * Callback in case of page with error.
+     */
+    public void onError();
+
+    /**
+     * Callback that invokes after a query future is done.
+     */
+    public void onFinish();

Review comment:
       There is a comment about it, see invokation [1]. We should release latch after onDone() invokation. Also I've described this in previous conversation: https://github.com/apache/ignite/pull/9081#discussion_r649824494
   
   [1] [GridCacheQueryFutureAdapter:332](https://github.com/apache/ignite/blob/0add93b488ebe7265b2a0f951fcdfd55adbcc01f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java#L332)

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java
##########
@@ -69,205 +53,69 @@ protected GridCacheDistributedQueryFuture(GridCacheContext<K, V> ctx, long reqId
         GridCacheQueryManager<K, V> mgr = ctx.queries();
 
         assert mgr != null;
-
-        synchronized (this) {
-            for (ClusterNode node : nodes)
-                subgrid.add(node.id());
-        }
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
-        final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
-
-        assert qryMgr != null;
-
-        try {
-            Collection<ClusterNode> allNodes = cctx.discovery().allNodes();
-            Collection<ClusterNode> nodes;
-
-            synchronized (this) {
-                nodes = F.retain(allNodes, true,
-                    new P1<ClusterNode>() {
-                        @Override public boolean apply(ClusterNode node) {
-                            return !cctx.localNodeId().equals(node.id()) && subgrid.contains(node.id());
-                        }
-                    }
-                );
-
-                subgrid.clear();
-            }
-
-            final GridCacheQueryRequest req = new GridCacheQueryRequest(cctx.cacheId(),
-                reqId,
-                fields(),
-                qryMgr.queryTopologyVersion(),
-                cctx.deploymentEnabled());
-
-            // Process cancel query directly (without sending) for local node,
-            cctx.closures().callLocalSafe(new GridPlainCallable<Object>() {
-                @Override public Object call() {
-                    qryMgr.processQueryRequest(cctx.localNodeId(), req);
+    @Override protected void cancelQuery() {
+        reducer.onCancel();
 
-                    return null;
-                }
-            });
-
-            if (!nodes.isEmpty()) {
-                for (ClusterNode node : nodes) {
-                    try {
-                        cctx.io().send(node, req, cctx.ioPolicy());
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (cctx.io().checkNodeLeft(node.id(), e, false)) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to send cancel request, node failed: " + node);
-                        }
-                        else
-                            U.error(log, "Failed to send cancel request [node=" + node + ']', e);
-                    }
-                }
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
-        }
-
-        qryMgr.onQueryFutureCanceled(reqId);
+        cctx.queries().onQueryFutureCanceled(reqId);
 
         clear();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
     @Override protected void onNodeLeft(UUID nodeId) {
-        boolean callOnPage;
-
-        synchronized (this) {
-            callOnPage = !loc && subgrid.contains(nodeId);
-        }
+        boolean qryNode = reducer.mapNode(nodeId);
 
-        if (callOnPage)
-            onPage(nodeId, Collections.emptyList(),
+        if (qryNode)
+            onPage(nodeId, null,
                 new ClusterTopologyCheckedException("Remote node has left topology: " + nodeId), true);
     }
 
     /** {@inheritDoc} */
-    @Override public void awaitFirstPage() throws IgniteCheckedException {
-        try {
-            firstPageLatch.await();
+    @Override public void awaitFirstItemAvailable() throws IgniteCheckedException {
+        reducer.awaitInitialization();
 
-            if (isDone() && error() != null)
-                // Throw the exception if future failed.
-                get();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
+        if (isDone() && error() != null)
+            // Throw the exception if future failed.
+            get();
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean onPage(UUID nodeId, boolean last) {
-        assert Thread.holdsLock(this);
-
-        if (!loc) {
-            rcvd.add(nodeId);
-
-            if (rcvd.containsAll(subgrid))
-                firstPageLatch.countDown();
-        }
-
-        boolean futFinish;
-
-        if (last) {
-            futFinish = loc || (subgrid.remove(nodeId) && subgrid.isEmpty());
-
-            if (futFinish)
-                firstPageLatch.countDown();
-        }
-        else
-            futFinish = false;
-
-        return futFinish;
+    @Override protected CacheQueryReducer<R> reducer() {
+        return reducer;
     }
 
-    /** {@inheritDoc} */
-    @Override protected void loadPage() {
-        assert !Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && rcvd.containsAll(subgrid)) {
-                rcvd.clear();
-
-                nodes = nodes();
-            }
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, false);
+    /** Set reducer. */
+    void reducer(DistributedCacheQueryReducer<R> reducer) {
+        this.reducer = reducer;
     }
 
     /** {@inheritDoc} */
-    @Override protected void loadAllPages() throws IgniteInterruptedCheckedException {
-        assert !Thread.holdsLock(this);
-
-        U.await(firstPageLatch);
-
-        Collection<ClusterNode> nodes = null;
-
-        synchronized (this) {
-            if (!isDone() && !subgrid.isEmpty())
-                nodes = nodes();
-        }
-
-        if (nodes != null)
-            cctx.queries().loadPage(reqId, qry.query(), nodes, true);
-    }
-
-    /**
-     * @return Nodes to send requests to.
-     */
-    private Collection<ClusterNode> nodes() {
-        assert Thread.holdsLock(this);
-
-        Collection<ClusterNode> nodes = new ArrayList<>(subgrid.size());
-
-        for (UUID nodeId : subgrid) {
-            ClusterNode node = cctx.discovery().node(nodeId);
-
-            if (node != null)
-                nodes.add(node);
-        }
-
-        return nodes;
+    @Override public Collection<R> get() throws IgniteCheckedException {
+        return get0();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onDone(Collection<R> res, Throwable err) {
-        boolean done = super.onDone(res, err);
-
-        // Must release the lath after onDone() in order for a waiting thread to see an exception, if any.
-        firstPageLatch.countDown();
-
-        return done;
+    @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteCheckedException {
+        return get0();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean onCancelled() {
-        firstPageLatch.countDown();
-
-        return super.onCancelled();
+    @Override public Collection<R> getUninterruptibly() throws IgniteCheckedException {
+        return get0();
     }
 
-    /** {@inheritDoc} */
-    @Override public void onTimeout() {
-        firstPageLatch.countDown();
+    /**
+     * Completion of distributed query future depends on user that iterates over query result with lazy page loading.
+     * Then {@link #get()} can lock on unpredictably long period of time. So we should avoid call it.
+     */
+    private Collection<R> get0() throws IgniteCheckedException {
+        if (!isDone())
+            throw new IgniteIllegalStateException("Unexpected lock on iterator over distributed cache query result.");

Review comment:
       The only place we use get() it's in this class, some lines above.
   
   https://github.com/apache/ignite/blob/0add93b488ebe7265b2a0f951fcdfd55adbcc01f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryFuture.java#L82
   
   But I agree, that it a little bit misleading. So I will replace this line of code with `super.get()` and then we can forbid `get()` at all.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] timoninmaxim commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
timoninmaxim commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r682466010



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/DistributedCacheQueryReducer.java
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.cache.query;
+
+import java.util.UUID;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+
+/**
+ * Reducer for distributed cache query.
+ */
+public interface DistributedCacheQueryReducer<T> extends CacheQueryReducer<T> {
+    /**
+     * Checks whether node with provided {@code nodeId} is a map node for the query.
+     * Note: if all pages were received from this node, then the method will return {@code false}.
+     *
+     * @param nodeId Node ID.
+     * @return {@code true} if node with provided {@code nodeId} is a map node for the query, {@code false} otherwise.
+     */
+    public boolean mapNode(UUID nodeId);

Review comment:
       According to code style [1] we should not use "is" as prefix for internal API.
   
   [1] https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-Gettersandsetters
   
   




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r645145710



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/CacheQueryReducer.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.cache.query;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is responsible for reducing results of cache query. Query results are delivered with function
+ * {@link #addPage(UUID, Collection)}. Note that this reducer deeply interacts with corresponding query future
+ * {@link GridCacheQueryFutureAdapter}, so they used the same lock object. It guards reducer pages operations
+ * and the future status. Custom reduce logic is applied within {@link #next()} and {@link #hasNext()}.
+ *
+ * <T> is a type of cache query result item.
+ */
+public interface CacheQueryReducer<T> {
+    /**
+     * @return Next item.
+     */
+    public T next() throws IgniteCheckedException;
+
+    /**
+     * @return {@code true} if there is a next item, otherwise {@code false}.
+     */
+    public boolean hasNext() throws IgniteCheckedException;
+
+    /**
+     * Offer query result page for reduce. Note that the data collection may contain extension of type T.
+     * In such cases it stores additional payload for custom reducer logic.
+     *
+     * @param nodeId Node ID that sent this page. {@code null} means local node or error page.
+     * @param data Page data rows.
+     */
+    public void addPage(@Nullable UUID nodeId, Collection<T> data);
+
+    /**
+     * Callback that invokes after reducer get last query result page.
+     * Also invokes for failed queries to let reducer know that there won't be new pages.

Review comment:
       Why there is no onCancel() or onError() callbacks that will cause Exception throwing in `next` method?




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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9081: IGNITE-14703. Add MergeSort distributed cache query reducer.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #9081:
URL: https://github.com/apache/ignite/pull/9081#discussion_r714008233



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.NoSuchElementException;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /**
+     * Queue of pages from all nodes. Order of streams is set with {@link #pageCmp}.
+     */
+    private final PriorityQueue<NodePage<R>> nodePages;
+
+    /**
+     * Compares head pages from all nodes to get the lowest value at the moment.
+     */
+    private final Comparator<NodePage<R>> pageCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester, queueLock, nodes);
+
+        pageCmp = (o1, o2) -> rowCmp.compare(o1.head(), o2.head());

Review comment:
       Seems, pageCmp can be local variable.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.NoSuchElementException;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /**
+     * Queue of pages from all nodes. Order of streams is set with {@link #pageCmp}.
+     */
+    private final PriorityQueue<NodePage<R>> nodePages;
+
+    /**
+     * Compares head pages from all nodes to get the lowest value at the moment.
+     */
+    private final Comparator<NodePage<R>> pageCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester, queueLock, nodes);
+
+        pageCmp = (o1, o2) -> rowCmp.compare(o1.head(), o2.head());
+
+        nodePages = new PriorityQueue<>(nodes.size(), pageCmp);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        // Initial sort.
+        if (nodePages.isEmpty() && !streams.isEmpty()) {
+            Set<UUID> nodes = new HashSet<>(streams.keySet());
+
+            for (UUID nodeId : nodes)
+                fillWithPage(nodeId);
+        }
+
+        return !nodePages.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        if (nodePages.isEmpty())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        NodePage<R> page = nodePages.poll();
+
+        R o = page.next();
+
+        if (page.hasNext())
+            nodePages.offer(page);
+        else
+            fillWithPage(page.nodeId());
+
+        return o;
+    }
+
+    /** */
+    private void fillWithPage(UUID nodeId) throws IgniteCheckedException {
+        NodePage<R> page = streams.get(nodeId).nextPage();
+
+        if (!page.hasNext())
+            streams.remove(nodeId);
+        else
+            nodePages.offer(page);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        super.cancel();
+
+        nodePages.clear();

Review comment:
       Not in sync with the fillWithPage() method.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/reducer/MergeSortDistributedCacheQueryReducer.java
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.cache.query.reducer;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.NoSuchElementException;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.processors.cache.query.CacheQueryPageRequester;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter;
+
+/**
+ * Reducer of distirbuted query that sort result through all nodes. Note that it's assumed that every node
+ * returns pre-sorted collection of data.
+ */
+public class MergeSortDistributedCacheQueryReducer<R> extends AbstractDistributedCacheQueryReducer<R> {
+    /**
+     * Queue of pages from all nodes. Order of streams is set with {@link #pageCmp}.
+     */
+    private final PriorityQueue<NodePage<R>> nodePages;
+
+    /**
+     * Compares head pages from all nodes to get the lowest value at the moment.
+     */
+    private final Comparator<NodePage<R>> pageCmp;
+
+    /**
+     * @param rowCmp Comparator to sort query results from different nodes.
+     */
+    public MergeSortDistributedCacheQueryReducer(
+        GridCacheQueryFutureAdapter fut, long reqId, CacheQueryPageRequester pageRequester,
+        Object queueLock, Collection<ClusterNode> nodes, Comparator<R> rowCmp
+    ) {
+        super(fut, reqId, pageRequester, queueLock, nodes);
+
+        pageCmp = (o1, o2) -> rowCmp.compare(o1.head(), o2.head());
+
+        nodePages = new PriorityQueue<>(nodes.size(), pageCmp);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() throws IgniteCheckedException {
+        // Initial sort.
+        if (nodePages.isEmpty() && !streams.isEmpty()) {
+            Set<UUID> nodes = new HashSet<>(streams.keySet());
+
+            for (UUID nodeId : nodes)
+                fillWithPage(nodeId);
+        }
+
+        return !nodePages.isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override public R next() throws IgniteCheckedException {
+        if (nodePages.isEmpty())
+            throw new NoSuchElementException("No next element. Please, be sure to invoke hasNext() before next().");
+
+        NodePage<R> page = nodePages.poll();
+
+        R o = page.next();
+
+        if (page.hasNext())
+            nodePages.offer(page);
+        else
+            fillWithPage(page.nodeId());
+
+        return o;
+    }
+
+    /** */
+    private void fillWithPage(UUID nodeId) throws IgniteCheckedException {
+        NodePage<R> page = streams.get(nodeId).nextPage();
+
+        if (!page.hasNext())
+            streams.remove(nodeId);
+        else
+            nodePages.offer(page);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        super.cancel();
+
+        nodePages.clear();

Review comment:
       Seems, it is possible to lost last pages silently on concurrent cancellation.




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

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org