/** @return Nodes to execute on. */
  private Collection<ClusterNode> nodes() {
    CacheMode cacheMode = cctx.config().getCacheMode();

    switch (cacheMode) {
      case LOCAL:
        if (prj != null)
          U.warn(
              log,
              "Ignoring query projection because it's executed over LOCAL cache "
                  + "(only local node will be queried): "
                  + this);

        return Collections.singletonList(cctx.localNode());

      case REPLICATED:
        if (prj != null || partition() != null) return nodes(cctx, prj, partition());

        return cctx.affinityNode()
            ? Collections.singletonList(cctx.localNode())
            : Collections.singletonList(F.rand(nodes(cctx, null, partition())));

      case PARTITIONED:
        return nodes(cctx, prj, partition());

      default:
        throw new IllegalStateException("Unknown cache distribution mode: " + cacheMode);
    }
  }
  /**
   * Calculates partition mapping for partitioned cache on unstable topology.
   *
   * @param cctx Cache context for main space.
   * @param extraSpaces Extra spaces.
   * @return Partition mapping or {@code null} if we can't calculate it due to repartitioning and we
   *     need to retry.
   */
  @SuppressWarnings("unchecked")
  private Map<ClusterNode, IntArray> partitionedUnstableDataNodes(
      final GridCacheContext<?, ?> cctx, List<String> extraSpaces) {
    assert !cctx.isReplicated() && !cctx.isLocal() : cctx.name() + " must be partitioned";

    final int partsCnt = cctx.affinity().partitions();

    if (extraSpaces != null) { // Check correct number of partitions for partitioned caches.
      for (String extraSpace : extraSpaces) {
        GridCacheContext<?, ?> extraCctx = cacheContext(extraSpace);

        if (extraCctx.isReplicated() || extraCctx.isLocal()) continue;

        int parts = extraCctx.affinity().partitions();

        if (parts != partsCnt)
          throw new CacheException(
              "Number of partitions must be the same for correct collocation [cache1="
                  + cctx.name()
                  + ", parts1="
                  + partsCnt
                  + ", cache2="
                  + extraSpace
                  + ", parts2="
                  + parts
                  + "]");
      }
    }

    Set<ClusterNode>[] partLocs = new Set[partsCnt];

    // Fill partition locations for main cache.
    for (int p = 0, parts = cctx.affinity().partitions(); p < parts; p++) {
      List<ClusterNode> owners = cctx.topology().owners(p);

      if (F.isEmpty(owners)) {
        if (!F.isEmpty(dataNodes(cctx.name(), NONE))) return null; // Retry.

        throw new CacheException(
            "Failed to find data nodes [cache=" + cctx.name() + ", part=" + p + "]");
      }

      partLocs[p] = new HashSet<>(owners);
    }

    if (extraSpaces != null) {
      // Find owner intersections for each participating partitioned cache partition.
      // We need this for logical collocation between different partitioned caches with the same
      // affinity.
      for (String extraSpace : extraSpaces) {
        GridCacheContext<?, ?> extraCctx = cacheContext(extraSpace);

        if (extraCctx.isReplicated() || extraCctx.isLocal()) continue;

        for (int p = 0, parts = extraCctx.affinity().partitions(); p < parts; p++) {
          List<ClusterNode> owners = extraCctx.topology().owners(p);

          if (F.isEmpty(owners)) {
            if (!F.isEmpty(dataNodes(extraSpace, NONE))) return null; // Retry.

            throw new CacheException(
                "Failed to find data nodes [cache=" + extraSpace + ", part=" + p + "]");
          }

          if (partLocs[p] == null) partLocs[p] = new HashSet<>(owners);
          else {
            partLocs[p].retainAll(owners); // Intersection of owners.

            if (partLocs[p].isEmpty()) return null; // Intersection is empty -> retry.
          }
        }
      }

      // Filter nodes where not all the replicated caches loaded.
      for (String extraSpace : extraSpaces) {
        GridCacheContext<?, ?> extraCctx = cacheContext(extraSpace);

        if (!extraCctx.isReplicated()) continue;

        Set<ClusterNode> dataNodes = replicatedUnstableDataNodes(extraCctx);

        if (F.isEmpty(dataNodes)) return null; // Retry.

        for (Set<ClusterNode> partLoc : partLocs) {
          partLoc.retainAll(dataNodes);

          if (partLoc.isEmpty()) return null; // Retry.
        }
      }
    }

    // Collect the final partitions mapping.
    Map<ClusterNode, IntArray> res = new HashMap<>();

    // Here partitions in all IntArray's will be sorted in ascending order, this is important.
    for (int p = 0; p < partLocs.length; p++) {
      Set<ClusterNode> pl = partLocs[p];

      assert !F.isEmpty(pl) : pl;

      ClusterNode n = pl.size() == 1 ? F.first(pl) : F.rand(pl);

      IntArray parts = res.get(n);

      if (parts == null) res.put(n, parts = new IntArray());

      parts.add(p);
    }

    return res;
  }
  /**
   * @param cctx Cache context.
   * @param qry Query.
   * @param keepPortable Keep portable.
   * @return Cursor.
   */
  public Iterator<List<?>> query(
      GridCacheContext<?, ?> cctx, GridCacheTwoStepQuery qry, boolean keepPortable) {
    for (int attempt = 0; ; attempt++) {
      if (attempt != 0) {
        try {
          Thread.sleep(attempt * 10); // Wait for exchange.
        } catch (InterruptedException e) {
          Thread.currentThread().interrupt();

          throw new CacheException("Query was interrupted.", e);
        }
      }

      long qryReqId = reqIdGen.incrementAndGet();

      QueryRun r = new QueryRun();

      r.pageSize = qry.pageSize() <= 0 ? GridCacheTwoStepQuery.DFLT_PAGE_SIZE : qry.pageSize();

      r.idxs = new ArrayList<>(qry.mapQueries().size());

      String space = cctx.name();

      r.conn = (JdbcConnection) h2.connectionForSpace(space);

      AffinityTopologyVersion topVer = h2.readyTopologyVersion();

      List<String> extraSpaces = extraSpaces(space, qry.spaces());

      Collection<ClusterNode> nodes;

      // Explicit partition mapping for unstable topology.
      Map<ClusterNode, IntArray> partsMap = null;

      if (isPreloadingActive(cctx, extraSpaces)) {
        if (cctx.isReplicated()) nodes = replicatedUnstableDataNodes(cctx, extraSpaces);
        else {
          partsMap = partitionedUnstableDataNodes(cctx, extraSpaces);

          nodes = partsMap == null ? null : partsMap.keySet();
        }
      } else nodes = stableDataNodes(topVer, cctx, extraSpaces);

      if (nodes == null) continue; // Retry.

      assert !nodes.isEmpty();

      if (cctx.isReplicated() || qry.explain()) {
        assert qry.explain() || !nodes.contains(ctx.discovery().localNode())
            : "We must be on a client node.";

        // Select random data node to run query on a replicated data or get EXPLAIN PLAN from a
        // single node.
        nodes = Collections.singleton(F.rand(nodes));
      }

      int tblIdx = 0;

      final boolean skipMergeTbl = !qry.explain() && qry.skipMergeTable();

      for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
        GridMergeIndex idx;

        if (!skipMergeTbl) {
          GridMergeTable tbl;

          try {
            tbl = createMergeTable(r.conn, mapQry, qry.explain());
          } catch (IgniteCheckedException e) {
            throw new IgniteException(e);
          }

          idx = tbl.getScanIndex(null);

          fakeTable(r.conn, tblIdx++).setInnerTable(tbl);
        } else idx = GridMergeIndexUnsorted.createDummy();

        for (ClusterNode node : nodes) idx.addSource(node.id());

        r.idxs.add(idx);
      }

      r.latch = new CountDownLatch(r.idxs.size() * nodes.size());

      runs.put(qryReqId, r);

      try {
        if (ctx.clientDisconnected()) {
          throw new CacheException(
              "Query was cancelled, client node disconnected.",
              new IgniteClientDisconnectedException(
                  ctx.cluster().clientReconnectFuture(), "Client node disconnected."));
        }

        Collection<GridCacheSqlQuery> mapQrys = qry.mapQueries();

        if (qry.explain()) {
          mapQrys = new ArrayList<>(qry.mapQueries().size());

          for (GridCacheSqlQuery mapQry : qry.mapQueries())
            mapQrys.add(new GridCacheSqlQuery("EXPLAIN " + mapQry.query(), mapQry.parameters()));
        }

        if (nodes.size() != 1 || !F.first(nodes).isLocal()) { // Marshall params for remotes.
          Marshaller m = ctx.config().getMarshaller();

          for (GridCacheSqlQuery mapQry : mapQrys) mapQry.marshallParams(m);
        }

        boolean retry = false;

        if (send(
            nodes,
            new GridQueryRequest(qryReqId, r.pageSize, space, mapQrys, topVer, extraSpaces, null),
            partsMap)) {
          awaitAllReplies(r, nodes);

          Object state = r.state.get();

          if (state != null) {
            if (state instanceof CacheException) {
              CacheException err = (CacheException) state;

              if (err.getCause() instanceof IgniteClientDisconnectedException) throw err;

              throw new CacheException("Failed to run map query remotely.", err);
            }

            if (state instanceof AffinityTopologyVersion) {
              retry = true;

              // If remote node asks us to retry then we have outdated full partition map.
              h2.awaitForReadyTopologyVersion((AffinityTopologyVersion) state);
            }
          }
        } else // Send failed.
        retry = true;

        Iterator<List<?>> resIter = null;

        if (!retry) {
          if (qry.explain()) return explainPlan(r.conn, space, qry);

          if (skipMergeTbl) {
            List<List<?>> res = new ArrayList<>();

            assert r.idxs.size() == 1 : r.idxs;

            GridMergeIndex idx = r.idxs.get(0);

            Cursor cur = idx.findInStream(null, null);

            while (cur.next()) {
              Row row = cur.get();

              int cols = row.getColumnCount();

              List<Object> resRow = new ArrayList<>(cols);

              for (int c = 0; c < cols; c++) resRow.add(row.getValue(c).getObject());

              res.add(resRow);
            }

            resIter = res.iterator();
          } else {
            GridCacheSqlQuery rdc = qry.reduceQuery();

            // Statement caching is prohibited here because we can't guarantee correct merge index
            // reuse.
            ResultSet res =
                h2.executeSqlQueryWithTimer(
                    space, r.conn, rdc.query(), F.asList(rdc.parameters()), false);

            resIter = new Iter(res);
          }
        }

        for (GridMergeIndex idx : r.idxs) {
          if (!idx.fetchedAll()) // We have to explicitly cancel queries on remote nodes.
          send(nodes, new GridQueryCancelRequest(qryReqId), null);
        }

        if (retry) {
          if (Thread.currentThread().isInterrupted())
            throw new IgniteInterruptedCheckedException("Query was interrupted.");

          continue;
        }

        return new GridQueryCacheObjectsIterator(resIter, cctx, keepPortable);
      } catch (IgniteCheckedException | RuntimeException e) {
        U.closeQuiet(r.conn);

        if (e instanceof CacheException) throw (CacheException) e;

        Throwable cause = e;

        if (e instanceof IgniteCheckedException) {
          Throwable disconnectedErr =
              ((IgniteCheckedException) e).getCause(IgniteClientDisconnectedException.class);

          if (disconnectedErr != null) cause = disconnectedErr;
        }

        throw new CacheException("Failed to run reduce query locally.", cause);
      } finally {
        if (!runs.remove(qryReqId, r)) U.warn(log, "Query run was already removed: " + qryReqId);

        if (!skipMergeTbl) {
          for (int i = 0, mapQrys = qry.mapQueries().size(); i < mapQrys; i++)
            fakeTable(null, i).setInnerTable(null); // Drop all merge tables.
        }
      }
    }
  }