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: [email protected]
