You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2015/06/19 01:55:04 UTC

[01/50] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1026 bb73b66c6 -> 1419d39ba


Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/51bf4b15
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/51bf4b15
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/51bf4b15

Branch: refs/heads/ignite-1026
Commit: 51bf4b1565a263a919635ab34f09f8673d8e42c9
Parents: b5db559 89a4f7c
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Jun 11 10:00:07 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Jun 11 10:00:07 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   6 +
 assembly/dependencies-fabric.xml                |   1 +
 examples/pom.xml                                |  34 ++
 modules/core/pom.xml                            |   1 -
 .../apache/ignite/cache/query/ScanQuery.java    |  45 +-
 .../configuration/CacheConfiguration.java       |   1 -
 .../processors/cache/GridCacheAdapter.java      |  15 +-
 .../processors/cache/GridCacheProcessor.java    |  30 +-
 .../processors/cache/GridCacheSwapManager.java  |  55 ++-
 .../processors/cache/IgniteCacheProxy.java      |  11 +-
 .../processors/cache/QueryCursorImpl.java       |  23 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   7 +
 .../processors/cache/query/CacheQuery.java      |   2 +-
 .../query/GridCacheDistributedQueryManager.java |   3 +
 .../cache/query/GridCacheQueryAdapter.java      | 147 ++++++-
 .../cache/query/GridCacheQueryManager.java      | 209 ++++++----
 .../cache/query/GridCacheQueryRequest.java      |  47 ++-
 .../processors/cache/query/QueryCursorEx.java   |   8 +
 .../datastructures/GridCacheSetImpl.java        |   4 +-
 .../processors/query/GridQueryIndexing.java     |   4 +-
 .../processors/query/GridQueryProcessor.java    |  18 +-
 .../service/GridServiceProcessor.java           |   2 +-
 .../ignite/internal/util/GridJavaProcess.java   |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |   4 +-
 .../shmem/IpcSharedMemoryClientEndpoint.java    |   2 +-
 .../ipc/shmem/IpcSharedMemoryNativeLoader.java  | 151 ++++++-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 .../util/nio/GridShmemCommunicationClient.java  | 146 +++++++
 .../communication/tcp/TcpCommunicationSpi.java  | 415 ++++++++++++++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |   8 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |  15 +
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  19 +
 .../IgniteCacheMessageRecoveryAbstractTest.java |   1 +
 ...achePartitionedPreloadLifecycleSelfTest.java |   2 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |   6 +-
 .../GridCacheSwapScanQueryAbstractSelfTest.java | 112 +++--
 .../ipc/shmem/IgfsSharedMemoryTestServer.java   |   2 +
 .../IpcSharedMemoryCrashDetectionSelfTest.java  |   2 +-
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |   2 +-
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |   2 +-
 .../LoadWithCorruptedLibFileTestRunner.java     |   2 +-
 .../IpcSharedMemoryBenchmarkReader.java         |   2 +-
 .../IpcSharedMemoryBenchmarkWriter.java         |   2 +-
 .../communication/GridIoManagerBenchmark0.java  |   1 +
 .../spi/GridTcpSpiForwardingSelfTest.java       |   1 +
 .../GridTcpCommunicationSpiAbstractTest.java    |  13 +
 ...mmunicationSpiConcurrentConnectSelfTest.java |   4 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |  21 +-
 ...pCommunicationSpiMultithreadedShmemTest.java |  28 ++
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |   1 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   1 +
 .../GridTcpCommunicationSpiShmemSelfTest.java   |  38 ++
 .../tcp/GridTcpCommunicationSpiTcpSelfTest.java |   7 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |   2 +
 modules/hadoop/pom.xml                          |   1 +
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  13 +
 ...oopSecondaryFileSystemConfigurationTest.java |  14 +
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |   7 +
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   7 +
 .../hadoop/HadoopAbstractSelfTest.java          |   7 +
 .../processors/query/h2/IgniteH2Indexing.java   |  44 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  23 +-
 ...CacheScanPartitionQueryFallbackSelfTest.java | 408 ++++++++++++++++++
 .../cache/GridCacheCrossCacheQuerySelfTest.java |  12 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  77 +++-
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 modules/scalar-2.10/README.txt                  |   4 +
 modules/scalar-2.10/licenses/apache-2.0.txt     | 202 +++++++++
 .../scalar-2.10/licenses/scala-bsd-license.txt  |  18 +
 modules/scalar-2.10/pom.xml                     | 197 +++++++++
 modules/spark-2.10/README.txt                   |   4 +
 modules/spark-2.10/licenses/apache-2.0.txt      | 202 +++++++++
 .../spark-2.10/licenses/scala-bsd-license.txt   |  18 +
 modules/spark-2.10/pom.xml                      | 120 ++++++
 modules/spark/README.txt                        |   8 +
 modules/spark/licenses/apache-2.0.txt           | 202 +++++++++
 modules/spark/licenses/scala-bsd-license.txt    |  18 +
 modules/spark/pom.xml                           | 114 +++++
 .../org/apache/ignite/spark/IgniteContext.scala | 119 ++++++
 .../org/apache/ignite/spark/IgniteRDD.scala     | 244 +++++++++++
 .../apache/ignite/spark/JavaIgniteContext.scala |  63 +++
 .../org/apache/ignite/spark/JavaIgniteRDD.scala |  99 +++++
 .../ignite/spark/impl/IgniteAbstractRDD.scala   |  39 ++
 .../ignite/spark/impl/IgnitePartition.scala     |  24 ++
 .../ignite/spark/impl/IgniteQueryIterator.scala |  27 ++
 .../apache/ignite/spark/impl/IgniteSqlRDD.scala |  41 ++
 .../spark/impl/JavaIgniteAbstractRDD.scala      |  34 ++
 .../ignite/spark/JavaIgniteRDDSelfTest.java     | 298 +++++++++++++
 .../scala/org/apache/ignite/spark/Entity.scala  |  28 ++
 .../org/apache/ignite/spark/IgniteRddSpec.scala | 231 +++++++++++
 modules/visor-console-2.10/README.txt           |   4 +
 modules/visor-console-2.10/pom.xml              | 174 ++++++++
 parent/pom.xml                                  |   4 +
 pom.xml                                         |  20 +-
 94 files changed, 4595 insertions(+), 265 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf4b15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf4b15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf4b15/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf4b15/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 49d2401,e187713..282875d
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@@ -706,8 -718,7 +709,8 @@@ public class GridQueryProcessor extend
                      String sql = qry.getSql();
                      Object[] args = qry.getArgs();
  
-                     GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args),
 -                    final GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
++                    final GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args),
 +                        idx.backupFilter(null, null, null));
  
                      sendQueryExecutedEvent(sql, args);
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf4b15/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf4b15/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 03da6d3,11054b7..3d2ae46
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@@ -397,353 -269,109 +397,338 @@@ public class GridReduceQueryExecutor 
       * @param qry Query.
       * @return Cursor.
       */
-     public QueryCursor<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry) {
 -    public Iterator<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry, boolean keepPortable) {
 -        long qryReqId = reqIdGen.incrementAndGet();
++    public QueryCursor<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry, boolean keepPortable) {
 +        for (;;) {
 +            long qryReqId = reqIdGen.incrementAndGet();
  
 -        QueryRun r = new QueryRun();
 +            QueryRun r = new QueryRun();
  
 -        r.pageSize = qry.pageSize() <= 0 ? GridCacheTwoStepQuery.DFLT_PAGE_SIZE : qry.pageSize();
 +            r.pageSize = qry.pageSize() <= 0 ? GridCacheTwoStepQuery.DFLT_PAGE_SIZE : qry.pageSize();
  
 -        r.tbls = new ArrayList<>(qry.mapQueries().size());
 +            r.tbls = new ArrayList<>(qry.mapQueries().size());
  
 -        String space = cctx.name();
 +            String space = cctx.name();
  
 -        r.conn = (JdbcConnection)h2.connectionForSpace(space);
 +            r.conn = (JdbcConnection)h2.connectionForSpace(space);
  
 -        // TODO    Add topology version.
 -        ClusterGroup dataNodes = ctx.grid().cluster().forDataNodes(space);
 +            AffinityTopologyVersion topVer = h2.readyTopologyVersion();
  
 -        if (cctx.isReplicated() || qry.explain()) {
 -            assert qry.explain() || dataNodes.node(ctx.localNodeId()) == null : "We must be on a client node.";
 +            List<String> extraSpaces = extraSpaces(space, qry.spaces());
  
 -            // Select random data node to run query on a replicated data or get EXPLAIN PLAN from a single node.
 -            dataNodes = dataNodes.forRandom();
 -        }
 +            Collection<ClusterNode> nodes;
  
 -        final Collection<ClusterNode> nodes = dataNodes.nodes();
 +            // Explicit partition mapping for unstable topology.
 +            Map<ClusterNode, IntArray> partsMap = null;
  
 -        for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
 -            GridMergeTable tbl;
 +            if (isPreloadingActive(cctx, extraSpaces)) {
 +                if (cctx.isReplicated())
 +                    nodes = replicatedDataNodes(cctx, extraSpaces);
 +                else {
 +                    partsMap = partitionLocations(cctx, extraSpaces);
  
 -            try {
 -                tbl = createFunctionTable(r.conn, mapQry, qry.explain()); // createTable(r.conn, mapQry); TODO
 +                    nodes = partsMap == null ? null : partsMap.keySet();
 +                }
              }
 -            catch (IgniteCheckedException e) {
 -                throw new IgniteException(e);
 +            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.cluster().get().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));
 +            }
 +
 +            for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
 +                GridMergeTable tbl;
 +
 +                try {
 +                    tbl = createFunctionTable(r.conn, mapQry, qry.explain()); // createTable(r.conn, mapQry); TODO
 +                }
 +                catch (IgniteCheckedException e) {
 +                    throw new IgniteException(e);
 +                }
 +
 +                GridMergeIndex idx = tbl.getScanIndex(null);
 +
 +                for (ClusterNode node : nodes)
 +                    idx.addSource(node.id());
 +
 +                r.tbls.add(tbl);
 +
 +                curFunTbl.set(tbl);
              }
  
 -            GridMergeIndex idx = tbl.getScanIndex(null);
 +            r.latch = new CountDownLatch(r.tbls.size() * nodes.size());
 +
 +            runs.put(qryReqId, r);
 +
 +            try {
 +                Collection<GridCacheSqlQuery> mapQrys = qry.mapQueries();
 +
 +                if (qry.explain()) {
 +                    mapQrys = new ArrayList<>(qry.mapQueries().size());
 +
 +                    for (GridCacheSqlQuery mapQry : qry.mapQueries())
 +                        mapQrys.add(new GridCacheSqlQuery(mapQry.alias(), "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)) {
 +                    U.await(r.latch);
 +
 +                    Object state = r.state.get();
 +
 +                    if (state != null) {
 +                        if (state instanceof CacheException)
 +                            throw new CacheException("Failed to run map query remotely.", (CacheException)state);
 +
 +                        if (state instanceof AffinityTopologyVersion) {
 +                            retry = true;
 +
 +                            // If remote node asks us to retry then we have outdated full partition map.
 +                            // TODO is this correct way to wait for a new map??
 +                            h2.awaitForReadyTopologyVersion((AffinityTopologyVersion)state);
 +                        }
 +                    }
 +                }
 +                else // Send failed.
 +                    retry = true;
 +
 +                ResultSet res = null;
 +
 +                if (!retry) {
 +                    if (qry.explain())
 +                        return explainPlan(r.conn, space, qry);
 +
 +                    GridCacheSqlQuery rdc = qry.reduceQuery();
 +
 +                    res = h2.executeSqlQueryWithTimer(space, r.conn, rdc.query(), F.asList(rdc.parameters()));
 +                }
 +
 +                for (GridMergeTable tbl : r.tbls) {
 +                    if (!tbl.getScanIndex(null).fetchedAll()) // We have to explicitly cancel queries on remote nodes.
 +                        send(nodes, new GridQueryCancelRequest(qryReqId), null);
 +
 +//                dropTable(r.conn, tbl.getName()); TODO
 +                }
  
 -            for (ClusterNode node : nodes)
 -                idx.addSource(node.id());
 +                if (retry) {
 +                    if (Thread.currentThread().isInterrupted())
 +                        throw new IgniteInterruptedCheckedException("Query was interrupted.");
  
 -            r.tbls.add(tbl);
 +                    continue;
 +                }
 +
 +                return new QueryCursorImpl<>(new GridQueryCacheObjectsIterator(new Iter(res), cctx, cctx.keepPortable()));
 +            }
 +            catch (IgniteCheckedException | RuntimeException e) {
 +                U.closeQuiet(r.conn);
  
 -            curFunTbl.set(tbl);
 +                if (e instanceof CacheException)
 +                    throw (CacheException)e;
 +
 +                throw new CacheException("Failed to run reduce query locally.", e);
 +            }
 +            finally {
 +                if (!runs.remove(qryReqId, r))
 +                    U.warn(log, "Query run was already removed: " + qryReqId);
 +
 +                curFunTbl.remove();
 +            }
          }
 +    }
 +
 +    /**
 +     * Calculates data nodes for replicated caches on unstable topology.
 +     *
 +     * @param cctx Cache context for main space.
 +     * @param extraSpaces Extra spaces.
 +     * @return Collection of all data nodes owning all the caches or {@code null} for retry.
 +     */
 +    private Collection<ClusterNode> replicatedDataNodes(final GridCacheContext<?,?> cctx, List<String> extraSpaces) {
 +        assert cctx.isReplicated() : cctx.name() + " must be replicated";
  
 -        r.latch = new CountDownLatch(r.tbls.size() * nodes.size());
 +        Set<ClusterNode> nodes = owningReplicatedDataNodes(cctx);
  
 -        runs.put(qryReqId, r);
 +        if (!F.isEmpty(extraSpaces)) {
 +            for (String extraSpace : extraSpaces) {
 +                GridCacheContext<?,?> extraCctx = cacheContext(extraSpace);
  
 -        try {
 -            Collection<GridCacheSqlQuery> mapQrys = qry.mapQueries();
 +                if (extraCctx.isLocal())
 +                    continue;
  
 -            if (qry.explain()) {
 -                mapQrys = new ArrayList<>(qry.mapQueries().size());
 +                if (!extraCctx.isReplicated())
 +                    throw new CacheException("Queries running on replicated cache should not contain JOINs " +
 +                        "with partitioned tables.");
  
 -                for (GridCacheSqlQuery mapQry : qry.mapQueries())
 -                    mapQrys.add(new GridCacheSqlQuery(mapQry.alias(), "EXPLAIN " + mapQry.query(), mapQry.parameters()));
 +                nodes.retainAll(owningReplicatedDataNodes(extraCctx));
 +
 +                if (nodes.isEmpty())
 +                    return null; // Retry.
              }
 +        }
  
 -            if (nodes.size() != 1 || !F.first(nodes).isLocal()) { // Marshall params for remotes.
 -                Marshaller m = ctx.config().getMarshaller();
 +        return nodes;
 +    }
  
 -                for (GridCacheSqlQuery mapQry : mapQrys)
 -                    mapQry.marshallParams(m);
 +    /**
 +     * Collects all the nodes owning all the partitions for the given replicated cache.
 +     *
 +     * @param cctx Cache context.
 +     * @return Owning nodes.
 +     */
 +    private Set<ClusterNode> owningReplicatedDataNodes(GridCacheContext<?,?> cctx) {
 +        assert cctx.isReplicated() : cctx.name() + " must be replicated";
 +
 +        String space = cctx.name();
 +
 +        Set<ClusterNode> dataNodes = new HashSet<>(ctx.discovery().cacheAffinityNodes(space, NONE));
 +
 +        if (dataNodes.isEmpty())
 +            throw new CacheException("No data nodes found for cache '" + space + "'");
 +
 +        // Find all the nodes owning all the partitions for replicated cache.
 +        for (int p = 0, extraParts = cctx.affinity().partitions(); p < extraParts; p++) {
 +            List<ClusterNode> owners = cctx.topology().owners(p);
 +
 +            if (owners.isEmpty())
 +                throw new CacheException("No data nodes found for cache '" + space +
 +                    "' for partition " + p);
 +
 +            dataNodes.retainAll(owners);
 +
 +            if (dataNodes.isEmpty())
 +                throw new CacheException("No data nodes found for cache '" + space +
 +                    "' owning all the partitions.");
 +        }
 +
 +        return dataNodes;
 +    }
 +
 +    /**
 +     * 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> partitionLocations(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 in " +
 +                        "caches " + cctx.name() + " and " + extraSpace + ".");
              }
 +        }
  
 -            send(nodes, new GridQueryRequest(qryReqId, r.pageSize, space, mapQrys));
 +        Set<ClusterNode>[] partLocs = new Set[partsCnt];
  
 -            r.latch.await();
 +        // 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 (r.rmtErr != null)
 -                throw new CacheException("Failed to run map query remotely.", r.rmtErr);
 +            if (F.isEmpty(owners))
 +                throw new CacheException("No data nodes found for cache '" + cctx.name() + "' for partition " + p);
  
 -            if (qry.explain())
 -                return explainPlan(r.conn, space, qry);
 +            partLocs[p] = new HashSet<>(owners);
 +        }
  
 -            GridCacheSqlQuery rdc = qry.reduceQuery();
 +        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);
  
 -            final ResultSet res = h2.executeSqlQueryWithTimer(space, r.conn, rdc.query(), F.asList(rdc.parameters()));
 +                if (extraCctx.isReplicated() || extraCctx.isLocal())
 +                    continue;
  
 -            for (GridMergeTable tbl : r.tbls) {
 -                if (!tbl.getScanIndex(null).fetchedAll()) // We have to explicitly cancel queries on remote nodes.
 -                    send(nodes, new GridQueryCancelRequest(qryReqId));
 +                for (int p = 0, parts =  extraCctx.affinity().partitions(); p < parts; p++) {
 +                    List<ClusterNode> owners = extraCctx.topology().owners(p);
  
 -//                dropTable(r.conn, tbl.getName()); TODO
 +                    if (F.isEmpty(owners))
 +                        throw new CacheException("No data nodes found for cache '" + extraSpace +
 +                            "' for partition " + 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 = owningReplicatedDataNodes(extraCctx);
- 
-                 for (Set<ClusterNode> partLoc : partLocs) {
-                     partLoc.retainAll(dataNodes);
- 
-                     if (partLoc.isEmpty())
-                         return null; // Retry.
-                 }
-             }
 -            return new GridQueryCacheObjectsIterator(new Iter(res), cctx, keepPortable);
++            return new QueryCursorImpl<>(new GridQueryCacheObjectsIterator(new Iter(res), cctx, cctx.keepPortable()));
          }
 -        catch (IgniteCheckedException | InterruptedException | RuntimeException e) {
 -            U.closeQuiet(r.conn);
  
 -            if (e instanceof CacheException)
 -                throw (CacheException)e;
 +        // Collect the final partitions mapping.
 +        Map<ClusterNode, IntArray> res = new HashMap<>();
  
 -            throw new CacheException("Failed to run reduce query locally.", e);
 +        // 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);
          }
 -        finally {
 -            if (!runs.remove(qryReqId, r))
 -                U.warn(log, "Query run was already removed: " + qryReqId);
  
 -            curFunTbl.remove();
 +        return res;
 +    }
 +
 +    /**
 +     * @param mainSpace Main space.
 +     * @param allSpaces All spaces.
 +     * @return List of all extra spaces or {@code null} if none.
 +     */
 +    private List<String> extraSpaces(String mainSpace, Set<String> allSpaces) {
 +        if (F.isEmpty(allSpaces) || (allSpaces.size() == 1 && allSpaces.contains(mainSpace)))
 +            return null;
 +
 +        ArrayList<String> res = new ArrayList<>(allSpaces.size());
 +
 +        for (String space : allSpaces) {
 +            if (!F.eq(space, mainSpace))
 +                res.add(space);
          }
 +
 +        return res;
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf4b15/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf4b15/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------


[28/50] incubator-ignite git commit: Fixed test.

Posted by vk...@apache.org.
Fixed test.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/fa6615a8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/fa6615a8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/fa6615a8

Branch: refs/heads/ignite-1026
Commit: fa6615a8665fb8296c63be3e719d6d186293ea0c
Parents: 5218210
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu Jun 18 09:44:45 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Jun 18 09:44:45 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/testframework/junits/GridTestKernalContext.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fa6615a8/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 37948d9..24502da 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -46,7 +46,7 @@ public class GridTestKernalContext extends GridKernalContextImpl {
             null,
             null,
             null,
-            null);
+            U.allPluginProviders());
 
         GridTestUtils.setFieldValue(grid(), "cfg", config());
 


[27/50] incubator-ignite git commit: Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/eebf9c1c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/eebf9c1c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/eebf9c1c

Branch: refs/heads/ignite-1026
Commit: eebf9c1c7fa07b442e4a341ffa2b5ce2828032c4
Parents: ed43dfe fffc2ec
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Jun 18 09:43:45 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Jun 18 09:43:45 2015 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java        |  4 +++-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java  | 19 ++++++++++---------
 .../junits/common/GridCommonAbstractTest.java    |  8 ++++++--
 3 files changed, 19 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[14/50] incubator-ignite git commit: Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/ca253a8a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/ca253a8a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/ca253a8a

Branch: refs/heads/ignite-1026
Commit: ca253a8a845129ece37fca3a12c384d8b9ef3e76
Parents: 8343058 dc05e88
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 12:21:53 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 12:21:53 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |   2 +-
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |  22 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   | 290 +++++++++++++++++++
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   6 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   | 148 ++++++++++
 .../configuration/CacheConfiguration.java       |   3 +-
 .../internal/interop/InteropBootstrap.java      |   3 +-
 .../internal/interop/InteropIgnition.java       |   5 +-
 .../processors/cache/GridCacheProcessor.java    |  23 +-
 .../processors/query/GridQueryProcessor.java    | 102 +++++--
 .../util/spring/IgniteSpringHelper.java         |  10 +
 .../SpringApplicationContextResource.java       |   4 +-
 .../apache/ignite/resources/SpringResource.java |   6 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  87 ++++--
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../apache/ignite/internal/GridSelfTest.java    |  12 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |  17 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   2 +
 ...ridCacheMultinodeUpdateAbstractSelfTest.java |   9 +
 .../cache/GridCacheVersionMultinodeTest.java    |   8 +-
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  15 +-
 .../IgniteCacheClientNodeConcurrentStart.java   |  14 +-
 .../distributed/IgniteCacheManyClientsTest.java |  69 ++++-
 .../GridCachePartitionedTxSalvageSelfTest.java  |  37 +--
 .../junits/common/GridCommonAbstractTest.java   |  11 +-
 .../IgniteCacheFailoverTestSuite.java           |   8 -
 .../IgniteCacheFailoverTestSuite2.java          |  47 +++
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |  16 +-
 .../hibernate/CacheHibernateBlobStore.java      |  87 +-----
 .../CacheHibernateBlobStoreFactory.java         | 235 +++++++++++++++
 .../hibernate/src/test/config/factory-cache.xml |  59 ++++
 .../src/test/config/factory-cache1.xml          |  61 ++++
 .../config/factory-incorrect-store-cache.xml    |  56 ++++
 .../CacheHibernateStoreFactorySelfTest.java     | 273 +++++++++++++++++
 .../testsuites/IgniteHibernateTestSuite.java    |   2 +
 modules/indexing/pom.xml                        |   2 +-
 .../CacheAbstractQueryMetricsSelfTest.java      | 205 +++++++++++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ++
 .../cache/GridCacheQueryMetricsSelfTest.java    | 206 -------------
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   2 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   9 +-
 .../GridResourceSpringBeanInjector.java         |   2 +-
 .../util/spring/IgniteSpringHelperImpl.java     |  17 ++
 .../src/test/config/incorrect-store-cache.xml   |  57 ++++
 modules/spring/src/test/config/node.xml         |  43 +++
 modules/spring/src/test/config/node1.xml        |  45 +++
 .../test/config/pojo-incorrect-store-cache.xml  |  56 ++++
 modules/spring/src/test/config/store-cache.xml  |  59 ++++
 modules/spring/src/test/config/store-cache1.xml |  62 ++++
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java | 172 +++++++++++
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java | 193 ++++++++++++
 .../testsuites/IgniteSpringTestSuite.java       |   5 +
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 87 files changed, 2520 insertions(+), 500 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ca253a8a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 174b7a0,718e6ba..4f2486b
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@@ -687,9 -716,16 +714,9 @@@ public class GridQueryProcessor extend
      }
  
      /**
 -     * @return Message factory for {@link GridIoManager}.
 -     */
 -    public MessageFactory messageFactory() {
 -        return idx == null ? null : idx.messageFactory();
 -    }
 -
 -    /**
       * Closeable iterator.
       */
-     private static interface ClIter<X> extends AutoCloseable, Iterator<X> {
+     private interface ClIter<X> extends AutoCloseable, Iterator<X> {
          // No-op.
      }
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ca253a8a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------


[25/50] incubator-ignite git commit: ignite-484-1 - relax test: 2 restarting nodes instead of 3

Posted by vk...@apache.org.
ignite-484-1 - relax test: 2 restarting nodes instead of 3


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/ed43dfe1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/ed43dfe1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/ed43dfe1

Branch: refs/heads/ignite-1026
Commit: ed43dfe1c7e9b7db3f6ef33a38dad9b28c41190c
Parents: e602ca0
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 20:33:54 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 20:33:54 2015 +0300

----------------------------------------------------------------------
 .../near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ed43dfe1/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
index 3f23005..b868cfb 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
@@ -194,7 +194,7 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
     public void testRestarts() throws Exception {
         int duration = 90 * 1000;
         int qryThreadNum = 5;
-        int restartThreadsNum = 3; // 3 of 4 data nodes
+        int restartThreadsNum = 2; // 2 of 4 data nodes
         final int nodeLifeTime = 2 * 1000;
         final int logFreq = 10;
 


[46/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/8c404f78
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/8c404f78
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/8c404f78

Branch: refs/heads/ignite-1026
Commit: 8c404f78cc264ddf43c71904b9fbac40ce0ee418
Parents: 4f3292d 4d1fa72
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Thu Jun 18 15:19:58 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Thu Jun 18 15:19:58 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheUtils.java        |  9 +++++
 .../processors/cache/IgniteCacheProxy.java      |  5 +++
 .../processors/query/GridQueryProcessor.java    |  7 ++++
 .../communication/tcp/TcpCommunicationSpi.java  | 42 ++++++++++----------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 31 +++++++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 41 +++++++++++++------
 .../tcp/internal/TcpDiscoveryNode.java          | 18 +++++++++
 .../DataStreamerMultiThreadedSelfTest.java      |  3 ++
 8 files changed, 115 insertions(+), 41 deletions(-)
----------------------------------------------------------------------



[09/50] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/d8825fd7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d8825fd7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d8825fd7

Branch: refs/heads/ignite-1026
Commit: d8825fd71e55f285e62bb958357c3a5581607a4d
Parents: 4a53405 5160088
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 12 17:24:12 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 12 17:24:12 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               | 10 +++
 .../ignite/internal/GridKernalContextImpl.java  |  5 +-
 .../apache/ignite/internal/IgniteKernal.java    | 21 +++++-
 .../internal/MarshallerContextAdapter.java      | 18 ++++-
 .../ignite/internal/MarshallerContextImpl.java  | 14 +++-
 .../GridClientOptimizedMarshaller.java          |  5 ++
 .../distributed/GridCacheTxRecoveryRequest.java | 26 +++----
 .../GridCacheTxRecoveryResponse.java            | 14 ++--
 .../distributed/GridDistributedBaseMessage.java | 77 +-------------------
 .../distributed/GridDistributedLockRequest.java | 54 +++++++-------
 .../GridDistributedLockResponse.java            | 14 ++--
 .../GridDistributedTxFinishRequest.java         | 46 ++++++------
 .../GridDistributedTxPrepareRequest.java        | 62 ++++++++--------
 .../GridDistributedTxPrepareResponse.java       | 64 +---------------
 .../GridDistributedUnlockRequest.java           |  6 +-
 .../distributed/dht/GridDhtLockRequest.java     | 72 ++++++++++++++----
 .../distributed/dht/GridDhtLockResponse.java    | 18 ++---
 .../distributed/dht/GridDhtTxFinishRequest.java | 38 +++++-----
 .../dht/GridDhtTxPrepareRequest.java            | 54 +++++++-------
 .../dht/GridDhtTxPrepareResponse.java           | 22 +++---
 .../distributed/dht/GridDhtUnlockRequest.java   |  6 +-
 .../distributed/near/GridNearLockRequest.java   | 58 +++++++--------
 .../distributed/near/GridNearLockResponse.java  | 26 +++----
 .../near/GridNearTxFinishRequest.java           | 26 +++----
 .../near/GridNearTxPrepareRequest.java          | 50 ++++++-------
 .../near/GridNearTxPrepareResponse.java         | 46 ++++++------
 .../distributed/near/GridNearUnlockRequest.java |  2 +-
 .../cache/transactions/IgniteTxHandler.java     |  3 -
 .../plugin/IgnitePluginProcessor.java           | 16 +---
 .../messages/GridQueryNextPageResponse.java     |  1 +
 .../cache/GridCachePutAllFailoverSelfTest.java  |  5 --
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java | 14 +++-
 .../dht/GridCacheColocatedFailoverSelfTest.java |  5 --
 .../GridCachePartitionedFailoverSelfTest.java   |  5 --
 .../GridCacheReplicatedFailoverSelfTest.java    |  5 --
 .../DataStreamProcessorSelfTest.java            |  3 +-
 .../marshaller/MarshallerContextTestImpl.java   | 18 +++++
 .../junits/GridTestKernalContext.java           |  1 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |  2 +-
 .../ignite/tools/classgen/ClassesGenerator.java | 18 ++++-
 40 files changed, 462 insertions(+), 488 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d8825fd7/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
index b881f93,b210625..0f62ae9
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
@@@ -191,11 -187,6 +191,12 @@@ public class GridQueryNextPageResponse 
  
                  writer.incrementState();
  
 +            case 6:
 +                if (!writer.writeMessage("retry", retry))
 +                    return false;
 +
 +                writer.incrementState();
++
          }
  
          return true;


[47/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/ad0a026f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/ad0a026f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/ad0a026f

Branch: refs/heads/ignite-1026
Commit: ad0a026f480ff67025b39737c2e068b20035feb6
Parents: fc81bcf 8c404f7
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jun 18 15:54:59 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jun 18 15:54:59 2015 +0300

----------------------------------------------------------------------
 .../communication/tcp/TcpCommunicationSpi.java  | 38 ++++++++++----------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 21 +++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  9 ++++-
 .../tcp/internal/TcpDiscoveryNode.java          | 18 ++++++++++
 .../apache/ignite/internal/GridSelfTest.java    | 20 ++++++-----
 5 files changed, 72 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ad0a026f/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ad0a026f/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------


[10/50] incubator-ignite git commit: Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/d7a18c4c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d7a18c4c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d7a18c4c

Branch: refs/heads/ignite-1026
Commit: d7a18c4c63077a5195a3c20076fe8ec76e2566c1
Parents: d8825fd 40f826b
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sun Jun 14 19:25:33 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sun Jun 14 19:25:33 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  2 +
 .../datastructures/DataStructuresProcessor.java | 67 +++++++++++++++++++-
 .../rest/client/message/GridRouterRequest.java  | 18 ++++++
 .../rest/client/message/GridRouterResponse.java | 18 ++++++
 .../ignite/tools/classgen/ClassesGenerator.java | 12 ++++
 5 files changed, 114 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[20/50] incubator-ignite git commit: ignite-484-1 - replicated client cache test added

Posted by vk...@apache.org.
ignite-484-1 - replicated client cache test added


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/10febf28
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/10febf28
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/10febf28

Branch: refs/heads/ignite-1026
Commit: 10febf28fdf3966ffcb369c5725792b604be6c18
Parents: 1fe215e
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 17:11:00 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 17:11:00 2015 +0300

----------------------------------------------------------------------
 ...lientQueryReplicatedNodeRestartSelfTest.java | 381 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   1 +
 2 files changed, 382 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/10febf28/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
new file mode 100644
index 0000000..23f44c0
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
@@ -0,0 +1,381 @@
+/*
+ * 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.distributed.near;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cache.query.annotations.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Test for distributed queries with replicated client cache and node restarts.
+ */
+public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final String QRY = "select co.id, count(*) cnt\n" +
+        "from \"pe\".Person pe, \"pr\".Product pr, \"co\".Company co, \"pu\".Purchase pu\n" +
+        "where pe.id = pu.personId and pu.productId = pr.id and pr.companyId = co.id \n" +
+        "group by co.id order by cnt desc, co.id";
+
+    /** */
+    private static final P1<ClusterNode> DATA_NODES_FILTER = new P1<ClusterNode>() {
+            @Override public boolean apply(ClusterNode clusterNode) {
+                String gridName = clusterNode.attribute(IgniteNodeAttributes.ATTR_GRID_NAME);
+
+                return !gridName.endsWith(String.valueOf(GRID_CNT - 1)); // The last one is client only.
+            }
+        };
+
+    /** */
+    private static final int GRID_CNT = 5;
+
+    /** */
+    private static final int PERS_CNT = 600;
+
+    /** */
+    private static final int PURCHASE_CNT = 6000;
+
+    /** */
+    private static final int COMPANY_CNT = 25;
+
+    /** */
+    private static final int PRODUCT_CNT = 100;
+
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        X.println("grid name: " + gridName);
+
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        int i = 0;
+
+        CacheConfiguration<?, ?>[] ccs = new CacheConfiguration[4];
+
+        for (String name : F.asList("co", "pr", "pe", "pu")) {
+            CacheConfiguration<?, ?> cc = defaultCacheConfiguration();
+
+            cc.setNodeFilter(DATA_NODES_FILTER);
+            cc.setName(name);
+            cc.setCacheMode(REPLICATED);
+            cc.setWriteSynchronizationMode(FULL_SYNC);
+            cc.setAtomicityMode(TRANSACTIONAL);
+            cc.setRebalanceMode(SYNC);
+            cc.setAffinity(new RendezvousAffinityFunction(false, 50));
+
+            switch (name) {
+                case "co":
+                    cc.setIndexedTypes(
+                        Integer.class, Company.class
+                    );
+
+                    break;
+
+                case "pr":
+                    cc.setIndexedTypes(
+                        Integer.class, Product.class
+                    );
+
+                    break;
+
+                case "pe":
+                    cc.setIndexedTypes(
+                        Integer.class, Person.class
+                    );
+
+                    break;
+
+                case "pu":
+                    cc.setIndexedTypes(
+                        AffinityKey.class, Purchase.class
+                    );
+
+                    break;
+            }
+
+            ccs[i++] = cc;
+        }
+
+        c.setCacheConfiguration(ccs);
+
+        return c;
+    }
+
+    /**
+     *
+     */
+    private void fillCaches() {
+        IgniteCache<Integer, Company> co = grid(0).cache("co");
+
+        for (int i = 0; i < COMPANY_CNT; i++)
+            co.put(i, new Company(i));
+
+        IgniteCache<Integer, Product> pr = grid(0).cache("pr");
+
+        Random rnd = new GridRandom();
+
+        for (int i = 0; i < PRODUCT_CNT; i++)
+            pr.put(i, new Product(i, rnd.nextInt(COMPANY_CNT)));
+
+        IgniteCache<Integer, Person> pe = grid(0).cache("pe");
+
+        for (int i = 0; i < PERS_CNT; i++)
+            pe.put(i, new Person(i));
+
+        IgniteCache<AffinityKey<Integer>, Purchase> pu = grid(0).cache("pu");
+
+        for (int i = 0; i < PURCHASE_CNT; i++) {
+            int persId = rnd.nextInt(PERS_CNT);
+            int prodId = rnd.nextInt(PRODUCT_CNT);
+
+            pu.put(new AffinityKey<>(i, persId), new Purchase(persId, prodId));
+        }
+    }
+
+    /**
+     * @param c Cache.
+     * @param client If it must be a client cache.
+     */
+    private void assertClient(IgniteCache<?,?> c, boolean client) {
+        assertTrue(((IgniteCacheProxy)c).context().affinityNode() == !client);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestarts() throws Exception {
+        int duration = 90 * 1000;
+        int qryThreadNum = 5;
+        int restartThreadsNum = 2; // 2 of 4 data nodes
+        final int nodeLifeTime = 2 * 1000;
+        final int logFreq = 10;
+
+        startGridsMultiThreaded(GRID_CNT);
+
+        final AtomicIntegerArray locks = new AtomicIntegerArray(GRID_CNT - 1); // The last is client only.
+
+        fillCaches();
+
+        final List<List<?>> pRes = grid(0).cache("pu").query(new SqlFieldsQuery(QRY)).getAll();
+
+        Thread.sleep(3000);
+
+        assertEquals(pRes, grid(0).cache("pu").query(new SqlFieldsQuery(QRY)).getAll());
+
+        assertFalse(pRes.isEmpty());
+
+        final AtomicInteger qryCnt = new AtomicInteger();
+        final AtomicBoolean qrysDone = new AtomicBoolean();
+
+        for (int i = 0; i < GRID_CNT - 1; i++) {
+            for (String cacheName : F.asList("co", "pr", "pe", "pu"))
+                assertClient(grid(i).cache(cacheName), false);
+        }
+
+        for (String cacheName : F.asList("co", "pr", "pe", "pu"))
+            assertClient(grid(GRID_CNT - 1).cache(cacheName), true);
+
+        final IgniteCache<?,?> clientCache = grid(GRID_CNT - 1).cache("pu");
+
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(new CAX() {
+            @Override public void applyx() throws IgniteCheckedException {
+                GridRandom rnd = new GridRandom();
+
+                while (!qrysDone.get()) {
+                    SqlFieldsQuery qry = new SqlFieldsQuery(QRY);
+
+                    boolean smallPageSize = rnd.nextBoolean();
+
+                    if (smallPageSize)
+                        qry.setPageSize(3);
+
+                    try {
+                        assertEquals(pRes, clientCache.query(qry).getAll());
+                    }
+                    catch (CacheException e) {
+                        assertTrue("On large page size must retry.", smallPageSize);
+
+                        boolean failedOnRemoteFetch = false;
+
+                        for (Throwable th = e; th != null; th = th.getCause()) {
+                            if (!(th instanceof CacheException))
+                                continue;
+
+                            if (th.getMessage() != null &&
+                                th.getMessage().startsWith("Failed to fetch data from node:")) {
+                                failedOnRemoteFetch = true;
+
+                                break;
+                            }
+                        }
+
+                        if (!failedOnRemoteFetch) {
+                            e.printStackTrace();
+
+                            fail("Must fail inside of GridResultPage.fetchNextPage or subclass.");
+                        }
+                    }
+
+                    int c = qryCnt.incrementAndGet();
+
+                    if (c % logFreq == 0)
+                        info("Executed queries: " + c);
+                }
+            }
+        }, qryThreadNum);
+
+        final AtomicInteger restartCnt = new AtomicInteger();
+
+        final AtomicBoolean restartsDone = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(new Callable<Object>() {
+            @SuppressWarnings({"BusyWait"})
+            @Override public Object call() throws Exception {
+                GridRandom rnd = new GridRandom();
+
+                while (!restartsDone.get()) {
+                    int g;
+
+                    do {
+                        g = rnd.nextInt(locks.length());
+                    }
+                    while (!locks.compareAndSet(g, 0, -1));
+
+                    stopGrid(g);
+
+                    Thread.sleep(rnd.nextInt(nodeLifeTime));
+
+                    startGrid(g);
+
+                    Thread.sleep(rnd.nextInt(nodeLifeTime));
+
+                    locks.set(g, 0);
+
+                    int c = restartCnt.incrementAndGet();
+
+                    if (c % logFreq == 0)
+                        info("Node restarts: " + c);
+                }
+
+                return true;
+            }
+        }, restartThreadsNum);
+
+        Thread.sleep(duration);
+
+        info("Stopping..");
+
+        restartsDone.set(true);
+
+        fut2.get();
+
+        info("Restarts stopped.");
+
+        qrysDone.set(true);
+
+        fut1.get();
+
+        info("Queries stopped.");
+    }
+
+    /**
+     *
+     */
+    private static class Person implements Serializable {
+        @QuerySqlField(index = true)
+        int id;
+
+        Person(int id) {
+            this.id = id;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Purchase implements Serializable {
+        @QuerySqlField(index = true)
+        int personId;
+
+        @QuerySqlField(index = true)
+        int productId;
+
+        Purchase(int personId, int productId) {
+            this.personId = personId;
+            this.productId = productId;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Company implements Serializable {
+        @QuerySqlField(index = true)
+        int id;
+
+        Company(int id) {
+            this.id = id;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class Product implements Serializable {
+        @QuerySqlField(index = true)
+        int id;
+
+        @QuerySqlField(index = true)
+        int companyId;
+
+        Product(int id, int companyId) {
+            this.id = id;
+            this.companyId = companyId;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/10febf28/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index c5a2f15..dee3078 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -67,6 +67,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest2.class);
+        suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
         suite.addTestSuite(GridCacheReduceQueryMultithreadedSelfTest.class);
         suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class);
         suite.addTestSuite(GridCacheQuerySerializationSelfTest.class);


[12/50] incubator-ignite git commit: ignite-484-1 - small page test + race fix

Posted by vk...@apache.org.
ignite-484-1 - small page test + race fix


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/8e8433b6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/8e8433b6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/8e8433b6

Branch: refs/heads/ignite-1026
Commit: 8e8433b6cc07afcdd88a0a570887fe0353e59ee7
Parents: 7e3f924
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Jun 16 13:37:19 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Jun 16 13:37:19 2015 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/GridMergeIndex.java        | 17 +++++++---
 .../h2/twostep/GridMergeIndexUnsorted.java      |  5 +--
 .../h2/twostep/GridReduceQueryExecutor.java     |  4 +--
 .../query/h2/twostep/GridResultPage.java        | 19 ++++++-----
 .../IgniteCacheQueryNodeRestartSelfTest2.java   | 34 +++++++++++++++++++-
 5 files changed, 59 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8433b6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
index 9136821..af29647 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
@@ -96,7 +96,11 @@ public abstract class GridMergeIndex extends BaseIndex {
      * @param nodeId Node ID.
      */
     public void fail(UUID nodeId) {
-        addPage0(new GridResultPage(null, nodeId, null, false));
+        addPage0(new GridResultPage(null, nodeId, null) {
+            @Override public boolean isFail() {
+                return true;
+            }
+        });
     }
 
     /**
@@ -134,10 +138,13 @@ public abstract class GridMergeIndex extends BaseIndex {
                 }
             }
 
-            if (last)
-                last = lastSubmitted.compareAndSet(false, true);
-
-            addPage0(new GridResultPage(null, page.source(), null, last));
+            if (last && lastSubmitted.compareAndSet(false, true)) {
+                addPage0(new GridResultPage(null, page.source(), null) {
+                    @Override public boolean isLast() {
+                        return true;
+                    }
+                });
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8433b6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
index fdee17a..e0a07ec 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndexUnsorted.java
@@ -44,8 +44,9 @@ public class GridMergeIndexUnsorted extends GridMergeIndex {
 
     /** {@inheritDoc} */
     @Override protected void addPage0(GridResultPage page) {
-        if (page.rowsInPage() != 0 || page.isLast() || queue.isEmpty())
-            queue.add(page);
+        assert page.rowsInPage() > 0 || page.isLast() || page.isFail();
+
+        queue.add(page);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8433b6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 343a439..c570d24 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -229,7 +229,7 @@ public class GridReduceQueryExecutor {
         GridResultPage page;
 
         try {
-            page = new GridResultPage(ctx, node.id(), msg, false) {
+            page = new GridResultPage(ctx, node.id(), msg) {
                 @Override public void fetchNextPage() {
                     Object errState = r.state.get();
 
@@ -251,7 +251,7 @@ public class GridReduceQueryExecutor {
                             ctx.io().send(node, GridTopic.TOPIC_QUERY, msg0, GridIoPolicy.PUBLIC_POOL);
                     }
                     catch (IgniteCheckedException e) {
-                        throw new CacheException(e);
+                        throw new CacheException("Failed to fetch data from node: " + node.id(), e);
                     }
                 }
             };

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8433b6/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
index 35bfab9..c9a7e48 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridResultPage.java
@@ -43,27 +43,19 @@ public class GridResultPage {
     private final int rowsInPage;
 
     /** */
-    private final boolean last;
-
-    /** */
     private Iterator<Value[]> rows;
 
     /**
      * @param ctx Kernal context.
      * @param src Source.
      * @param res Response.
-     * @param last If this is the globally last page.
      */
     @SuppressWarnings("unchecked")
-    public GridResultPage(final GridKernalContext ctx, UUID src, GridQueryNextPageResponse res, boolean last) {
+    public GridResultPage(final GridKernalContext ctx, UUID src, GridQueryNextPageResponse res) {
         assert src != null;
 
         this.src = src;
         this.res = res;
-        this.last = last;
-
-        if (last)
-            assert res == null : "The last page must be dummy.";
 
         // res == null means that it is a terminating dummy page for the given source node ID.
         if (res != null) {
@@ -117,10 +109,17 @@ public class GridResultPage {
     }
 
     /**
+     * @return {@code true} If this is a dummy fail page.
+     */
+    public boolean isFail() {
+        return false;
+    }
+
+    /**
      * @return {@code true} If this is a dummy last page for all the sources.
      */
     public boolean isLast() {
-        return last;
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8433b6/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
index e65cc13..d440b13 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
@@ -32,6 +32,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 
+import javax.cache.*;
 import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
@@ -215,7 +216,38 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
                     if (rnd.nextBoolean()) { // Partitioned query.
                         IgniteCache<?,?> cache = grid(g).cache("pu");
 
-                        assertEquals(pRes, cache.query(new SqlFieldsQuery(PARTITIONED_QRY)).getAll());
+                        SqlFieldsQuery qry = new SqlFieldsQuery(PARTITIONED_QRY);
+
+                        boolean smallPageSize = rnd.nextBoolean();
+
+                        if (smallPageSize)
+                            qry.setPageSize(3);
+
+                        try {
+                            assertEquals(pRes, cache.query(qry).getAll());
+                        }
+                        catch (CacheException e) {
+                            assertTrue("On large page size must retry.", smallPageSize);
+
+                            boolean failedOnRemoteFetch = false;
+
+                            for (Throwable th = e; th != null; th = th.getCause()) {
+                                if (!(th instanceof CacheException))
+                                    continue;
+
+                                if (th.getMessage().startsWith("Failed to fetch data from node:")) {
+                                    failedOnRemoteFetch = true;
+
+                                    break;
+                                }
+                            }
+
+                            if (!failedOnRemoteFetch) {
+                                e.printStackTrace();
+
+                                fail("Must fail inside of GridResultPage.fetchNextPage or subclass.");
+                            }
+                        }
                     }
                     else { // Replicated query.
                         IgniteCache<?,?> cache = grid(g).cache("co");


[08/50] incubator-ignite git commit: ignite-484-1 - refactor

Posted by vk...@apache.org.
ignite-484-1 - refactor


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/4a534059
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4a534059
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4a534059

Branch: refs/heads/ignite-1026
Commit: 4a534059e4bf46b061e1272d432fd00c6f87acb0
Parents: 2b7dc3b
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 12 17:21:14 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 12 17:21:14 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java  |  10 +-
 .../dht/GridDhtPartitionsReservation.java       | 169 ++++++++++++++-----
 .../query/h2/twostep/GridMapQueryExecutor.java  |  24 +--
 3 files changed, 141 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a534059/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 3670b8e..018ffd6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -143,19 +143,15 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
         assert state.getReference() != EVICTED : "we can reserve only active partitions";
         assert state.getStamp() != 0 : "partition must be already reserved before adding group reservation";
 
-        if (!reservations.addIfAbsent(r))
-            return false;
-
-        r.register(this);
-
-        return true;
+        return reservations.addIfAbsent(r);
     }
 
     /**
      * @param r Reservation.
      */
     public void removeReservation(GridDhtPartitionsReservation r) {
-        reservations.remove(r);
+        if (!reservations.remove(r))
+            throw new IllegalStateException("Reservation was already removed.");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a534059/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
index fcd6088..71a1859 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
@@ -19,70 +19,126 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.util.typedef.*;
 
 import java.util.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.*;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.RENTING;
 
 /**
  * Reservation mechanism for multiple partitions allowing to do a reservation in one operation.
  */
 public class GridDhtPartitionsReservation implements GridReservable {
     /** */
+    private static final GridDhtLocalPartition[] EMPTY = {};
+
+    /** */
+    private final Object appKey;
+
+    /** */
     private final GridCacheContext<?,?> cctx;
 
     /** */
     private final AffinityTopologyVersion topVer;
 
     /** */
-    private final List<GridDhtLocalPartition> parts = new ArrayList<>();
+    private final AtomicReference<GridDhtLocalPartition[]> parts = new AtomicReference<>();
 
     /** */
     private final AtomicInteger reservations = new AtomicInteger();
 
     /** */
-    private final IgniteInClosure<GridDhtPartitionsReservation> finalize;
+    private volatile CI1<GridDhtPartitionsReservation> unpublish;
 
     /**
      * @param topVer AffinityTopologyVersion version.
      * @param cctx Cache context.
-     * @param finalize Finalizing closure.
+     * @param appKey Application key for reservation.
      */
-    public GridDhtPartitionsReservation(
-        AffinityTopologyVersion topVer,
-        GridCacheContext<?,?> cctx,
-        IgniteInClosure<GridDhtPartitionsReservation> finalize) {
+    public GridDhtPartitionsReservation(AffinityTopologyVersion topVer, GridCacheContext<?,?> cctx, Object appKey) {
         assert topVer != null;
         assert cctx != null;
+        assert appKey != null;
 
         this.topVer = topVer;
         this.cctx = cctx;
-        this.finalize = finalize;
+        this.appKey = appKey;
     }
 
     /**
-     * @return Topology version.
+     * Registers all the given partitions for this reservation.
+     *
+     * @param parts Partitions.
+     * @return {@code true} If registration succeeded and this reservation can be published.
      */
-    public AffinityTopologyVersion topologyVersion() {
-        return topVer;
-    }
+    public boolean register(Collection<? extends GridReservable> parts) {
+        assert !F.isEmpty(parts) : "empty partitions list";
 
-    /**
-     * @return Cache context.
-     */
-    public GridCacheContext<?,?> cacheContext() {
-        return cctx;
+        GridDhtLocalPartition[] arr = new GridDhtLocalPartition[parts.size()];
+
+        int i = 0;
+        int prevPart = -1;
+        boolean sorted = true; // Most probably it is a sorted list.
+
+        for (GridReservable part : parts) {
+            arr[i] = (GridDhtLocalPartition)part;
+
+            if (sorted) { // Make sure it will be a sorted array.
+                int id = arr[i].id();
+
+                if (id <= prevPart)
+                    sorted = false;
+
+                prevPart = id;
+            }
+
+            i++;
+        }
+
+        if (!sorted)
+            Arrays.sort(arr);
+
+        i = 0;
+        prevPart = -1;
+
+        // Register in correct sort order.
+        for (GridDhtLocalPartition part : arr) {
+            if (prevPart == part.id())
+                throw new IllegalStateException("Duplicated partitions.");
+
+            prevPart = part.id();
+
+            if (!part.addReservation(this)) {
+                if (i != 0)
+                    throw new IllegalStateException(
+                        "Trying to reserve different sets of partitions for the same topology version.");
+
+                return false;
+            }
+
+            i++;
+        }
+
+        if (!this.parts.compareAndSet(null, arr))
+            throw new IllegalStateException("Partitions can be registered only once.");
+
+        return true;
     }
 
     /**
-     * Registers partition for this group reservation.
+     * Must be called when this reservation is published.
      *
-     * @param part Partition.
+     * @param unpublish Closure to unpublish this reservation when it will become invalid.
      */
-    public void register(GridDhtLocalPartition part) {
-        parts.add(part);
+    public void onPublish(CI1<GridDhtPartitionsReservation> unpublish) {
+        assert unpublish != null;
+        assert this.unpublish == null;
+
+        this.unpublish = unpublish;
+
+        if (reservations.get() == -1)
+            unregister();
     }
 
     /**
@@ -91,6 +147,8 @@ public class GridDhtPartitionsReservation implements GridReservable {
      * @return {@code true} If succeeded.
      */
     @Override public boolean reserve() {
+        assert parts.get() != null : "partitions must be registered before the first reserve attempt";
+
         for (;;) {
             int r = reservations.get();
 
@@ -105,6 +163,25 @@ public class GridDhtPartitionsReservation implements GridReservable {
     }
 
     /**
+     * @param parts Partitions.
+     */
+    private static void tryEvict(GridDhtLocalPartition[] parts) {
+        if (parts == null)  // Can be not initialized yet.
+            return ;
+
+        for (GridDhtLocalPartition part : parts)
+            tryEvict(part);
+    }
+
+    /**
+     * @param part Partition.
+     */
+    private static void tryEvict(GridDhtLocalPartition part) {
+        if (part.state() == RENTING && part.reservations() == 0)
+            part.tryEvictAsync(true);
+    }
+
+    /**
      * Releases all the registered partitions.
      */
     @Override public void release() {
@@ -116,12 +193,8 @@ public class GridDhtPartitionsReservation implements GridReservable {
 
             if (reservations.compareAndSet(r, r - 1)) {
                 // If it was the last reservation and topology version changed -> attempt to evict partitions.
-                if (r == 1 && !topVer.equals(cctx.topology().topologyVersion())) {
-                    for (GridDhtLocalPartition part : parts) {
-                        if (part.state() == RENTING)
-                            part.tryEvictAsync(true);
-                    }
-                }
+                if (r == 1 && !topVer.equals(cctx.topology().topologyVersion()))
+                    tryEvict(parts.get());
 
                 return;
             }
@@ -129,6 +202,26 @@ public class GridDhtPartitionsReservation implements GridReservable {
     }
 
     /**
+     * Unregisters this reservation from all the partitions.
+     */
+    private void unregister() {
+        GridDhtLocalPartition[] arr = parts.get();
+
+        if (!F.isEmpty(arr) && unpublish != null && parts.compareAndSet(arr, EMPTY)) {
+            // Reverse order makes sure that addReservation on the same topVer reservation will fail on the first partition.
+            for (int i = arr.length - 1; i >= 0; i--) {
+                GridDhtLocalPartition part = arr[i];
+
+                part.removeReservation(this);
+
+                tryEvict(part);
+            }
+
+            unpublish.apply(this);
+        }
+    }
+
+    /**
      * Must be checked in {@link GridDhtLocalPartition#tryEvict(boolean)}.
      * If returns {@code true} then probably partition will be evicted (or at least cleared),
      * so this reservation object becomes invalid and must be dropped from the partition.
@@ -146,12 +239,7 @@ public class GridDhtPartitionsReservation implements GridReservable {
             return r == -1;
 
         if (reservations.compareAndSet(0, -1)) {
-            // Remove our self.
-            for (GridDhtLocalPartition part : parts)
-                part.removeReservation(this);
-
-            if (finalize != null)
-                finalize.apply(this);
+            unregister();
 
             return true;
         }
@@ -169,13 +257,18 @@ public class GridDhtPartitionsReservation implements GridReservable {
 
         GridDhtPartitionsReservation that = (GridDhtPartitionsReservation)o;
 
-        return topVer.equals(that.topVer) && cctx == that.cctx;
+        return cctx == that.cctx && topVer.equals(that.topVer) && appKey.equals(that.appKey);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        String cache = cctx.name();
+        String name = cctx.name();
+
+        int result = name == null ? 0 : name.hashCode();
+
+        result = 31 * result + appKey.hashCode();
+        result = 31 * result + topVer.hashCode();
 
-        return 31 * topVer.hashCode() + cache == null ? 0 : cache.hashCode();
+        return result;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a534059/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index a8bc6e0..42f01cb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -290,27 +290,17 @@ public class GridMapQueryExecutor {
 
                     if (explicitParts == null) {
                         // We reserved all the primary partitions for cache, attempt to add group reservation.
-                        GridDhtPartitionsReservation reservation = new GridDhtPartitionsReservation(topVer, cctx,
-                            new CI1<GridDhtPartitionsReservation>() {
+                        GridDhtPartitionsReservation grp = new GridDhtPartitionsReservation(topVer, cctx, "SQL");
+
+                        if (grp.register(reserved.subList(reserved.size() - partIds.size(), reserved.size()))) {
+                            if (reservations.putIfAbsent(grpKey, grp) != null)
+                                throw new IllegalStateException("Reservation already exists.");
+
+                            grp.onPublish(new CI1<GridDhtPartitionsReservation>() {
                                 @Override public void apply(GridDhtPartitionsReservation r) {
                                     reservations.remove(grpKey, r);
                                 }
                             });
-
-                        for (int p = reserved.size() - partIds.size(); p < reserved.size(); p++) {
-                            if (!((GridDhtLocalPartition)reserved.get(p)).addReservation(reservation)) {
-                                // Can fail to add only on the first partition because of the same order of partitions.
-                                assert p == reserved.size() - partIds.size() : p;
-
-                                reservation = null;
-
-                                break;
-                            }
-                        }
-
-                        if (reservation != null) { // If we were able to add reservation to all partitions, publish it.
-                            if (reservations.putIfAbsent(grpKey, reservation) != null)
-                                throw new IllegalStateException();
                         }
                     }
                 }


[38/50] incubator-ignite git commit: # ignite-1003 add lastSuccessfulAddr for ping

Posted by vk...@apache.org.
# ignite-1003 add lastSuccessfulAddr for ping


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/b23f9300
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/b23f9300
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/b23f9300

Branch: refs/heads/ignite-1026
Commit: b23f9300794c7e84dfc83c1b8e49de673fa354e1
Parents: d874b00
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 13:52:45 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 13:52:45 2015 +0300

----------------------------------------------------------------------
 .../communication/tcp/TcpCommunicationSpi.java  | 38 ++++++++++----------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 21 +++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  9 ++++-
 .../tcp/internal/TcpDiscoveryNode.java          | 18 ++++++++++
 4 files changed, 60 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b23f9300/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 9e38788..39f4eeb 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -267,23 +267,19 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         if (!isNodeStopping()) {
                             GridNioRecoveryDescriptor recoveryData = ses.recoveryDescriptor();
 
-                            if (!getSpiContext().tryFailNode(id)) {
-                                if (recoveryData != null) {
-                                    if (recoveryData.nodeAlive(getSpiContext().node(id))) {
-                                        if (!recoveryData.messagesFutures().isEmpty()) {
-                                            if (log.isDebugEnabled())
-                                                log.debug("Session was closed but there are unacknowledged messages, " +
-                                                    "will try to reconnect [rmtNode=" + recoveryData.node().id() + ']');
-
-                                            commWorker.addReconnectRequest(recoveryData);
-                                        }
+                            if (recoveryData != null) {
+                                if (recoveryData.nodeAlive(getSpiContext().node(id))) {
+                                    if (!recoveryData.messagesFutures().isEmpty()) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Session was closed but there are unacknowledged messages, " +
+                                                "will try to reconnect [rmtNode=" + recoveryData.node().id() + ']');
+
+                                        commWorker.addReconnectRequest(recoveryData);
                                     }
-                                    else
-                                        recoveryData.onNodeLeft();
                                 }
+                                else
+                                    recoveryData.onNodeLeft();
                             }
-                            else
-                                recoveryData.onNodeLeft();
                         }
                     }
 
@@ -1884,18 +1880,20 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 "TCP communication addresses or mapped external addresses. Check configuration and make sure " +
                 "that you use the same communication SPI on all nodes. Remote node id: " + node.id());
 
-        List<InetSocketAddress> addrs;
+        LinkedHashSet<InetSocketAddress> addrs;
 
         // Try to connect first on bound addresses.
         if (isRmtAddrsExist) {
-            addrs = new ArrayList<>(U.toSocketAddresses(rmtAddrs0, rmtHostNames0, boundPort));
+            List<InetSocketAddress> addrs0 = new ArrayList<>(U.toSocketAddresses(rmtAddrs0, rmtHostNames0, boundPort));
 
             boolean sameHost = U.sameMacs(getSpiContext().localNode(), node);
 
-            Collections.sort(addrs, U.inetAddressesComparator(sameHost));
+            Collections.sort(addrs0, U.inetAddressesComparator(sameHost));
+
+            addrs = new LinkedHashSet<>(addrs0);
         }
         else
-            addrs = new ArrayList<>();
+            addrs = new LinkedHashSet<>();
 
         // Then on mapped external addresses.
         if (isExtAddrsExist)
@@ -2610,7 +2608,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         private void processRecovery(GridNioRecoveryDescriptor recoveryDesc) {
             ClusterNode node = recoveryDesc.node();
 
-            if (clients.containsKey(node.id()) || !recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
+            if (clients.containsKey(node.id()) ||
+                !recoveryDesc.nodeAlive(getSpiContext().node(node.id())) ||
+                !getSpiContext().pingNode(node.id()))
                 return;
 
             try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b23f9300/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 63f165d..8a9553e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -440,7 +440,12 @@ class ServerImpl extends TcpDiscoveryImpl {
                 // ID returned by the node should be the same as ID of the parameter for ping to succeed.
                 IgniteBiTuple<UUID, Boolean> t = pingNode(addr, clientNodeId);
 
-                return node.id().equals(t.get1()) && (clientNodeId == null || t.get2());
+                boolean res = node.id().equals(t.get1()) && (clientNodeId == null || t.get2());
+
+                if (res)
+                    node.lastSuccessfulAddress(addr);
+
+                return res;
             }
             catch (IgniteCheckedException e) {
                 if (log.isDebugEnabled())
@@ -458,8 +463,9 @@ class ServerImpl extends TcpDiscoveryImpl {
      * Pings the node by its address to see if it's alive.
      *
      * @param addr Address of the node.
+     * @param clientNodeId Client node ID.
      * @return ID of the remote node and "client exists" flag if node alive.
-     * @throws IgniteSpiException If an error occurs.
+     * @throws IgniteCheckedException If an error occurs.
      */
     private IgniteBiTuple<UUID, Boolean> pingNode(InetSocketAddress addr, @Nullable UUID clientNodeId)
         throws IgniteCheckedException {
@@ -1589,8 +1595,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     regAddrs,
                     F.notContains(currAddrs),
                     new P1<InetSocketAddress>() {
-                        private final Map<InetSocketAddress, Boolean> pingResMap =
-                            new HashMap<>();
+                        private final Map<InetSocketAddress, Boolean> pingResMap = new HashMap<>();
 
                         @Override public boolean apply(InetSocketAddress addr) {
                             Boolean res = pingResMap.get(addr);
@@ -2092,6 +2097,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     errs = null;
 
                                     success = true;
+
+                                    next.lastSuccessfulAddress(addr);
                                 }
                             }
                             catch (IOException | IgniteCheckedException e) {
@@ -2672,6 +2679,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                 try {
                     sendMessageDirectly(msg, addr);
 
+                    node.lastSuccessfulAddress(addr);
+
                     ex = null;
 
                     break;
@@ -4588,7 +4597,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         }
 
         /**
-         *
+         * @param res Ping result.
          */
         public void pingResult(boolean res) {
             GridFutureAdapter<Boolean> fut = pingFut.getAndSet(null);
@@ -4598,7 +4607,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         }
 
         /**
-         *
+         * @throws InterruptedException If interrupted.
          */
         public boolean ping() throws InterruptedException {
             if (spi.isNodeStopping0())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b23f9300/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index e4ef744..baada21 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -892,7 +892,14 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
 
         Collections.sort(addrs, U.inetAddressesComparator(sameHost));
 
-        LinkedHashSet<InetSocketAddress> res = new LinkedHashSet<>(addrs);
+        LinkedHashSet<InetSocketAddress> res = new LinkedHashSet<>();
+
+        InetSocketAddress lastAddr = node.lastSuccessfulAddress();
+
+        if (lastAddr != null)
+            res.add(lastAddr);
+
+        res.addAll(addrs);
 
         Collection<InetSocketAddress> extAddrs = node.attribute(createSpiAttributeName(ATTR_EXT_ADDRS));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b23f9300/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index cc61c9d..36ae39e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@ -111,6 +111,10 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
     @GridToStringExclude
     private UUID clientRouterNodeId;
 
+    /** */
+    @GridToStringExclude
+    private volatile transient InetSocketAddress lastSuccessfulAddr;
+
     /**
      * Public default no-arg constructor for {@link Externalizable} interface.
      */
@@ -152,6 +156,20 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
         sockAddrs = U.toSocketAddresses(this, discPort);
     }
 
+    /**
+     * @return Last successfully connected address.
+     */
+    @Nullable public InetSocketAddress lastSuccessfulAddress() {
+        return lastSuccessfulAddr;
+    }
+
+    /**
+     * @param lastSuccessfulAddr Last successfully connected address.
+     */
+    public void lastSuccessfulAddress(InetSocketAddress lastSuccessfulAddr) {
+        this.lastSuccessfulAddr = lastSuccessfulAddr;
+    }
+
     /** {@inheritDoc} */
     @Override public UUID id() {
         return id;


[24/50] incubator-ignite git commit: ignite-484-1 - error messages

Posted by vk...@apache.org.
ignite-484-1 - error messages


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/e602ca01
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/e602ca01
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/e602ca01

Branch: refs/heads/ignite-1026
Commit: e602ca01e48fd43f4cc33b729078c105d84601f6
Parents: efb4244
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 20:15:45 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 20:15:45 2015 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/GridMapQueryExecutor.java  |  4 +--
 .../h2/twostep/GridReduceQueryExecutor.java     | 31 ++++++++++----------
 2 files changed, 17 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e602ca01/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 2503a87..ba95d43 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -401,7 +401,7 @@ public class GridMapQueryExecutor {
                 }
             }
             catch (IgniteCheckedException e) {
-                throw new CacheException(e);
+                throw new CacheException("Failed to unmarshall parameters.", e);
             }
 
             List<String> caches = (List<String>)F.concat(true, req.space(), req.extraSpaces());
@@ -422,7 +422,7 @@ public class GridMapQueryExecutor {
             GridCacheContext<?,?> mainCctx = cacheContext(req.space());
 
             if (mainCctx == null)
-                throw new CacheException("Cache was destroyed: " + req.space());
+                throw new CacheException("Failed to find cache: " + req.space());
 
             qr = new QueryResults(req.requestId(), qrys.size(), mainCctx);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e602ca01/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index b956167..32d1c95 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -338,7 +338,7 @@ public class GridReduceQueryExecutor {
         Set<ClusterNode> nodes = new HashSet<>(dataNodes(space, topVer));
 
         if (F.isEmpty(nodes))
-            throw new CacheException("No data nodes found for cache: " + space);
+            throw new CacheException("Failed to find data nodes for cache: " + space);
 
         if (!F.isEmpty(extraSpaces)) {
             for (String extraSpace : extraSpaces) {
@@ -349,12 +349,12 @@ public class GridReduceQueryExecutor {
 
                 if (cctx.isReplicated() && !extraCctx.isReplicated())
                     throw new CacheException("Queries running on replicated cache should not contain JOINs " +
-                        "with partitioned tables.");
+                        "with partitioned tables [rCache=" + cctx.name() + ", pCache=" + extraSpace + "]");
 
                 Collection<ClusterNode> extraNodes = dataNodes(extraSpace, topVer);
 
                 if (F.isEmpty(extraNodes))
-                    throw new CacheException("No data nodes found for cache: " + extraSpace);
+                    throw new CacheException("Failed to find data nodes for cache: " + extraSpace);
 
                 if (cctx.isReplicated() && extraCctx.isReplicated()) {
                     nodes.retainAll(extraNodes);
@@ -363,8 +363,8 @@ public class GridReduceQueryExecutor {
                         if (isPreloadingActive(cctx, extraSpaces))
                             return null; // Retry.
                         else
-                            throw new CacheException("Caches '" + cctx.name() + "' and '" + extraSpace +
-                                "' have distinct set of data nodes.");
+                            throw new CacheException("Caches have distinct sets of data nodes [cache1=" + cctx.name() +
+                                ", cache2=" + extraSpace + "]");
                     }
                 }
                 else if (!cctx.isReplicated() && extraCctx.isReplicated()) {
@@ -372,16 +372,16 @@ public class GridReduceQueryExecutor {
                         if (isPreloadingActive(cctx, extraSpaces))
                             return null; // Retry.
                         else
-                            throw new CacheException("Caches '" + cctx.name() + "' and '" + extraSpace +
-                                "' have distinct set of data nodes.");
+                            throw new CacheException("Caches have distinct sets of data nodes [cache1=" + cctx.name() +
+                                ", cache2=" + extraSpace + "]");
                 }
                 else if (!cctx.isReplicated() && !extraCctx.isReplicated()) {
                     if (extraNodes.size() != nodes.size() || !nodes.containsAll(extraNodes))
                         if (isPreloadingActive(cctx, extraSpaces))
                             return null; // Retry.
                         else
-                            throw new CacheException("Caches '" + cctx.name() + "' and '" + extraSpace +
-                                "' have distinct set of data nodes.");
+                            throw new CacheException("Caches have distinct sets of data nodes [cache1=" + cctx.name() +
+                                ", cache2=" + extraSpace + "]");
                 }
                 else
                     throw new IllegalStateException();
@@ -586,7 +586,7 @@ public class GridReduceQueryExecutor {
 
                 if (!extraCctx.isReplicated())
                     throw new CacheException("Queries running on replicated cache should not contain JOINs " +
-                        "with partitioned tables.");
+                        "with tables in partitioned caches [rCache=" + cctx.name() + ", pCache=" + extraSpace + "]");
 
                 Set<ClusterNode> extraOwners = replicatedUnstableDataNodes(extraCctx);
 
@@ -628,7 +628,7 @@ public class GridReduceQueryExecutor {
         Set<ClusterNode> dataNodes = new HashSet<>(dataNodes(space, NONE));
 
         if (dataNodes.isEmpty())
-            throw new CacheException("No data nodes found for cache '" + space + "'");
+            throw new CacheException("Failed to find data nodes for cache: " + space);
 
         // Find all the nodes owning all the partitions for replicated cache.
         for (int p = 0, parts = cctx.affinity().partitions(); p < parts; p++) {
@@ -670,8 +670,8 @@ public class GridReduceQueryExecutor {
                 int parts = extraCctx.affinity().partitions();
 
                 if (parts != partsCnt)
-                    throw new CacheException("Number of partitions must be the same for correct collocation in " +
-                        "caches " + cctx.name() + " and " + extraSpace + ".");
+                    throw new CacheException("Number of partitions must be the same for correct collocation [cache1=" +
+                        cctx.name() + ", parts1=" + partsCnt + ", cache2=" + extraSpace + ", parts2=" + parts + "]");
             }
         }
 
@@ -685,7 +685,7 @@ public class GridReduceQueryExecutor {
                 if (!F.isEmpty(dataNodes(cctx.name(), NONE)))
                     return null; // Retry.
 
-                throw new CacheException("No data nodes found for cache '" + cctx.name() + "' for partition " + p);
+                throw new CacheException("Failed to find data nodes [cache=" + cctx.name() + ", part=" + p + "]");
             }
 
             partLocs[p] = new HashSet<>(owners);
@@ -707,8 +707,7 @@ public class GridReduceQueryExecutor {
                         if (!F.isEmpty(dataNodes(extraSpace, NONE)))
                             return null; // Retry.
 
-                        throw new CacheException("No data nodes found for cache '" + extraSpace +
-                            "' for partition " + p);
+                        throw new CacheException("Failed to find data nodes [cache=" + extraSpace + ", part=" + p + "]");
                     }
 
                     if (partLocs[p] == null)


[06/50] incubator-ignite git commit: Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
#	modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
#	modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
#	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
#	modules/scalar-2.10/pom.xml
#	modules/spark-2.10/pom.xml
#	modules/spark/pom.xml
#	modules/visor-console-2.10/pom.xml


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/0a5e97c2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/0a5e97c2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/0a5e97c2

Branch: refs/heads/ignite-1026
Commit: 0a5e97c248b2c3aa401ce9062b9ebd9787a4db24
Parents: a7229dc b087aca
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 12 10:36:11 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 12 10:36:11 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |  15 +
 examples/config/example-cache.xml               |   2 +
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 .../client/router/TcpSslRouterSelfTest.java     |   5 +
 .../client/suite/IgniteClientTestSuite.java     |   3 +-
 modules/cloud/pom.xml                           |   2 +-
 .../cloud/TcpDiscoveryCloudIpFinder.java        |  25 +-
 .../TcpDiscoveryCloudIpFinderSelfTest.java      |   3 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../apache/ignite/cache/query/ScanQuery.java    |  20 +
 .../ignite/internal/GridPluginContext.java      |   6 +
 .../apache/ignite/internal/IgniteKernal.java    |   2 +-
 .../internal/MarshallerContextAdapter.java      |  30 +-
 .../ignite/internal/MarshallerContextImpl.java  |  10 +-
 .../internal/interop/InteropIgnition.java       |  54 +-
 .../internal/interop/InteropProcessor.java      |   8 +
 .../processors/cache/GridCacheMessage.java      |  51 --
 .../processors/cache/GridCacheProcessor.java    |  27 +-
 .../processors/cache/KeyCacheObjectImpl.java    |  11 +-
 .../cache/query/GridCacheQueryManager.java      |   5 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   9 +-
 .../IgniteCacheObjectProcessorImpl.java         |  12 +-
 .../continuous/GridContinuousProcessor.java     |  15 +-
 .../datastreamer/DataStreamerCacheUpdaters.java |   2 +-
 .../datastreamer/DataStreamerImpl.java          |   8 +-
 .../portable/GridPortableInputStream.java       |  10 +
 .../processors/query/GridQueryProcessor.java    |   2 +-
 .../ignite/internal/util/IgniteUtils.java       |   3 +
 .../util/ipc/shmem/IpcSharedMemoryUtils.java    |   4 +-
 .../internal/visor/VisorMultiNodeTask.java      |   2 +-
 .../ignite/marshaller/MarshallerContext.java    |   8 +
 .../org/apache/ignite/plugin/PluginContext.java |   6 +
 .../java/org/jsr166/ConcurrentHashMap8.java     |   8 +-
 .../java/org/jsr166/ConcurrentLinkedDeque8.java | 586 ++++++-------------
 .../src/main/java/org/jsr166/LongAdder8.java    |  35 +-
 .../core/src/main/java/org/jsr166/README.txt    |  11 +
 .../src/main/java/org/jsr166/Striped64_8.java   |  22 +-
 .../java/org/jsr166/ThreadLocalRandom8.java     |  19 +-
 .../src/main/java/org/jsr166/package-info.java  |  12 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 modules/core/src/test/config/tests.properties   |   2 +-
 .../ignite/GridSuppressedExceptionSelfTest.java |   4 +-
 .../internal/GridDiscoveryEventSelfTest.java    |   6 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |   2 +
 .../GridFailoverTaskWithPredicateSelfTest.java  |   3 -
 .../GridJobMasterLeaveAwareSelfTest.java        |   2 -
 .../internal/GridJobStealingSelfTest.java       |   3 -
 ...ectionLocalJobMultipleArgumentsSelfTest.java |   2 -
 .../GridTaskExecutionContextSelfTest.java       |   9 -
 .../IgniteComputeEmptyClusterGroupTest.java     |   3 -
 .../IgniteComputeTopologyExceptionTest.java     |   9 -
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   5 +
 .../cache/GridCacheAbstractSelfTest.java        |   3 -
 .../cache/GridCacheAffinityRoutingSelfTest.java |   4 +-
 .../cache/GridCacheDeploymentSelfTest.java      |   3 -
 .../cache/GridCacheEntryMemorySizeSelfTest.java |  91 +--
 .../cache/GridCacheMemoryModeSelfTest.java      |   2 -
 ...inodeUpdateNearEnabledNoBackupsSelfTest.java |   2 +-
 ...CacheMultinodeUpdateNearEnabledSelfTest.java |   2 +-
 .../processors/cache/GridCacheOffHeapTest.java  |  28 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   5 +
 .../GridCacheReferenceCleanupSelfTest.java      |   3 -
 .../processors/cache/GridCacheStopSelfTest.java |   5 +
 .../cache/GridCacheVersionMultinodeTest.java    |   2 +-
 .../cache/IgniteCacheAbstractTest.java          |   3 -
 .../IgniteCacheEntryListenerAbstractTest.java   |  14 +-
 .../IgniteCacheInterceptorSelfTestSuite.java    |   2 +-
 .../cache/IgniteCacheInvokeReadThroughTest.java |   5 +
 ...gniteCacheTransactionalStopBusySelfTest.java |   5 +
 ...teStartCacheInTransactionAtomicSelfTest.java |  32 +
 .../IgniteStartCacheInTransactionSelfTest.java  | 254 ++++++++
 .../IgniteTxMultiThreadedAbstractTest.java      |   4 +-
 ...cheAtomicReferenceMultiNodeAbstractTest.java |  11 -
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |   2 -
 ...dCacheQueueMultiNodeConsistencySelfTest.java |   5 +
 ...CacheQueueRotativeMultiNodeAbstractTest.java |  10 -
 .../GridCacheSetAbstractSelfTest.java           |   9 -
 ...omicOffheapQueueCreateMultiNodeSelfTest.java |   5 +
 ...ionedAtomicQueueCreateMultiNodeSelfTest.java |   5 +
 ...rtitionedDataStructuresFailoverSelfTest.java |   5 +
 ...edOffheapDataStructuresFailoverSelfTest.java |   5 +
 ...PartitionedQueueCreateMultiNodeSelfTest.java |   5 +
 ...dCachePartitionedQueueEntryMoveSelfTest.java |   5 +
 ...nedQueueFailoverDataConsistencySelfTest.java |   5 +
 ...eplicatedDataStructuresFailoverSelfTest.java |   5 +
 ...CacheLoadingConcurrentGridStartSelfTest.java |   5 +
 .../GridCacheAbstractJobExecutionTest.java      |   3 -
 .../GridCachePreloadLifecycleAbstractTest.java  |   2 -
 ...heAbstractTransformWriteThroughSelfTest.java |   3 -
 .../dht/GridCacheColocatedFailoverSelfTest.java |   5 +
 .../GridCacheColocatedTxExceptionSelfTest.java  |   5 +
 ...ePartitionedNearDisabledMetricsSelfTest.java |   4 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   5 +
 .../near/GridCacheNearEvictionSelfTest.java     |   3 -
 .../near/GridCacheNearTxExceptionSelfTest.java  |   5 +
 .../GridCachePartitionedFailoverSelfTest.java   |   5 +
 ...PartitionedFullApiMultithreadedSelfTest.java |   5 +
 ...idCachePartitionedHitsAndMissesSelfTest.java |   3 -
 .../GridCachePartitionedNodeRestartTest.java    |   5 +
 ...ePartitionedOptimisticTxNodeRestartTest.java |   5 +
 ...CachePartitionedTxMultiThreadedSelfTest.java |   5 +
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 +
 ...eReplicatedFullApiMultithreadedSelfTest.java |   5 +
 .../GridCacheReplicatedInvalidateSelfTest.java  |   4 +-
 ...ridCacheReplicatedMultiNodeLockSelfTest.java |   5 +
 .../GridCacheReplicatedMultiNodeSelfTest.java   |   5 +
 .../GridCacheReplicatedNodeRestartSelfTest.java |   5 +
 .../GridCacheReplicatedTxExceptionSelfTest.java |   5 +
 .../replicated/GridReplicatedTxPreloadTest.java |   2 +
 ...acheAtomicReplicatedNodeRestartSelfTest.java |   5 +
 .../GridCacheEvictionFilterSelfTest.java        |   4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   5 +
 .../IgniteCacheExpiryPolicyAbstractTest.java    |  10 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |   4 +-
 ...dCacheLocalFullApiMultithreadedSelfTest.java |   5 +
 .../GridCacheLocalTxExceptionSelfTest.java      |   5 +
 .../GridCacheSwapScanQueryAbstractSelfTest.java |   3 -
 ...ridCacheContinuousQueryAbstractSelfTest.java |   2 -
 .../closure/GridClosureProcessorSelfTest.java   |  29 +-
 .../continuous/GridEventConsumeSelfTest.java    |   2 -
 .../DataStreamProcessorSelfTest.java            |  44 +-
 .../processors/igfs/IgfsModesSelfTest.java      |   4 +-
 .../internal/util/nio/GridNioSelfTest.java      |  13 +-
 .../internal/util/nio/GridNioSslSelfTest.java   |   2 +
 .../unsafe/GridUnsafeMemorySelfTest.java        |   4 +-
 .../tostring/GridToStringBuilderSelfTest.java   |   4 +-
 .../marshaller/MarshallerContextTestImpl.java   |  11 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   3 -
 .../GridP2PContinuousDeploymentSelfTest.java    |   2 +
 .../p2p/GridP2PLocalDeploymentSelfTest.java     |   6 +-
 .../p2p/GridP2PRemoteClassLoadersSelfTest.java  |  31 +-
 .../spi/GridTcpSpiForwardingSelfTest.java       |   3 -
 .../ignite/testframework/GridTestUtils.java     |  14 +
 .../config/GridTestProperties.java              |  14 +-
 .../junits/IgniteTestResources.java             |  16 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |  29 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |  24 +-
 .../IgniteCacheEvictionSelfTestSuite.java       |   3 +-
 .../IgniteCacheFailoverTestSuite.java           |  22 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |   8 +-
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |  20 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |  10 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |  44 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   4 +-
 .../testsuites/IgniteCacheTestSuite3.java       |  14 +-
 .../testsuites/IgniteCacheTestSuite4.java       |  15 +-
 .../testsuites/IgniteKernalSelfTestSuite.java   |  14 +-
 .../IgniteMarshallerSelfTestSuite.java          |  28 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |  18 +-
 .../apache/ignite/util/GridRandomSelfTest.java  |   4 +-
 modules/extdata/p2p/pom.xml                     |   2 +-
 .../tests/p2p/P2PTestTaskExternalPath1.java     |  10 +-
 .../tests/p2p/P2PTestTaskExternalPath2.java     |   8 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   2 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |  21 +-
 .../collections/HadoopHashMapSelfTest.java      |   4 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |   2 +
 .../HadoopExternalCommunicationSelfTest.java    |   5 +
 .../testsuites/IgniteHadoopTestSuite.java       |   7 +-
 modules/hibernate/pom.xml                       |   2 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   5 +
 .../HibernateL2CacheTransactionalSelfTest.java  |   5 +
 .../testsuites/IgniteHibernateTestSuite.java    |   4 +-
 modules/indexing/pom.xml                        |  18 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |  10 +-
 .../cache/GridCacheOffHeapSelfTest.java         |   1 -
 ...idCacheReduceQueryMultithreadedSelfTest.java |  10 -
 .../processors/cache/GridCacheSwapSelfTest.java |   3 -
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |  13 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |   2 -
 ...hePartitionedQueryMultiThreadedSelfTest.java |  40 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   1 -
 .../IgniteCacheQueryNodeRestartSelfTest.java    |   5 +
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |   1 -
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   4 +-
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |   5 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/README.txt                        |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 .../apache/ignite/mesos/ClusterProperties.java  |  15 +
 .../apache/ignite/mesos/IgniteScheduler.java    |  10 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |  40 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 .../IgniteWebSessionSelfTestSuite.java          |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 212 files changed, 1554 insertions(+), 1032 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a5e97c2/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
index 11a8c84,e6b69bc..90000e8
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
@@@ -99,26 -99,23 +99,46 @@@ public final class ScanQuery<K, V> exte
      /**
       * Gets partition number over which this query should iterate. Will return {@code null} if partition was not
       * set. In this case query will iterate over all partitions in the cache.
 +     *
 +     * @return Partition number or {@code null}.
 +     */
 +    @Nullable public Integer getPartition() {
 +        return part;
 +    }
 +
 +    /**
 +     * Sets partition number over which this query should iterate. If {@code null}, query will iterate over
 +     * all partitions in the cache. Must be in the range [0, N) where N is partition number in the cache.
 +     *
 +     * @param part Partition number over which this query should iterate.
 +     * @return {@code this} for chaining.
 +     */
 +    public ScanQuery<K, V> setPartition(@Nullable Integer part) {
 +        this.part = part;
 +
 +        return this;
 +    }
 +
++    /**
++     * Gets partition number over which this query should iterate. Will return {@code null} if partition was not
++     * set. In this case query will iterate over all partitions in the cache.
+      *
+      * @return Partition number or {@code null}.
+      */
+     @Nullable public Integer getPartition() {
+         return part;
+     }
+ 
+     /**
+      * Sets partition number over which this query should iterate. If {@code null}, query will iterate over
+      * all partitions in the cache. Must be in the range [0, N) where N is partition number in the cache.
+      *
+      * @param part Partition number over which this query should iterate.
+      */
+     public void setPartition(@Nullable Integer part) {
+         this.part = part;
+     }
+ 
      /** {@inheritDoc} */
      @Override public ScanQuery<K, V> setPageSize(int pageSize) {
          return (ScanQuery<K, V>)super.setPageSize(pageSize);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a5e97c2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a5e97c2/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a5e97c2/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index ed9fc9a,c598e38..7fa038c
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@@ -140,6 -138,9 +138,11 @@@ public class IgniteCacheTestSuite4 exte
  
          suite.addTestSuite(IgniteCacheManyClientsTest.class);
  
+         suite.addTestSuite(IgniteStartCacheInTransactionSelfTest.class);
+         suite.addTestSuite(IgniteStartCacheInTransactionAtomicSelfTest.class);
+ 
++        suite.addTestSuite(IgniteCacheManyClientsTest.class);
++
          return suite;
      }
  }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a5e97c2/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a5e97c2/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
----------------------------------------------------------------------


[23/50] incubator-ignite git commit: Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/efb42447
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/efb42447
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/efb42447

Branch: refs/heads/ignite-1026
Commit: efb4244779b94c9c9f35c63708e4a41da2430bce
Parents: 94060c9 4298238 af829d0
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 19:50:12 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 19:50:12 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../processors/cache/IgniteCacheProxy.java      |   7 +
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  12 +-
 .../dr/IgniteDrDataStreamerCacheUpdater.java    |   7 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   | 305 +++++++++++++++++++
 ...eUsageMultinodeDynamicStartAbstractTest.java | 169 ++++++++++
 ...oreUsageMultinodeDynamicStartAtomicTest.java |  32 ++
 ...heStoreUsageMultinodeDynamicStartTxTest.java |  32 ++
 ...reUsageMultinodeStaticStartAbstractTest.java | 158 ++++++++++
 ...toreUsageMultinodeStaticStartAtomicTest.java |  32 ++
 ...cheStoreUsageMultinodeStaticStartTxTest.java |  32 ++
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 .../h2/twostep/GridReduceQueryExecutor.java     |   3 +-
 15 files changed, 793 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/efb42447/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 6635dde,6c407d9,11054b7..b956167
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@@@ -273,514 -273,477 -265,113 +273,515 @@@@ public class GridReduceQueryExecutor 
       }
   
       /**
  +     * @param r Query run.
  +     * @param retryVer Retry version.
  +     * @param nodeId Node ID.
  +     */
  +    private void retry(QueryRun r, AffinityTopologyVersion retryVer, UUID nodeId) {
  +        r.state(retryVer, nodeId);
  +    }
  +
  +    /**
  +     * @param cctx Cache context for main space.
  +     * @param extraSpaces Extra spaces.
  +     * @return {@code true} If preloading is active.
  +     */
  +    private boolean isPreloadingActive(final GridCacheContext<?,?> cctx, List<String> extraSpaces) {
  +        if (hasMovingPartitions(cctx))
  +            return true;
  +
  +        if (extraSpaces != null) {
  +            for (String extraSpace : extraSpaces) {
  +                if (hasMovingPartitions(cacheContext(extraSpace)))
  +                    return true;
  +            }
  +        }
  +
  +        return false;
  +    }
  +
  +    /**
  +     * @return {@code true} If cache context
  +     */
  +    private boolean hasMovingPartitions(GridCacheContext<?,?> cctx) {
  +        GridDhtPartitionFullMap fullMap = cctx.topology().partitionMap(false);
  +
  +        for (GridDhtPartitionMap map : fullMap.values()) {
  +            if (map.hasMovingPartitions())
  +                return true;
  +        }
  +
  +        return false;
  +    }
  +
  +    /**
  +     * @param name Cache name.
  +     * @return Cache context.
  +     */
  +    private GridCacheContext<?,?> cacheContext(String name) {
  +        return ctx.cache().internalCache(name).context();
  +    }
  +
  +    /**
  +     * @param topVer Topology version.
  +     * @param cctx Cache context for main space.
  +     * @param extraSpaces Extra spaces.
  +     * @return Data nodes or {@code null} if repartitioning started and we need to retry..
  +     */
  +    private Collection<ClusterNode> stableDataNodes(
  +        AffinityTopologyVersion topVer,
  +        final GridCacheContext<?,?> cctx,
  +        List<String> extraSpaces
  +    ) {
  +        String space = cctx.name();
  +
 -         Set<ClusterNode> nodes = new HashSet<>(ctx.discovery().cacheAffinityNodes(space, topVer));
 ++        Set<ClusterNode> nodes = new HashSet<>(dataNodes(space, topVer));
  +
  +        if (F.isEmpty(nodes))
  +            throw new CacheException("No data nodes found for cache: " + space);
  +
  +        if (!F.isEmpty(extraSpaces)) {
  +            for (String extraSpace : extraSpaces) {
  +                GridCacheContext<?,?> extraCctx = cacheContext(extraSpace);
  +
  +                if (extraCctx.isLocal())
  +                    continue; // No consistency guaranties for local caches.
  +
  +                if (cctx.isReplicated() && !extraCctx.isReplicated())
  +                    throw new CacheException("Queries running on replicated cache should not contain JOINs " +
  +                        "with partitioned tables.");
  +
 -                 Collection<ClusterNode> extraNodes = ctx.discovery().cacheAffinityNodes(extraSpace, topVer);
 ++                Collection<ClusterNode> extraNodes = dataNodes(extraSpace, topVer);
  +
  +                if (F.isEmpty(extraNodes))
  +                    throw new CacheException("No data nodes found for cache: " + extraSpace);
  +
  +                if (cctx.isReplicated() && extraCctx.isReplicated()) {
  +                    nodes.retainAll(extraNodes);
  +
  +                    if (nodes.isEmpty()) {
  +                        if (isPreloadingActive(cctx, extraSpaces))
  +                            return null; // Retry.
  +                        else
  +                            throw new CacheException("Caches '" + cctx.name() + "' and '" + extraSpace +
  +                                "' have distinct set of data nodes.");
  +                    }
  +                }
  +                else if (!cctx.isReplicated() && extraCctx.isReplicated()) {
  +                    if (!extraNodes.containsAll(nodes))
  +                        if (isPreloadingActive(cctx, extraSpaces))
  +                            return null; // Retry.
  +                        else
  +                            throw new CacheException("Caches '" + cctx.name() + "' and '" + extraSpace +
  +                                "' have distinct set of data nodes.");
  +                }
  +                else if (!cctx.isReplicated() && !extraCctx.isReplicated()) {
  +                    if (extraNodes.size() != nodes.size() || !nodes.containsAll(extraNodes))
  +                        if (isPreloadingActive(cctx, extraSpaces))
  +                            return null; // Retry.
  +                        else
  +                            throw new CacheException("Caches '" + cctx.name() + "' and '" + extraSpace +
  +                                "' have distinct set of data nodes.");
  +                }
  +                else
  +                    throw new IllegalStateException();
  +            }
  +        }
  +
  +        return nodes;
  +    }
  +
  +    /**
        * @param cctx Cache context.
        * @param qry Query.
  +     * @param keepPortable Keep portable.
        * @return Cursor.
        */
       public Iterator<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry, boolean keepPortable) {
 -         for (;;) {
  -        long qryReqId = reqIdGen.incrementAndGet();
 ++        for (int attempt = 0;; attempt++) {
 ++            if (attempt != 0) {
 ++                try {
 ++                    Thread.sleep(attempt * 10); // Wait for exchange.
 ++                }
 ++                catch (InterruptedException e) {
 ++                    Thread.currentThread().interrupt();
 + 
  -        QueryRun r = new QueryRun();
 ++                    throw new CacheException("Query was interrupted.", e);
 ++                }
 ++            }
 + 
  -        r.pageSize = qry.pageSize() <= 0 ? GridCacheTwoStepQuery.DFLT_PAGE_SIZE : qry.pageSize();
  +            long qryReqId = reqIdGen.incrementAndGet();
   
  -        r.tbls = new ArrayList<>(qry.mapQueries().size());
  +            QueryRun r = new QueryRun();
   
  -        String space = cctx.name();
  +            r.pageSize = qry.pageSize() <= 0 ? GridCacheTwoStepQuery.DFLT_PAGE_SIZE : qry.pageSize();
   
  -        r.conn = (JdbcConnection)h2.connectionForSpace(space);
  +            r.tbls = new ArrayList<>(qry.mapQueries().size());
   
  -        // TODO    Add topology version.
  -        ClusterGroup dataNodes = ctx.grid().cluster().forDataNodes(space);
  +            String space = cctx.name();
   
  -        if (cctx.isReplicated() || qry.explain()) {
  -            assert qry.explain() || dataNodes.node(ctx.localNodeId()) == null : "We must be on a client node.";
  +            r.conn = (JdbcConnection)h2.connectionForSpace(space);
   
  -            // Select random data node to run query on a replicated data or get EXPLAIN PLAN from a single node.
  -            dataNodes = dataNodes.forRandom();
  -        }
  +            AffinityTopologyVersion topVer = h2.readyTopologyVersion();
   
  -        final Collection<ClusterNode> nodes = dataNodes.nodes();
  +            List<String> extraSpaces = extraSpaces(space, qry.spaces());
   
  -        for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
  -            GridMergeTable tbl;
  +            Collection<ClusterNode> nodes;
  +
  +            // Explicit partition mapping for unstable topology.
  +            Map<ClusterNode, IntArray> partsMap = null;
  +
  +            if (isPreloadingActive(cctx, extraSpaces)) {
  +                if (cctx.isReplicated())
 -                     nodes = replicatedDataNodes(cctx, extraSpaces);
 ++                    nodes = replicatedUnstableDataNodes(cctx, extraSpaces);
  +                else {
 -                     partsMap = partitionLocations(cctx, extraSpaces);
 ++                    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.cluster().get().localNode()) : "We must be on a client node.";
+ +                assert qry.explain() || !nodes.contains(ctx.cluster().get().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));
  +            }
  +
  +            for (GridCacheSqlQuery mapQry : qry.mapQueries()) {
  +                GridMergeTable tbl;
  +
  +                try {
  +                    tbl = createFunctionTable(r.conn, mapQry, qry.explain()); // createTable(r.conn, mapQry); TODO
  +                }
  +                catch (IgniteCheckedException e) {
  +                    throw new IgniteException(e);
  +                }
  +
  +                GridMergeIndex idx = tbl.getScanIndex(null);
  +
  +                for (ClusterNode node : nodes)
  +                    idx.addSource(node.id());
  +
  +                r.tbls.add(tbl);
  +
  +                curFunTbl.set(tbl);
  +            }
  +
  +            r.latch = new CountDownLatch(r.tbls.size() * nodes.size());
  +
  +            runs.put(qryReqId, r);
   
               try {
  -                tbl = createFunctionTable(r.conn, mapQry, qry.explain()); // createTable(r.conn, mapQry); TODO
  +                Collection<GridCacheSqlQuery> mapQrys = qry.mapQueries();
  +
  +                if (qry.explain()) {
  +                    mapQrys = new ArrayList<>(qry.mapQueries().size());
  +
  +                    for (GridCacheSqlQuery mapQry : qry.mapQueries())
  +                        mapQrys.add(new GridCacheSqlQuery(mapQry.alias(), "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)) {
  +                    U.await(r.latch);
  +
  +                    Object state = r.state.get();
  +
  +                    if (state != null) {
  +                        if (state instanceof CacheException)
  +                            throw new CacheException("Failed to run map query remotely.", (CacheException)state);
  +
  +                        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;
  +
  +                ResultSet res = null;
  +
  +                if (!retry) {
  +                    if (qry.explain())
  +                        return explainPlan(r.conn, space, qry);
  +
  +                    GridCacheSqlQuery rdc = qry.reduceQuery();
  +
  +                    res = h2.executeSqlQueryWithTimer(space, r.conn, rdc.query(), F.asList(rdc.parameters()));
  +                }
  +
  +                for (GridMergeTable tbl : r.tbls) {
  +                    if (!tbl.getScanIndex(null).fetchedAll()) // We have to explicitly cancel queries on remote nodes.
  +                        send(nodes, new GridQueryCancelRequest(qryReqId), null);
  +
  +//                dropTable(r.conn, tbl.getName()); TODO
  +                }
  +
  +                if (retry) {
  +                    if (Thread.currentThread().isInterrupted())
  +                        throw new IgniteInterruptedCheckedException("Query was interrupted.");
  +
  +                    continue;
  +                }
  +
  +                return new GridQueryCacheObjectsIterator(new Iter(res), cctx, keepPortable);
               }
  -            catch (IgniteCheckedException e) {
  -                throw new IgniteException(e);
  +            catch (IgniteCheckedException | RuntimeException e) {
  +                U.closeQuiet(r.conn);
  +
 -                 if (e instanceof CacheException)
 -                     throw (CacheException)e;
 - 
  +                throw new CacheException("Failed to run reduce query locally.", e);
  +            }
  +            finally {
  +                if (!runs.remove(qryReqId, r))
  +                    U.warn(log, "Query run was already removed: " + qryReqId);
  +
  +                curFunTbl.remove();
               }
  +        }
  +    }
  +
  +    /**
  +     * Calculates data nodes for replicated caches on unstable topology.
  +     *
  +     * @param cctx Cache context for main space.
  +     * @param extraSpaces Extra spaces.
  +     * @return Collection of all data nodes owning all the caches or {@code null} for retry.
  +     */
 -     private Collection<ClusterNode> replicatedDataNodes(final GridCacheContext<?,?> cctx, List<String> extraSpaces) {
 ++    private Collection<ClusterNode> replicatedUnstableDataNodes(final GridCacheContext<?,?> cctx,
 ++        List<String> extraSpaces) {
  +        assert cctx.isReplicated() : cctx.name() + " must be replicated";
  +
 -         Set<ClusterNode> nodes = owningReplicatedDataNodes(cctx);
 ++        Set<ClusterNode> nodes = replicatedUnstableDataNodes(cctx);
 + 
  -            GridMergeIndex idx = tbl.getScanIndex(null);
 ++        if (F.isEmpty(nodes))
 ++            return null; // Retry.
   
  -            for (ClusterNode node : nodes)
  -                idx.addSource(node.id());
  +        if (!F.isEmpty(extraSpaces)) {
  +            for (String extraSpace : extraSpaces) {
  +                GridCacheContext<?,?> extraCctx = cacheContext(extraSpace);
   
  -            r.tbls.add(tbl);
  +                if (extraCctx.isLocal())
  +                    continue;
   
  -            curFunTbl.set(tbl);
  +                if (!extraCctx.isReplicated())
  +                    throw new CacheException("Queries running on replicated cache should not contain JOINs " +
  +                        "with partitioned tables.");
  +
 -                 nodes.retainAll(owningReplicatedDataNodes(extraCctx));
 ++                Set<ClusterNode> extraOwners = replicatedUnstableDataNodes(extraCctx);
 ++
 ++                if (F.isEmpty(extraOwners))
 ++                    return null; // Retry.
 ++
 ++                nodes.retainAll(extraOwners);
  +
  +                if (nodes.isEmpty())
  +                    return null; // Retry.
  +            }
           }
   
  -        r.latch = new CountDownLatch(r.tbls.size() * nodes.size());
  +        return nodes;
  +    }
  +
  +    /**
 ++     * @param space Cache name.
 ++     * @param topVer Topology version.
 ++     * @return Collection of data nodes.
 ++     */
 ++    private Collection<ClusterNode> dataNodes(String space, AffinityTopologyVersion topVer) {
 ++        Collection<ClusterNode> res = ctx.discovery().cacheAffinityNodes(space, topVer);
 + 
  -        runs.put(qryReqId, r);
 ++        return res != null ? res : Collections.<ClusterNode>emptySet();
 ++    }
 + 
  -        try {
  -            Collection<GridCacheSqlQuery> mapQrys = qry.mapQueries();
 ++    /**
  +     * Collects all the nodes owning all the partitions for the given replicated cache.
  +     *
  +     * @param cctx Cache context.
 -      * @return Owning nodes.
 ++     * @return Owning nodes or {@code null} if we can't find owners for some partitions.
  +     */
 -     private Set<ClusterNode> owningReplicatedDataNodes(GridCacheContext<?,?> cctx) {
 ++    private Set<ClusterNode> replicatedUnstableDataNodes(GridCacheContext<?,?> cctx) {
  +        assert cctx.isReplicated() : cctx.name() + " must be replicated";
  +
  +        String space = cctx.name();
  +
 -         Set<ClusterNode> dataNodes = new HashSet<>(ctx.discovery().cacheAffinityNodes(space, NONE));
 ++        Set<ClusterNode> dataNodes = new HashSet<>(dataNodes(space, NONE));
  +
  +        if (dataNodes.isEmpty())
  +            throw new CacheException("No data nodes found for cache '" + space + "'");
  +
  +        // Find all the nodes owning all the partitions for replicated cache.
 -         for (int p = 0, extraParts = cctx.affinity().partitions(); p < extraParts; p++) {
 ++        for (int p = 0, parts = cctx.affinity().partitions(); p < parts; p++) {
  +            List<ClusterNode> owners = cctx.topology().owners(p);
  +
 -             if (owners.isEmpty())
 -                 throw new CacheException("No data nodes found for cache '" + space +
 -                     "' for partition " + p);
 ++            if (F.isEmpty(owners))
 ++                return null; // Retry.
  +
  +            dataNodes.retainAll(owners);
  +
  +            if (dataNodes.isEmpty())
 -                 throw new CacheException("No data nodes found for cache '" + space +
 -                     "' owning all the partitions.");
 ++                return null; // Retry.
  +        }
  +
  +        return dataNodes;
  +    }
  +
  +    /**
  +     * 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> partitionLocations(final GridCacheContext<?,?> cctx, List<String> extraSpaces) {
 ++    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 (qry.explain()) {
  -                mapQrys = new ArrayList<>(qry.mapQueries().size());
  +                if (extraCctx.isReplicated() || extraCctx.isLocal())
  +                    continue;
   
  -                for (GridCacheSqlQuery mapQry : qry.mapQueries())
  -                    mapQrys.add(new GridCacheSqlQuery(mapQry.alias(), "EXPLAIN " + mapQry.query(), mapQry.parameters()));
  +                int parts = extraCctx.affinity().partitions();
  +
  +                if (parts != partsCnt)
  +                    throw new CacheException("Number of partitions must be the same for correct collocation in " +
  +                        "caches " + cctx.name() + " and " + extraSpace + ".");
               }
  +        }
  +
  +        Set<ClusterNode>[] partLocs = new Set[partsCnt];
   
  -            if (nodes.size() != 1 || !F.first(nodes).isLocal()) { // Marshall params for remotes.
  -                Marshaller m = ctx.config().getMarshaller();
  +        // 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))
  -                for (GridCacheSqlQuery mapQry : mapQrys)
  -                    mapQry.marshallParams(m);
 ++            if (F.isEmpty(owners)) {
 ++                if (!F.isEmpty(dataNodes(cctx.name(), NONE)))
 ++                    return null; // Retry.
 ++
  +                throw new CacheException("No data nodes found for cache '" + cctx.name() + "' for partition " + p);
 +             }
   
  -            send(nodes, new GridQueryRequest(qryReqId, r.pageSize, space, mapQrys));
  +            partLocs[p] = new HashSet<>(owners);
  +        }
   
  -            r.latch.await();
  +        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 (r.rmtErr != null)
  -                throw new CacheException("Failed to run map query remotely.", r.rmtErr);
  +                if (extraCctx.isReplicated() || extraCctx.isLocal())
  +                    continue;
   
  -            if (qry.explain())
  -                return explainPlan(r.conn, space, qry);
  +                for (int p = 0, parts =  extraCctx.affinity().partitions(); p < parts; p++) {
  +                    List<ClusterNode> owners = extraCctx.topology().owners(p);
   
 -                     if (F.isEmpty(owners))
  -            GridCacheSqlQuery rdc = qry.reduceQuery();
 ++                    if (F.isEmpty(owners)) {
 ++                        if (!F.isEmpty(dataNodes(extraSpace, NONE)))
 ++                            return null; // Retry.
 + 
  -            final ResultSet res = h2.executeSqlQueryWithTimer(space, r.conn, rdc.query(), F.asList(rdc.parameters()));
  +                        throw new CacheException("No data nodes found for cache '" + extraSpace +
  +                            "' for partition " + p);
 ++                    }
   
  -            for (GridMergeTable tbl : r.tbls) {
  -                if (!tbl.getScanIndex(null).fetchedAll()) // We have to explicitly cancel queries on remote nodes.
  -                    send(nodes, new GridQueryCancelRequest(qryReqId));
  +                    if (partLocs[p] == null)
  +                        partLocs[p] = new HashSet<>(owners);
  +                    else {
  +                        partLocs[p].retainAll(owners); // Intersection of owners.
   
  -//                dropTable(r.conn, tbl.getName()); TODO
  +                        if (partLocs[p].isEmpty())
  +                            return null; // Intersection is empty -> retry.
  +                    }
  +                }
               }
   
  -            return new GridQueryCacheObjectsIterator(new Iter(res), cctx, keepPortable);
  +            // Filter nodes where not all the replicated caches loaded.
  +            for (String extraSpace : extraSpaces) {
  +                GridCacheContext<?,?> extraCctx = cacheContext(extraSpace);
  +
  +                if (!extraCctx.isReplicated())
  +                    continue;
  +
 -                 Set<ClusterNode> dataNodes = owningReplicatedDataNodes(extraCctx);
 ++                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.
  +                }
  +            }
           }
  -        catch (IgniteCheckedException | InterruptedException | RuntimeException e) {
  -            U.closeQuiet(r.conn);
   
  -            if (e instanceof CacheException)
  -                throw (CacheException)e;
  +        // 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];
   
  -            throw new CacheException("Failed to run reduce query locally.", e);
  +            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);
           }
  -        finally {
  -            if (!runs.remove(qryReqId, r))
  -                U.warn(log, "Query run was already removed: " + qryReqId);
   
  -            curFunTbl.remove();
  +        return res;
  +    }
  +
  +    /**
  +     * @param mainSpace Main space.
  +     * @param allSpaces All spaces.
  +     * @return List of all extra spaces or {@code null} if none.
  +     */
  +    private List<String> extraSpaces(String mainSpace, Set<String> allSpaces) {
  +        if (F.isEmpty(allSpaces) || (allSpaces.size() == 1 && allSpaces.contains(mainSpace)))
  +            return null;
  +
  +        ArrayList<String> res = new ArrayList<>(allSpaces.size());
  +
  +        for (String space : allSpaces) {
  +            if (!F.eq(space, mainSpace))
  +                res.add(space);
           }
  +
  +        return res;
       }
   
       /**


[16/50] incubator-ignite git commit: Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/0b8587ba
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/0b8587ba
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/0b8587ba

Branch: refs/heads/ignite-1026
Commit: 0b8587ba81400a5227e8d3bb8be52f8c0d7d179f
Parents: c1fc7b5 5218210
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 14:40:23 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 14:40:23 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 13 +-----------
 .../client/GridClientConfiguration.java         |  2 +-
 .../GridClientOptimizedMarshaller.java          | 21 ++++++++++++++++++++
 .../impl/GridTcpRouterNioListenerAdapter.java   |  2 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |  3 ++-
 .../ignite/internal/util/IgniteUtils.java       | 21 ++++++++++++++++++++
 6 files changed, 47 insertions(+), 15 deletions(-)
----------------------------------------------------------------------



[44/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1003' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1003' into ignite-sprint-6


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/4d1fa723
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4d1fa723
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4d1fa723

Branch: refs/heads/ignite-1026
Commit: 4d1fa7235c1381bacaaeaf458658675d2ba34923
Parents: 31cf802 b23f930
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 15:15:49 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 15:15:49 2015 +0300

----------------------------------------------------------------------
 .../communication/tcp/TcpCommunicationSpi.java  | 38 ++++++++++----------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 21 +++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  9 ++++-
 .../tcp/internal/TcpDiscoveryNode.java          | 18 ++++++++++
 4 files changed, 60 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d1fa723/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d1fa723/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d1fa723/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------


[13/50] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/8343058d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/8343058d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/8343058d

Branch: refs/heads/ignite-1026
Commit: 8343058d5079344593af1dfdc04f004a9a02d843
Parents: 8e8433b f4b1123
Author: S.Vladykin <sv...@gridgain.com>
Authored: Tue Jun 16 13:52:51 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Tue Jun 16 13:52:51 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/query/ScanQuery.java    |  20 +-
 .../apache/ignite/internal/IgniteKernal.java    |   1 -
 .../discovery/GridDiscoveryManager.java         |   9 +-
 .../processors/cache/GridCacheProcessor.java    |  11 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |   6 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  28 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 439 ++++++++++++-------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 221 ++++++----
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  66 +++
 .../ipfinder/TcpDiscoveryIpFinderAdapter.java   |  34 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  19 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |  10 +-
 .../cache/CacheClientStoreSelfTest.java         | 228 ++++++++++
 ...acheReadOnlyTransactionalClientSelfTest.java | 327 --------------
 .../distributed/IgniteCacheManyClientsTest.java | 142 +++++-
 ...CacheClientWriteBehindStoreAbstractTest.java | 104 +++++
 ...teCacheClientWriteBehindStoreAtomicTest.java |  38 ++
 .../IgnteCacheClientWriteBehindStoreTxTest.java |  32 ++
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  73 ++-
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +-
 .../IgniteCacheWriteBehindTestSuite.java        |   2 +
 23 files changed, 1172 insertions(+), 646 deletions(-)
----------------------------------------------------------------------



[41/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/cb862b65
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/cb862b65
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/cb862b65

Branch: refs/heads/ignite-1026
Commit: cb862b65ffc1e0ecfc9a93b6af10881586df1ede
Parents: 5011c2b ca81476
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jun 18 14:50:56 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jun 18 14:50:56 2015 +0300

----------------------------------------------------------------------
 .../datastreamer/DataStreamerMultiThreadedSelfTest.java           | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------



[04/50] incubator-ignite git commit: Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branch 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/4aee1e2e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4aee1e2e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4aee1e2e

Branch: refs/heads/ignite-1026
Commit: 4aee1e2e5fabc623f91a5a07378c936b33fa4b26
Parents: 1bd7f05 b52e47a
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Jun 11 15:10:22 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Jun 11 15:10:22 2015 +0300

----------------------------------------------------------------------
 idea/ignite_codeStyle.xml                       | 147 +++++++++++++++++++
 .../apache/ignite/cache/query/ScanQuery.java    |   5 +-
 parent/pom.xml                                  |   1 +
 3 files changed, 152 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[43/50] incubator-ignite git commit: # disco spi usability - doc and warning messages on failure detection speed up

Posted by vk...@apache.org.
# disco spi usability - doc and warning messages on failure detection speed up


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/fc81bcfd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/fc81bcfd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/fc81bcfd

Branch: refs/heads/ignite-1026
Commit: fc81bcfd0a4fd20e4322a060d7745754f14767e1
Parents: 31cf802
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jun 18 15:08:33 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jun 18 15:08:33 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java  | 4 ++++
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java  | 7 ++++---
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fc81bcfd/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index e9a949a..f7b6de9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -2320,6 +2320,10 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 for (TcpDiscoveryNode n : failedNodes)
                     msgWorker.addMessage(new TcpDiscoveryNodeFailedMessage(locNodeId, n.id(), n.internalOrder()));
+
+                LT.warn(log, null, "Local node has detected failed nodes and started cluster-wide procedure. " +
+                        "To speed up failure detection please see 'Failure Detection' section under javadoc" +
+                        "for 'TcpDiscoverySpi'");
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fc81bcfd/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 9a26867..d7c33a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -64,9 +64,9 @@ import java.util.concurrent.atomic.*;
  * {@link TcpDiscoveryIpFinder} about self start (stops when send succeeds)
  * and then this info goes to coordinator. When coordinator processes join request
  * and issues node added messages and all other nodes then receive info about new node.
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Important Notice</h2>
- * Configuration defaults are chosen to make possible for discovery SPI to reliably work on
+ * <h1 class="header">Failure Detection</h1>
+ * Configuration defaults (see Configuration section below for details)
+ * are chosen to make possible for discovery SPI work reliably on
  * most of hardware and virtual deployments, but this has made failure detection time worse.
  * <p>
  * For stable low-latency networks the following more aggressive settings are recommended
@@ -76,6 +76,7 @@ import java.util.concurrent.atomic.*;
  * <li>Socket timeout (see {@link #setSocketTimeout(long)}) - 200ms</li>
  * <li>Message acknowledgement timeout (see {@link #setAckTimeout(long)}) - 50ms</li>
  * </ul>
+ * <h1 class="header">Configuration</h1>
  * <h2 class="header">Mandatory</h2>
  * There are no mandatory configuration parameters.
  * <h2 class="header">Optional</h2>


[29/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/4cc376be
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4cc376be
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4cc376be

Branch: refs/heads/ignite-1026
Commit: 4cc376bedd63aee6cb5de434c435b3384431ed63
Parents: fa6615a fffc2ec
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu Jun 18 09:45:26 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Jun 18 09:45:26 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../processors/cache/IgniteCacheProxy.java      |   7 +
 .../dht/GridDhtPartitionTopologyImpl.java       |   4 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  12 +-
 .../dr/IgniteDrDataStreamerCacheUpdater.java    |   7 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   | 305 +++++++++++++++++++
 ...eUsageMultinodeDynamicStartAbstractTest.java | 169 ++++++++++
 ...oreUsageMultinodeDynamicStartAtomicTest.java |  32 ++
 ...heStoreUsageMultinodeDynamicStartTxTest.java |  32 ++
 ...reUsageMultinodeStaticStartAbstractTest.java | 158 ++++++++++
 ...toreUsageMultinodeStaticStartAtomicTest.java |  32 ++
 ...cheStoreUsageMultinodeStaticStartTxTest.java |  32 ++
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  19 +-
 .../junits/common/GridCommonAbstractTest.java   |   8 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 17 files changed, 810 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[35/50] incubator-ignite git commit: sprint-6 - ignite-indexing module check

Posted by vk...@apache.org.
sprint-6 - ignite-indexing module check


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/c1eee186
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/c1eee186
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/c1eee186

Branch: refs/heads/ignite-1026
Commit: c1eee18615be56c7bb60d34614451612710b6cef
Parents: d874b00
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Jun 18 13:39:19 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Jun 18 13:39:19 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/IgniteCacheProxy.java    | 5 +++++
 .../ignite/internal/processors/query/GridQueryProcessor.java  | 7 +++++++
 2 files changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1eee186/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 0d70792..48fd259 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -556,6 +556,11 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
             !(qry instanceof ContinuousQuery))
             throw new CacheException("Indexing is disabled for cache: " + ctx.cache().name() +
                 ". Use setIndexedTypes or setTypeMetadata methods on CacheConfiguration to enable.");
+
+        if (!ctx.kernalContext().query().moduleEnabled() &&
+            (qry instanceof SqlQuery || qry instanceof SqlFieldsQuery || qry instanceof TextQuery))
+            throw new CacheException("Failed to execute query. Add module 'ignite-indexing' to the classpath " +
+                "of all Ignite nodes.");
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1eee186/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 4f2486b..e080c6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -107,6 +107,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @return {@code true} If indexing module is in classpath and successfully initialized.
+     */
+    public boolean moduleEnabled() {
+        return idx != null;
+    }
+
+    /**
      * @param ccfg Cache configuration.
      * @throws IgniteCheckedException If failed.
      */


[37/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/b23ea747
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/b23ea747
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/b23ea747

Branch: refs/heads/ignite-1026
Commit: b23ea747d9cf351f607d60f520781f5d46e98715
Parents: 08d134d c1eee18
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jun 18 13:47:19 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jun 18 13:47:19 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/IgniteCacheProxy.java    | 5 +++++
 .../ignite/internal/processors/query/GridQueryProcessor.java  | 7 +++++++
 2 files changed, 12 insertions(+)
----------------------------------------------------------------------



[02/50] incubator-ignite git commit: ignite-484-1 - compilation

Posted by vk...@apache.org.
ignite-484-1 - compilation


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/d340fe72
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d340fe72
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d340fe72

Branch: refs/heads/ignite-1026
Commit: d340fe72a99deab268dc019f6eaf474702f408b8
Parents: 51bf4b1
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Jun 11 10:04:22 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Jun 11 10:04:22 2015 +0300

----------------------------------------------------------------------
 .../h2/twostep/GridReduceQueryExecutor.java     | 26 +++++++++++++++-----
 1 file changed, 20 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d340fe72/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 3d2ae46..343a439 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.query.h2.twostep;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
@@ -38,7 +37,7 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.h2.command.*;
 import org.h2.command.ddl.*;
-import org.h2.command.dml.Query;
+import org.h2.command.dml.*;
 import org.h2.engine.*;
 import org.h2.expression.*;
 import org.h2.index.*;
@@ -395,9 +394,10 @@ public class GridReduceQueryExecutor {
     /**
      * @param cctx Cache context.
      * @param qry Query.
+     * @param keepPortable Keep portable.
      * @return Cursor.
      */
-    public QueryCursor<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry, boolean keepPortable) {
+    public Iterator<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry, boolean keepPortable) {
         for (;;) {
             long qryReqId = reqIdGen.incrementAndGet();
 
@@ -501,7 +501,6 @@ public class GridReduceQueryExecutor {
                             retry = true;
 
                             // If remote node asks us to retry then we have outdated full partition map.
-                            // TODO is this correct way to wait for a new map??
                             h2.awaitForReadyTopologyVersion((AffinityTopologyVersion)state);
                         }
                     }
@@ -534,7 +533,7 @@ public class GridReduceQueryExecutor {
                     continue;
                 }
 
-                return new QueryCursorImpl<>(new GridQueryCacheObjectsIterator(new Iter(res), cctx, cctx.keepPortable()));
+                return new GridQueryCacheObjectsIterator(new Iter(res), cctx, keepPortable);
             }
             catch (IgniteCheckedException | RuntimeException e) {
                 U.closeQuiet(r.conn);
@@ -687,7 +686,22 @@ public class GridReduceQueryExecutor {
                 }
             }
 
-            return new QueryCursorImpl<>(new GridQueryCacheObjectsIterator(new Iter(res), cctx, cctx.keepPortable()));
+            // Filter nodes where not all the replicated caches loaded.
+            for (String extraSpace : extraSpaces) {
+                GridCacheContext<?,?> extraCctx = cacheContext(extraSpace);
+
+                if (!extraCctx.isReplicated())
+                    continue;
+
+                Set<ClusterNode> dataNodes = owningReplicatedDataNodes(extraCctx);
+
+                for (Set<ClusterNode> partLoc : partLocs) {
+                    partLoc.retainAll(dataNodes);
+
+                    if (partLoc.isEmpty())
+                        return null; // Retry.
+                }
+            }
         }
 
         // Collect the final partitions mapping.


[30/50] incubator-ignite git commit: Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-6

Posted by vk...@apache.org.
Merge branches 'ignite-484-1' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-6


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/de531896
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/de531896
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/de531896

Branch: refs/heads/ignite-1026
Commit: de53189605c859a6eb9aa09c8181d5720c5dc28b
Parents: 4cc376b eebf9c1
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Jun 18 09:51:50 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Jun 18 09:51:50 2015 +0300

----------------------------------------------------------------------
 .../affinity/AffinityTopologyVersion.java       |   7 -
 .../distributed/dht/GridDhtLocalPartition.java  |  56 +-
 .../dht/GridDhtPartitionsReservation.java       | 292 +++++++++
 .../cache/distributed/dht/GridReservable.java   |  35 +
 .../dht/preloader/GridDhtPartitionMap.java      |  26 +-
 .../cache/query/GridCacheQueryManager.java      |  33 -
 .../cache/query/GridCacheTwoStepQuery.java      |  22 +-
 .../processors/query/GridQueryIndexing.java     |  14 +-
 .../processors/query/GridQueryProcessor.java    |  14 +-
 .../messages/GridQueryNextPageResponse.java     |  34 +-
 .../h2/twostep/messages/GridQueryRequest.java   | 111 +++-
 .../apache/ignite/internal/util/GridDebug.java  |  19 +
 .../processors/query/h2/IgniteH2Indexing.java   |  79 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  49 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  | 332 +++++++---
 .../query/h2/twostep/GridMergeIndex.java        |  17 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   7 +-
 .../h2/twostep/GridReduceQueryExecutor.java     | 650 ++++++++++++++++---
 .../query/h2/twostep/GridResultPage.java        |  21 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   3 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   1 -
 ...lientQueryReplicatedNodeRestartSelfTest.java | 419 ++++++++++++
 .../IgniteCacheQueryNodeRestartSelfTest.java    |  36 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   | 383 +++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 25 files changed, 2380 insertions(+), 282 deletions(-)
----------------------------------------------------------------------



[17/50] incubator-ignite git commit: # i-484-1 review

Posted by vk...@apache.org.
# i-484-1 review


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/642f1c7a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/642f1c7a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/642f1c7a

Branch: refs/heads/ignite-1026
Commit: 642f1c7a2786fd95a43d97cc5b0d955ad890f0c4
Parents: 0b8587b
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jun 17 15:14:16 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jun 17 15:14:16 2015 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionsReservation.java       |  3 +-
 .../cache/query/GridCacheTwoStepQuery.java      | 10 +++----
 .../IgniteCacheQueryNodeRestartSelfTest.java    |  3 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   | 30 ++++++++++++++------
 4 files changed, 30 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/642f1c7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
index aced999..207c7f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
@@ -218,7 +218,8 @@ public class GridDhtPartitionsReservation implements GridReservable {
 
         // Unregister from partitions.
         if (!F.isEmpty(arr) && parts.compareAndSet(arr, EMPTY)) {
-            // Reverse order makes sure that addReservation on the same topVer reservation will fail on the first partition.
+            // Reverse order makes sure that addReservation on the same topVer
+            // reservation will fail on the first partition.
             for (int i = arr.length - 1; i >= 0; i--) {
                 GridDhtLocalPartition part = arr[i];
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/642f1c7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
index 1aa5890..1dacd10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -117,11 +117,6 @@ public class GridCacheTwoStepQuery {
         return mapQrys.values();
     }
 
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridCacheTwoStepQuery.class, this);
-    }
-
     /**
      * @return Spaces.
      */
@@ -135,4 +130,9 @@ public class GridCacheTwoStepQuery {
     public void spaces(Set<String> spaces) {
         this.spaces = spaces;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridCacheTwoStepQuery.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/642f1c7a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
index d729e4a..5c87603 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java
@@ -39,6 +39,7 @@ import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
 
 /**
  * Test for distributed queries with node restarts.
@@ -84,7 +85,7 @@ public class IgniteCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTe
         cc.setBackups(1);
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cc.setAtomicityMode(TRANSACTIONAL);
-        cc.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cc.setRebalanceMode(SYNC);
         cc.setAffinity(new RendezvousAffinityFunction(false, 15));
         cc.setIndexedTypes(
             Integer.class, Integer.class

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/642f1c7a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
index 527dfea..93831cc 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cache.query.*;
@@ -40,6 +39,8 @@ import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 
 /**
  * Test for distributed queries with node restarts.
@@ -95,9 +96,9 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
             cc.setName(name);
             cc.setCacheMode(PARTITIONED);
             cc.setBackups(2);
-            cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+            cc.setWriteSynchronizationMode(FULL_SYNC);
             cc.setAtomicityMode(TRANSACTIONAL);
-            cc.setRebalanceMode(CacheRebalanceMode.SYNC);
+            cc.setRebalanceMode(SYNC);
             cc.setAffinity(new RendezvousAffinityFunction(false, 60));
 
             if (name.equals("pe")) {
@@ -119,9 +120,9 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
 
             cc.setName(name);
             cc.setCacheMode(REPLICATED);
-            cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+            cc.setWriteSynchronizationMode(FULL_SYNC);
             cc.setAtomicityMode(TRANSACTIONAL);
-            cc.setRebalanceMode(CacheRebalanceMode.SYNC);
+            cc.setRebalanceMode(SYNC);
             cc.setAffinity(new RendezvousAffinityFunction(false, 50));
 
             if (name.equals("co")) {
@@ -143,6 +144,9 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
         return c;
     }
 
+    /**
+     *
+     */
     private void fillCaches() {
         IgniteCache<Integer, Company> co = grid(0).cache("co");
 
@@ -172,8 +176,6 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
     }
 
     /**
-     * JUnit.
-     *
      * @throws Exception If failed.
      */
     public void testRestarts() throws Exception {
@@ -323,7 +325,9 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
         info("Queries stopped.");
     }
 
-    // Partitioned
+    /**
+     *
+     */
     private static class Person implements Serializable {
         @QuerySqlField(index = true)
         int id;
@@ -333,6 +337,9 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
         }
     }
 
+    /**
+     *
+     */
     private static class Purchase implements Serializable {
         @QuerySqlField(index = true)
         int personId;
@@ -346,7 +353,9 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
         }
     }
 
-    // Replicated
+    /**
+     *
+     */
     private static class Company implements Serializable {
         @QuerySqlField(index = true)
         int id;
@@ -356,6 +365,9 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
         }
     }
 
+    /**
+     *
+     */
     private static class Product implements Serializable {
         @QuerySqlField(index = true)
         int id;


[42/50] incubator-ignite git commit: # changed discovery error message

Posted by vk...@apache.org.
# changed discovery error message


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/31cf8027
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/31cf8027
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/31cf8027

Branch: refs/heads/ignite-1026
Commit: 31cf8027216afca3d02461caa4f4245bdd8d0e73
Parents: cb862b6
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jun 18 14:56:12 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jun 18 14:56:12 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/discovery/tcp/ServerImpl.java   | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/31cf8027/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 63f165d..e9a949a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -4103,8 +4103,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                         if (U.isMacInvalidArgumentError(e))
                             LT.error(log, e, "Failed to initialize connection [sock=" + sock + "]\n\t" +
                                 U.MAC_INVALID_ARG_MSG);
-                        else
-                            LT.error(log, e, "Failed to initialize connection [sock=" + sock + ']');
+                        else {
+                            U.error(
+                                log,
+                                "Failed to initialize connection (this can happen due to short time " +
+                                    "network problems and can be ignored if does not affect node discovery) " +
+                                    "[sock=" + sock + ']',
+                                e);
+                        }
                     }
 
                     onException("Caught exception on handshake [err=" + e + ", sock=" + sock + ']', e);


[05/50] incubator-ignite git commit: Merge branches 'ignite-484-1' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1

Posted by vk...@apache.org.
Merge branches 'ignite-484-1' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-484-1


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/a7229dcb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/a7229dcb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/a7229dcb

Branch: refs/heads/ignite-1026
Commit: a7229dcba352f9b0c3ddb2432bb5beb7d9bfca50
Parents: 4aee1e2 4375529
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 12 00:16:51 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 12 00:16:51 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  38 +------
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 114 ++++++++++++++++++-
 2 files changed, 115 insertions(+), 37 deletions(-)
----------------------------------------------------------------------



[26/50] incubator-ignite git commit: # ignite-sprint-6 more info in assert

Posted by vk...@apache.org.
# ignite-sprint-6 more info in assert


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/fffc2ecc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/fffc2ecc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/fffc2ecc

Branch: refs/heads/ignite-1026
Commit: fffc2ecc86628a9a25c6f855d89277d124711873
Parents: af829d0
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 08:57:39 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 08:57:39 2015 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java        |  4 +++-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java  | 19 ++++++++++---------
 .../junits/common/GridCommonAbstractTest.java    |  8 ++++++--
 3 files changed, 19 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fffc2ecc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 374ab87..de7f876 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -620,7 +620,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         lock.readLock().lock();
 
         try {
-            assert node2part != null && node2part.valid() : "Invalid node-to-partitions map [topVer=" + topVer +
+            assert node2part != null && node2part.valid() : "Invalid node-to-partitions map [topVer1=" + topVer +
+                ", topVer2=" + this.topVer +
+                ", cache=" + cctx.name() +
                 ", node2part=" + node2part + ']';
 
             Collection<ClusterNode> nodes = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fffc2ecc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
index 017f7db..356d67b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
@@ -26,11 +26,13 @@ import org.apache.ignite.transactions.*;
 import javax.cache.*;
 import java.io.*;
 
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
 /**
  * Checks behavior on exception while unmarshalling key.
  */
 public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmarshallingErrorTest {
-
     /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
         return CacheAtomicityMode.TRANSACTIONAL;
@@ -41,7 +43,7 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         if (!gridName.endsWith("0"))
-            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); //allows to check GridDhtLockRequest fail.
+            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); // Allows to check GridDhtLockRequest fail.
 
         return cfg;
     }
@@ -52,8 +54,7 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
     protected void failOptimistic() {
         IgniteCache<Object, Object> cache = jcache(0);
 
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.OPTIMISTIC,
-            TransactionIsolation.REPEATABLE_READ)) {
+        try (Transaction tx = grid(0).transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
 
             cache.put(new TestKey(String.valueOf(++key)), "");
 
@@ -65,7 +66,7 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
             assert X.hasCause(e, IOException.class);
         }
 
-        assert readCnt.get() == 0; //ensure we have read count as expected.
+        assert readCnt.get() == 0; // Ensure we have read count as expected.
     }
 
     /**
@@ -74,8 +75,8 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
     protected void failPessimictic() {
         IgniteCache<Object, Object> cache = jcache(0);
 
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC,
-            TransactionIsolation.REPEATABLE_READ)) {
+        try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC,
+            REPEATABLE_READ)) {
 
             cache.put(new TestKey(String.valueOf(++key)), "");
 
@@ -85,7 +86,7 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
             assert X.hasCause(e, IOException.class);
         }
 
-        assert readCnt.get() == 0; //ensure we have read count as expected.
+        assert readCnt.get() == 0; // Ensure we have read count as expected.
     }
 
     /** {@inheritDoc} */
@@ -108,7 +109,7 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
         //GridDhtLockRequest unmarshalling failed test
         readCnt.set(3);
 
-        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+        try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
             jcache(0).put(new TestKey(String.valueOf(++key)), ""); //No failure at client side.
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fffc2ecc/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index bc33746..9941ca8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -379,8 +379,10 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
                             int exp = affNodes.size();
 
-                            Collection<ClusterNode> owners = top.topologyVersion() == AffinityTopologyVersion.NONE ?
-                                Collections.<ClusterNode>emptyList() : top.nodes(p, AffinityTopologyVersion.NONE);
+                            GridDhtTopologyFuture topFut = top.topologyVersionFuture();
+
+                            Collection<ClusterNode> owners = (topFut != null && topFut.isDone()) ?
+                                top.nodes(p, AffinityTopologyVersion.NONE) : Collections.<ClusterNode>emptyList();
 
                             int actual = owners.size();
 
@@ -390,6 +392,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                     ", cache=" + cfg.getName() +
                                     ", cacheId=" + dht.context().cacheId() +
                                     ", topVer=" + top.topologyVersion() +
+                                    ", topFut=" + topFut +
                                     ", p=" + p +
                                     ", affNodesCnt=" + exp +
                                     ", ownersCnt=" + actual +
@@ -406,6 +409,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                         ", cache=" + cfg.getName() +
                                         ", cacheId=" + dht.context().cacheId() +
                                         ", topVer=" + top.topologyVersion() +
+                                        ", topFut=" + topFut +
                                         ", p=" + p +
                                         ", affNodesCnt=" + exp +
                                         ", ownersCnt=" + actual +


[40/50] incubator-ignite git commit: # changed defaults for discovery

Posted by vk...@apache.org.
# changed defaults for discovery


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/5011c2b5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/5011c2b5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/5011c2b5

Branch: refs/heads/ignite-1026
Commit: 5011c2b5e9671ced95e21f122c2e5ef9102c19fe
Parents: b23ea74
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jun 18 14:50:43 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jun 18 14:50:43 2015 +0300

----------------------------------------------------------------------
 .../communication/tcp/TcpCommunicationSpi.java  |  6 ++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 32 +++++++++++++-------
 2 files changed, 24 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5011c2b5/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 9e38788..fc504f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -174,8 +174,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Default socket send and receive buffer size. */
     public static final int DFLT_SOCK_BUF_SIZE = 32 * 1024;
 
-    /** Default connection timeout (value is <tt>1000</tt>ms). */
-    public static final long DFLT_CONN_TIMEOUT = 1000;
+    /** Default connection timeout (value is <tt>5000</tt>ms). */
+    public static final long DFLT_CONN_TIMEOUT = 5000;
 
     /** Default Maximum connection timeout (value is <tt>600,000</tt>ms). */
     public static final long DFLT_MAX_CONN_TIMEOUT = 10 * 60 * 1000;
@@ -680,7 +680,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /** Recovery and idle clients handler. */
     private CommunicationWorker commWorker;
-    
+
     /** Shared memory accept worker. */
     private ShmemAcceptWorker shmemAcceptWorker;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5011c2b5/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index e4ef744..9a26867 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -66,6 +65,17 @@ import java.util.concurrent.atomic.*;
  * and then this info goes to coordinator. When coordinator processes join request
  * and issues node added messages and all other nodes then receive info about new node.
  * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Important Notice</h2>
+ * Configuration defaults are chosen to make possible for discovery SPI to reliably work on
+ * most of hardware and virtual deployments, but this has made failure detection time worse.
+ * <p>
+ * For stable low-latency networks the following more aggressive settings are recommended
+ * (which allows failure detection time ~200ms):
+ * <ul>
+ * <li>Heartbeat frequency (see {@link #setHeartbeatFrequency(long)}) - 100ms</li>
+ * <li>Socket timeout (see {@link #setSocketTimeout(long)}) - 200ms</li>
+ * <li>Message acknowledgement timeout (see {@link #setAckTimeout(long)}) - 50ms</li>
+ * </ul>
  * <h2 class="header">Mandatory</h2>
  * There are no mandatory configuration parameters.
  * <h2 class="header">Optional</h2>
@@ -164,23 +174,23 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** Default value for thread priority (value is <tt>10</tt>). */
     public static final int DFLT_THREAD_PRI = 10;
 
-    /** Default heartbeat messages issuing frequency (value is <tt>100ms</tt>). */
-    public static final long DFLT_HEARTBEAT_FREQ = 100;
+    /** Default heartbeat messages issuing frequency (value is <tt>2000ms</tt>). */
+    public static final long DFLT_HEARTBEAT_FREQ = 2000;
 
     /** Default size of topology snapshots history. */
     public static final int DFLT_TOP_HISTORY_SIZE = 1000;
 
-    /** Default socket operations timeout in milliseconds (value is <tt>200ms</tt>). */
-    public static final long DFLT_SOCK_TIMEOUT = 200;
+    /** Default socket operations timeout in milliseconds (value is <tt>5000ms</tt>). */
+    public static final long DFLT_SOCK_TIMEOUT = 5000;
 
-    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>50ms</tt>). */
-    public static final long DFLT_ACK_TIMEOUT = 50;
+    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>5000ms</tt>). */
+    public static final long DFLT_ACK_TIMEOUT = 5000;
 
-    /** Default socket operations timeout in milliseconds (value is <tt>700ms</tt>). */
-    public static final long DFLT_SOCK_TIMEOUT_CLIENT = 700;
+    /** Default socket operations timeout in milliseconds (value is <tt>5000ms</tt>). */
+    public static final long DFLT_SOCK_TIMEOUT_CLIENT = 5000;
 
-    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>700ms</tt>). */
-    public static final long DFLT_ACK_TIMEOUT_CLIENT = 700;
+    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>5000ms</tt>). */
+    public static final long DFLT_ACK_TIMEOUT_CLIENT = 5000;
 
     /** Default reconnect attempts count (value is <tt>10</tt>). */
     public static final int DFLT_RECONNECT_CNT = 10;


[50/50] incubator-ignite git commit: IGNITE-1026 - Data structures fix

Posted by vk...@apache.org.
IGNITE-1026 - Data structures fix


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/1419d39b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/1419d39b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/1419d39b

Branch: refs/heads/ignite-1026
Commit: 1419d39ba80da9f85b28c708589f74dcec6514d5
Parents: 2f089f1
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 16:54:51 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 16:54:51 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java     | 2 +-
 .../processors/datastructures/DataStructuresProcessor.java       | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1419d39b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index ac2d7b1..4f1a3e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -2535,7 +2535,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
 
         if (cache == null) {
-            dynamicStartCache(null, name, null, false, false);
+            dynamicStartCache(null, name, null, false, true).get();
 
             cache = jCacheProxies.get(masked);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1419d39b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index dcd22cd..26e2f6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -177,7 +177,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         if (initLatch.getCount() > 0) {
             initFailed = true;
-            
+
             initLatch.countDown();
         }
 
@@ -890,7 +890,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
             String cacheName = ((CollectionInfo)oldInfo.info).cacheName;
 
-            GridCacheContext cacheCtx = ctx.cache().internalCache(cacheName).context();
+            GridCacheContext cacheCtx = ctx.cache().getOrStartCache(cacheName).context();
 
             return c.applyx(cacheCtx);
         }


[11/50] incubator-ignite git commit: ignite-484-1 - tryEvict fix + minor refactor

Posted by vk...@apache.org.
ignite-484-1 - tryEvict fix + minor refactor


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/7e3f924b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/7e3f924b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/7e3f924b

Branch: refs/heads/ignite-1026
Commit: 7e3f924b2f6f2dcc0a942cdc569b7e27647de5cb
Parents: d7a18c4
Author: S.Vladykin <sv...@gridgain.com>
Authored: Sun Jun 14 20:59:24 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Sun Jun 14 20:59:24 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java  | 12 ++++++----
 .../dht/GridDhtPartitionsReservation.java       | 25 +++++++++++++-------
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |  2 +-
 3 files changed, 25 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7e3f924b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 018ffd6..5938fc8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -137,7 +137,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      * Adds group reservation to this partition.
      *
      * @param r Reservation.
-     * @return {@code true} If reservation added successfully.
+     * @return {@code false} If such reservation already added.
      */
     public boolean addReservation(GridDhtPartitionsReservation r) {
         assert state.getReference() != EVICTED : "we can reserve only active partitions";
@@ -509,7 +509,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         for (GridDhtPartitionsReservation reservation : reservations) {
             if (!reservation.canEvict())
-                reserved = true;
+                reserved = true; // Calling all the reservations to allow them unregister themselves.
         }
 
         return reserved;
@@ -520,11 +520,13 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      * @return {@code True} if entry has been transitioned to state EVICTED.
      */
     boolean tryEvict(boolean updateSeq) {
+        if (state.getReference() != RENTING || state.getStamp() != 0 || groupReserved())
+            return false;
+
         // Attempt to evict partition entries from cache.
-        if (state.getReference() == RENTING && state.getStamp() == 0 && !groupReserved())
-            clearAll();
+        clearAll();
 
-        if (map.isEmpty() && !groupReserved() && state.compareAndSet(RENTING, EVICTED, 0, 0)) {
+        if (map.isEmpty() && state.compareAndSet(RENTING, EVICTED, 0, 0)) {
             if (log.isDebugEnabled())
                 log.debug("Evicted partition: " + this);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7e3f924b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
index 71a1859..a32946a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
@@ -46,10 +46,10 @@ public class GridDhtPartitionsReservation implements GridReservable {
     private final AtomicReference<GridDhtLocalPartition[]> parts = new AtomicReference<>();
 
     /** */
-    private final AtomicInteger reservations = new AtomicInteger();
+    private final AtomicReference<CI1<GridDhtPartitionsReservation>> unpublish = new AtomicReference<>();
 
     /** */
-    private volatile CI1<GridDhtPartitionsReservation> unpublish;
+    private final AtomicInteger reservations = new AtomicInteger();
 
     /**
      * @param topVer AffinityTopologyVersion version.
@@ -123,6 +123,8 @@ public class GridDhtPartitionsReservation implements GridReservable {
         if (!this.parts.compareAndSet(null, arr))
             throw new IllegalStateException("Partitions can be registered only once.");
 
+        assert reservations.get() != -1 : "all the partitions must be reserved before register, we can't be invalidated";
+
         return true;
     }
 
@@ -133,9 +135,9 @@ public class GridDhtPartitionsReservation implements GridReservable {
      */
     public void onPublish(CI1<GridDhtPartitionsReservation> unpublish) {
         assert unpublish != null;
-        assert this.unpublish == null;
 
-        this.unpublish = unpublish;
+        if (!this.unpublish.compareAndSet(null, unpublish))
+            throw new IllegalStateException("Unpublishing closure can be set only once.");
 
         if (reservations.get() == -1)
             unregister();
@@ -202,12 +204,13 @@ public class GridDhtPartitionsReservation implements GridReservable {
     }
 
     /**
-     * Unregisters this reservation from all the partitions.
+     * Unregisters from all the partitions and unpublishes this reservation.
      */
     private void unregister() {
         GridDhtLocalPartition[] arr = parts.get();
 
-        if (!F.isEmpty(arr) && unpublish != null && parts.compareAndSet(arr, EMPTY)) {
+        // Unregister from partitions.
+        if (!F.isEmpty(arr) && parts.compareAndSet(arr, EMPTY)) {
             // Reverse order makes sure that addReservation on the same topVer reservation will fail on the first partition.
             for (int i = arr.length - 1; i >= 0; i--) {
                 GridDhtLocalPartition part = arr[i];
@@ -216,9 +219,13 @@ public class GridDhtPartitionsReservation implements GridReservable {
 
                 tryEvict(part);
             }
-
-            unpublish.apply(this);
         }
+
+        // Unpublish.
+        CI1<GridDhtPartitionsReservation> u = unpublish.get();
+
+        if (u != null && unpublish.compareAndSet(u, null))
+            u.apply(this);
     }
 
     /**
@@ -231,6 +238,8 @@ public class GridDhtPartitionsReservation implements GridReservable {
      * @return {@code true} If this reservation is NOT reserved and partition CAN be evicted.
      */
     public boolean canEvict() {
+        assert parts.get() != null : "all parts must be reserved before registration";
+
         int r = reservations.get();
 
         assert r >= -1 : r;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7e3f924b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
index 746cc45..e65cc13 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
@@ -180,7 +180,7 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
         int qryThreadNum = 4;
         int restartThreadsNum = 2; // 4 + 2 = 6 nodes
         final int nodeLifeTime = 2 * 1000;
-        final int logFreq = 50;
+        final int logFreq = 10;
 
         startGridsMultiThreaded(GRID_CNT);
 


[22/50] incubator-ignite git commit: ignite-484-1 - improved retry

Posted by vk...@apache.org.
ignite-484-1 - improved retry


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/94060c9e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/94060c9e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/94060c9e

Branch: refs/heads/ignite-1026
Commit: 94060c9ef41161c7262a28044ddb176f86814b01
Parents: 10febf2
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 19:46:42 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 19:46:42 2015 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/GridMapQueryExecutor.java  | 26 ++++--
 .../h2/twostep/GridReduceQueryExecutor.java     | 86 ++++++++++++++------
 ...lientQueryReplicatedNodeRestartSelfTest.java | 50 ++++++++++--
 3 files changed, 125 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/94060c9e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index aaf64ee..2503a87 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -48,6 +48,7 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.*;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.*;
 import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2ValueMessageFactory.*;
 
@@ -230,6 +231,15 @@ public class GridMapQueryExecutor {
     }
 
     /**
+     * @param cctx Cache context.
+     * @param p Partition ID.
+     * @return Partition.
+     */
+    private GridDhtLocalPartition partition(GridCacheContext<?, ?> cctx, int p) {
+        return cctx.topology().localPartition(p, NONE, false);
+    }
+
+    /**
      * @param cacheNames Cache names.
      * @param topVer Topology version.
      * @param explicitParts Explicit partitions list.
@@ -263,10 +273,12 @@ public class GridMapQueryExecutor {
             GridReservable r = reservations.get(grpKey);
 
             if (explicitParts == null && r != null) { // Try to reserve group partition if any and no explicits.
-                if (!r.reserve())
-                    return false; // We need explicit partitions here -> retry.
+                if (r != ReplicatedReservation.INSTANCE) {
+                    if (!r.reserve())
+                        return false; // We need explicit partitions here -> retry.
 
-                reserved.add(r);
+                    reserved.add(r);
+                }
             }
             else { // Try to reserve partitions one by one.
                 int partsCnt = cctx.affinity().partitions();
@@ -274,7 +286,7 @@ public class GridMapQueryExecutor {
                 if (cctx.isReplicated()) { // Check all the partitions are in owning state for replicated cache.
                     if (r == null) { // Check only once.
                         for (int p = 0; p < partsCnt; p++) {
-                            GridDhtLocalPartition part = cctx.topology().localPartition(p, topVer, false);
+                            GridDhtLocalPartition part = partition(cctx, p);
 
                             // We don't need to reserve partitions because they will not be evicted in replicated caches.
                             if (part == null || part.state() != OWNING)
@@ -290,7 +302,7 @@ public class GridMapQueryExecutor {
                         partIds = cctx.affinity().primaryPartitions(ctx.localNodeId(), topVer);
 
                     for (int partId : partIds) {
-                        GridDhtLocalPartition part = cctx.topology().localPartition(partId, topVer, false);
+                        GridDhtLocalPartition part = partition(cctx, partId);
 
                         if (part == null || part.state() != OWNING || !part.reserve())
                             return false;
@@ -806,12 +818,12 @@ public class GridMapQueryExecutor {
 
         /** {@inheritDoc} */
         @Override public boolean reserve() {
-            return true;
+            throw new IllegalStateException();
         }
 
         /** {@inheritDoc} */
         @Override public void release() {
-            // No-op.
+            throw new IllegalStateException();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/94060c9e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index c570d24..6635dde 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -335,7 +335,7 @@ public class GridReduceQueryExecutor {
     ) {
         String space = cctx.name();
 
-        Set<ClusterNode> nodes = new HashSet<>(ctx.discovery().cacheAffinityNodes(space, topVer));
+        Set<ClusterNode> nodes = new HashSet<>(dataNodes(space, topVer));
 
         if (F.isEmpty(nodes))
             throw new CacheException("No data nodes found for cache: " + space);
@@ -351,7 +351,7 @@ public class GridReduceQueryExecutor {
                     throw new CacheException("Queries running on replicated cache should not contain JOINs " +
                         "with partitioned tables.");
 
-                Collection<ClusterNode> extraNodes = ctx.discovery().cacheAffinityNodes(extraSpace, topVer);
+                Collection<ClusterNode> extraNodes = dataNodes(extraSpace, topVer);
 
                 if (F.isEmpty(extraNodes))
                     throw new CacheException("No data nodes found for cache: " + extraSpace);
@@ -398,7 +398,18 @@ public class GridReduceQueryExecutor {
      * @return Cursor.
      */
     public Iterator<List<?>> query(GridCacheContext<?,?> cctx, GridCacheTwoStepQuery qry, boolean keepPortable) {
-        for (;;) {
+        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();
@@ -422,9 +433,9 @@ public class GridReduceQueryExecutor {
 
             if (isPreloadingActive(cctx, extraSpaces)) {
                 if (cctx.isReplicated())
-                    nodes = replicatedDataNodes(cctx, extraSpaces);
+                    nodes = replicatedUnstableDataNodes(cctx, extraSpaces);
                 else {
-                    partsMap = partitionLocations(cctx, extraSpaces);
+                    partsMap = partitionedUnstableDataNodes(cctx, extraSpaces);
 
                     nodes = partsMap == null ? null : partsMap.keySet();
                 }
@@ -538,9 +549,6 @@ public class GridReduceQueryExecutor {
             catch (IgniteCheckedException | RuntimeException e) {
                 U.closeQuiet(r.conn);
 
-                if (e instanceof CacheException)
-                    throw (CacheException)e;
-
                 throw new CacheException("Failed to run reduce query locally.", e);
             }
             finally {
@@ -559,10 +567,14 @@ public class GridReduceQueryExecutor {
      * @param extraSpaces Extra spaces.
      * @return Collection of all data nodes owning all the caches or {@code null} for retry.
      */
-    private Collection<ClusterNode> replicatedDataNodes(final GridCacheContext<?,?> cctx, List<String> extraSpaces) {
+    private Collection<ClusterNode> replicatedUnstableDataNodes(final GridCacheContext<?,?> cctx,
+        List<String> extraSpaces) {
         assert cctx.isReplicated() : cctx.name() + " must be replicated";
 
-        Set<ClusterNode> nodes = owningReplicatedDataNodes(cctx);
+        Set<ClusterNode> nodes = replicatedUnstableDataNodes(cctx);
+
+        if (F.isEmpty(nodes))
+            return null; // Retry.
 
         if (!F.isEmpty(extraSpaces)) {
             for (String extraSpace : extraSpaces) {
@@ -575,7 +587,12 @@ public class GridReduceQueryExecutor {
                     throw new CacheException("Queries running on replicated cache should not contain JOINs " +
                         "with partitioned tables.");
 
-                nodes.retainAll(owningReplicatedDataNodes(extraCctx));
+                Set<ClusterNode> extraOwners = replicatedUnstableDataNodes(extraCctx);
+
+                if (F.isEmpty(extraOwners))
+                    return null; // Retry.
+
+                nodes.retainAll(extraOwners);
 
                 if (nodes.isEmpty())
                     return null; // Retry.
@@ -586,34 +603,43 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * @param space Cache name.
+     * @param topVer Topology version.
+     * @return Collection of data nodes.
+     */
+    private Collection<ClusterNode> dataNodes(String space, AffinityTopologyVersion topVer) {
+        Collection<ClusterNode> res = ctx.discovery().cacheAffinityNodes(space, topVer);
+
+        return res != null ? res : Collections.<ClusterNode>emptySet();
+    }
+
+    /**
      * Collects all the nodes owning all the partitions for the given replicated cache.
      *
      * @param cctx Cache context.
-     * @return Owning nodes.
+     * @return Owning nodes or {@code null} if we can't find owners for some partitions.
      */
-    private Set<ClusterNode> owningReplicatedDataNodes(GridCacheContext<?,?> cctx) {
+    private Set<ClusterNode> replicatedUnstableDataNodes(GridCacheContext<?,?> cctx) {
         assert cctx.isReplicated() : cctx.name() + " must be replicated";
 
         String space = cctx.name();
 
-        Set<ClusterNode> dataNodes = new HashSet<>(ctx.discovery().cacheAffinityNodes(space, NONE));
+        Set<ClusterNode> dataNodes = new HashSet<>(dataNodes(space, NONE));
 
         if (dataNodes.isEmpty())
             throw new CacheException("No data nodes found for cache '" + space + "'");
 
         // Find all the nodes owning all the partitions for replicated cache.
-        for (int p = 0, extraParts = cctx.affinity().partitions(); p < extraParts; p++) {
+        for (int p = 0, parts = cctx.affinity().partitions(); p < parts; p++) {
             List<ClusterNode> owners = cctx.topology().owners(p);
 
-            if (owners.isEmpty())
-                throw new CacheException("No data nodes found for cache '" + space +
-                    "' for partition " + p);
+            if (F.isEmpty(owners))
+                return null; // Retry.
 
             dataNodes.retainAll(owners);
 
             if (dataNodes.isEmpty())
-                throw new CacheException("No data nodes found for cache '" + space +
-                    "' owning all the partitions.");
+                return null; // Retry.
         }
 
         return dataNodes;
@@ -627,7 +653,8 @@ public class GridReduceQueryExecutor {
      * @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> partitionLocations(final GridCacheContext<?,?> cctx, List<String> extraSpaces) {
+    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();
@@ -653,8 +680,12 @@ public class GridReduceQueryExecutor {
         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(owners)) {
+                if (!F.isEmpty(dataNodes(cctx.name(), NONE)))
+                    return null; // Retry.
+
                 throw new CacheException("No data nodes found for cache '" + cctx.name() + "' for partition " + p);
+            }
 
             partLocs[p] = new HashSet<>(owners);
         }
@@ -671,9 +702,13 @@ public class GridReduceQueryExecutor {
                 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(owners)) {
+                        if (!F.isEmpty(dataNodes(extraSpace, NONE)))
+                            return null; // Retry.
+
                         throw new CacheException("No data nodes found for cache '" + extraSpace +
                             "' for partition " + p);
+                    }
 
                     if (partLocs[p] == null)
                         partLocs[p] = new HashSet<>(owners);
@@ -693,7 +728,10 @@ public class GridReduceQueryExecutor {
                 if (!extraCctx.isReplicated())
                     continue;
 
-                Set<ClusterNode> dataNodes = owningReplicatedDataNodes(extraCctx);
+                Set<ClusterNode> dataNodes = replicatedUnstableDataNodes(extraCctx);
+
+                if (F.isEmpty(dataNodes))
+                    return null; // Retry.
 
                 for (Set<ClusterNode> partLoc : partLocs) {
                     partLoc.retainAll(dataNodes);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/94060c9e/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
index 23f44c0..3f23005 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheClientQueryReplicatedNodeRestartSelfTest.java
@@ -64,6 +64,9 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
         };
 
     /** */
+    private static final List<List<?>> FAKE = new LinkedList<>();
+
+    /** */
     private static final int GRID_CNT = 5;
 
     /** */
@@ -191,7 +194,7 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
     public void testRestarts() throws Exception {
         int duration = 90 * 1000;
         int qryThreadNum = 5;
-        int restartThreadsNum = 2; // 2 of 4 data nodes
+        int restartThreadsNum = 3; // 3 of 4 data nodes
         final int nodeLifeTime = 2 * 1000;
         final int logFreq = 10;
 
@@ -212,13 +215,32 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
         final AtomicInteger qryCnt = new AtomicInteger();
         final AtomicBoolean qrysDone = new AtomicBoolean();
 
+        final List<Integer> cacheSize = new ArrayList<>(4);
+
         for (int i = 0; i < GRID_CNT - 1; i++) {
-            for (String cacheName : F.asList("co", "pr", "pe", "pu"))
-                assertClient(grid(i).cache(cacheName), false);
+            int j = 0;
+
+            for (String cacheName : F.asList("co", "pr", "pe", "pu")) {
+                IgniteCache<?,?> cache = grid(i).cache(cacheName);
+
+                assertClient(cache, false);
+
+                if (i == 0)
+                    cacheSize.add(cache.size());
+                else
+                    assertEquals(cacheSize.get(j++).intValue(), cache.size());
+            }
         }
 
-        for (String cacheName : F.asList("co", "pr", "pe", "pu"))
-            assertClient(grid(GRID_CNT - 1).cache(cacheName), true);
+        int j = 0;
+
+        for (String cacheName : F.asList("co", "pr", "pe", "pu")) {
+            IgniteCache<?,?> cache = grid(GRID_CNT - 1).cache(cacheName);
+
+            assertClient(cache, true);
+
+            assertEquals(cacheSize.get(j++).intValue(), cache.size());
+        }
 
         final IgniteCache<?,?> clientCache = grid(GRID_CNT - 1).cache("pu");
 
@@ -234,8 +256,10 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
                     if (smallPageSize)
                         qry.setPageSize(3);
 
+                    List<List<?>> res;
+
                     try {
-                        assertEquals(pRes, clientCache.query(qry).getAll());
+                        res = clientCache.query(qry).getAll();
                     }
                     catch (CacheException e) {
                         assertTrue("On large page size must retry.", smallPageSize);
@@ -259,6 +283,20 @@ public class IgniteCacheClientQueryReplicatedNodeRestartSelfTest extends GridCom
 
                             fail("Must fail inside of GridResultPage.fetchNextPage or subclass.");
                         }
+
+                        res = FAKE;
+                    }
+
+                    if (res != FAKE && !res.equals(pRes)) {
+                        int j = 0;
+
+                        // Check for data loss.
+                        for (String cacheName : F.asList("co", "pr", "pe", "pu")) {
+                            assertEquals(cacheName, cacheSize.get(j++).intValue(),
+                                grid(GRID_CNT - 1).cache(cacheName).size());
+                        }
+
+                        assertEquals(pRes, res); // Fail with nice message.
                     }
 
                     int c = qryCnt.incrementAndGet();


[19/50] incubator-ignite git commit: ignite-484-1 - replicated cache group reservation fix + drop reservations group for dead nodes

Posted by vk...@apache.org.
ignite-484-1 - replicated cache group reservation fix + drop reservations group for dead nodes


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/1fe215e2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/1fe215e2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/1fe215e2

Branch: refs/heads/ignite-1026
Commit: 1fe215e2cc83954f25cc7c2f0974dcf312694eb8
Parents: 68c35e7
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 15:57:24 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 15:57:24 2015 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/GridMapQueryExecutor.java  | 22 ++++++++++++++++----
 1 file changed, 18 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fe215e2/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 42f01cb..aaf64ee 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -123,6 +123,18 @@ public class GridMapQueryExecutor {
             }
         }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
 
+        // Drop group reservations for dead caches.
+        ctx.event().addLocalEventListener(new GridLocalEventListener() {
+            @Override public void onEvent(Event evt) {
+                String cacheName = ((CacheEvent)evt).cacheName();
+
+                for (T2<String,AffinityTopologyVersion> grpKey : reservations.keySet()) {
+                    if (F.eq(grpKey.get1(), cacheName))
+                        reservations.remove(grpKey);
+                }
+            }
+        }, EventType.EVT_CACHE_STOPPED);
+
         ctx.io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() {
             @Override public void onMessage(UUID nodeId, Object msg) {
                 if (!busyLock.enterBusy())
@@ -244,7 +256,9 @@ public class GridMapQueryExecutor {
             if (cctx.isLocal())
                 continue;
 
-            final T2<String,AffinityTopologyVersion> grpKey = new T2<>(cctx.name(), topVer);
+            // For replicated cache topology version does not make sense.
+            final T2<String,AffinityTopologyVersion> grpKey =
+                new T2<>(cctx.name(), cctx.isReplicated() ? null : topVer);
 
             GridReservable r = reservations.get(grpKey);
 
@@ -265,10 +279,10 @@ public class GridMapQueryExecutor {
                             // We don't need to reserve partitions because they will not be evicted in replicated caches.
                             if (part == null || part.state() != OWNING)
                                 return false;
-
-                            // Mark that we checked this replicated cache.
-                            reservations.putIfAbsent(grpKey, ReplicatedReservation.INSTANCE);
                         }
+
+                        // Mark that we checked this replicated cache.
+                        reservations.putIfAbsent(grpKey, ReplicatedReservation.INSTANCE);
                     }
                 }
                 else { // Reserve primary partitions for partitioned cache (if no explicit given).


[15/50] incubator-ignite git commit: ignite-484-1 - minor

Posted by vk...@apache.org.
ignite-484-1 - minor


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/c1fc7b5d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/c1fc7b5d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/c1fc7b5d

Branch: refs/heads/ignite-1026
Commit: c1fc7b5d6d7deb895bdf13d3b7e199161d494277
Parents: ca253a8
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 13:44:36 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 13:44:36 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java  |  8 +++-----
 .../dht/GridDhtPartitionsReservation.java       | 20 ++++++++++++++------
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |  8 ++++++--
 3 files changed, 23 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1fc7b5d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 5938fc8..1392f5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -505,14 +505,12 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
      * @return {@code true} If there is a group reservation.
      */
     private boolean groupReserved() {
-        boolean reserved = false;
-
         for (GridDhtPartitionsReservation reservation : reservations) {
-            if (!reservation.canEvict())
-                reserved = true; // Calling all the reservations to allow them unregister themselves.
+            if (!reservation.invalidate())
+                return true; // Failed to invalidate reservation -> we are reserved.
         }
 
-        return reserved;
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1fc7b5d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
index a32946a..aced999 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsReservation.java
@@ -34,6 +34,13 @@ public class GridDhtPartitionsReservation implements GridReservable {
     private static final GridDhtLocalPartition[] EMPTY = {};
 
     /** */
+    private static final CI1<GridDhtPartitionsReservation> NO_OP = new CI1<GridDhtPartitionsReservation>() {
+        @Override public void apply(GridDhtPartitionsReservation gridDhtPartitionsReservation) {
+            throw new IllegalStateException();
+        }
+    };
+
+    /** */
     private final Object appKey;
 
     /** */
@@ -154,7 +161,7 @@ public class GridDhtPartitionsReservation implements GridReservable {
         for (;;) {
             int r = reservations.get();
 
-            if (r == -1) // Invalidated by successful canEvict call.
+            if (r == -1) // Invalidated.
                 return false;
 
             assert r >= 0 : r;
@@ -224,20 +231,21 @@ public class GridDhtPartitionsReservation implements GridReservable {
         // Unpublish.
         CI1<GridDhtPartitionsReservation> u = unpublish.get();
 
-        if (u != null && unpublish.compareAndSet(u, null))
+        if (u != null && u != NO_OP && unpublish.compareAndSet(u, NO_OP))
             u.apply(this);
     }
 
     /**
      * Must be checked in {@link GridDhtLocalPartition#tryEvict(boolean)}.
-     * If returns {@code true} then probably partition will be evicted (or at least cleared),
-     * so this reservation object becomes invalid and must be dropped from the partition.
+     * If returns {@code true} this reservation object becomes invalid and partitions
+     * can be evicted or at least cleared.
      * Also this means that after returning {@code true} here method {@link #reserve()} can not
      * return {@code true} anymore.
      *
-     * @return {@code true} If this reservation is NOT reserved and partition CAN be evicted.
+     * @return {@code true} If this reservation was successfully invalidated because it was not
+     *          reserved and partitions can be evicted.
      */
-    public boolean canEvict() {
+    public boolean invalidate() {
         assert parts.get() != null : "all parts must be reserved before registration";
 
         int r = reservations.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1fc7b5d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
index d440b13..527dfea 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest2.java
@@ -177,7 +177,7 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testRestarts() throws Exception {
-        int duration = 150 * 1000;
+        int duration = 90 * 1000;
         int qryThreadNum = 4;
         int restartThreadsNum = 2; // 4 + 2 = 6 nodes
         final int nodeLifeTime = 2 * 1000;
@@ -197,6 +197,9 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
 
         final List<List<?>> rRes = grid(0).cache("co").query(new SqlFieldsQuery(REPLICATED_QRY)).getAll();
 
+        assertFalse(pRes.isEmpty());
+        assertFalse(rRes.isEmpty());
+
         final AtomicInteger qryCnt = new AtomicInteger();
 
         final AtomicBoolean qrysDone = new AtomicBoolean();
@@ -235,7 +238,8 @@ public class IgniteCacheQueryNodeRestartSelfTest2 extends GridCommonAbstractTest
                                 if (!(th instanceof CacheException))
                                     continue;
 
-                                if (th.getMessage().startsWith("Failed to fetch data from node:")) {
+                                if (th.getMessage() != null &&
+                                    th.getMessage().startsWith("Failed to fetch data from node:")) {
                                     failedOnRemoteFetch = true;
 
                                     break;


[48/50] incubator-ignite git commit: Merge branches 'ignite-1026' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-1026

Posted by vk...@apache.org.
Merge branches 'ignite-1026' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-1026


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/d6dd707b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d6dd707b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d6dd707b

Branch: refs/heads/ignite-1026
Commit: d6dd707ba0806d3a01dc7acc09d6948226c0177d
Parents: eef2b37 ad0a026
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 12:37:21 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 12:37:21 2015 -0700

----------------------------------------------------------------------
 .../s3/S3CheckpointManagerSelfTest.java         |   2 +-
 .../checkpoint/s3/S3CheckpointSpiSelfTest.java  |   4 +-
 .../s3/S3CheckpointSpiStartStopSelfTest.java    |   2 +-
 .../s3/S3SessionCheckpointSelfTest.java         |   2 +-
 .../s3/TcpDiscoveryS3IpFinderSelfTest.java      |   2 +-
 .../affinity/AffinityTopologyVersion.java       |   7 -
 .../processors/cache/GridCacheUtils.java        |   9 +
 .../processors/cache/IgniteCacheProxy.java      |   5 +
 .../distributed/dht/GridDhtLocalPartition.java  |  56 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   4 +-
 .../dht/GridDhtPartitionsReservation.java       | 292 +++++++++
 .../cache/distributed/dht/GridReservable.java   |  35 +
 .../dht/preloader/GridDhtPartitionMap.java      |  26 +-
 .../cache/query/GridCacheQueryManager.java      |  33 -
 .../cache/query/GridCacheTwoStepQuery.java      |  22 +-
 .../processors/query/GridQueryIndexing.java     |  14 +-
 .../processors/query/GridQueryProcessor.java    |  21 +-
 .../messages/GridQueryNextPageResponse.java     |  34 +-
 .../h2/twostep/messages/GridQueryRequest.java   | 111 +++-
 .../apache/ignite/internal/util/GridDebug.java  |  19 +
 .../communication/tcp/TcpCommunicationSpi.java  |  42 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  35 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  42 +-
 .../tcp/internal/TcpDiscoveryNode.java          |  18 +
 .../apache/ignite/internal/GridSelfTest.java    |  20 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  24 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |  30 +-
 .../IgniteCacheAtomicStopBusySelfTest.java      |   8 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  19 +-
 ...gniteCacheTransactionalStopBusySelfTest.java |   8 +-
 .../DataStreamerMultiThreadedSelfTest.java      |   3 +
 .../junits/GridTestKernalContext.java           |   2 +-
 .../junits/common/GridCommonAbstractTest.java   |   8 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  79 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  49 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  | 332 +++++++---
 .../query/h2/twostep/GridMergeIndex.java        |  17 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   7 +-
 .../h2/twostep/GridReduceQueryExecutor.java     | 650 ++++++++++++++++---
 .../query/h2/twostep/GridResultPage.java        |  21 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   3 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   1 -
 ...lientQueryReplicatedNodeRestartSelfTest.java | 419 ++++++++++++
 .../IgniteCacheQueryNodeRestartSelfTest.java    |  36 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   | 383 +++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 46 files changed, 2577 insertions(+), 381 deletions(-)
----------------------------------------------------------------------



[31/50] incubator-ignite git commit: # ignite-sprint-6 fixed test to work in offheap mode

Posted by vk...@apache.org.
# ignite-sprint-6 fixed test to work in offheap mode


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/cc20ef6a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/cc20ef6a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/cc20ef6a

Branch: refs/heads/ignite-1026
Commit: cc20ef6adce9bdeea19d37f5817c0693e380cfa5
Parents: de53189
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 10:30:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 10:30:20 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAbstractFullApiSelfTest.java | 24 +++++++++++---------
 1 file changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cc20ef6a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 128d452..151c249 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -1020,25 +1020,27 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         throws Exception {
         IgniteCache<String, Integer> cache = jcache();
 
+        final String key = primaryKeysForCache(cache, 1).get(0);
+
         Transaction tx = txEnabled() ? ignite(0).transactions().txStart(concurrency, READ_COMMITTED) : null;
 
         try {
             if (startVal)
-                cache.put("key", 2);
+                cache.put(key, 2);
             else
-                assertEquals(null, cache.get("key"));
+                assertEquals(null, cache.get(key));
 
-            Integer expectedRes = startVal ? 2 : null;
+            Integer expRes = startVal ? 2 : null;
 
-            assertEquals(String.valueOf(expectedRes), cache.invoke("key", INCR_PROCESSOR));
+            assertEquals(String.valueOf(expRes), cache.invoke(key, INCR_PROCESSOR));
 
-            expectedRes = startVal ? 3 : 1;
+            expRes = startVal ? 3 : 1;
 
-            assertEquals(String.valueOf(expectedRes), cache.invoke("key", INCR_PROCESSOR));
+            assertEquals(String.valueOf(expRes), cache.invoke(key, INCR_PROCESSOR));
 
-            expectedRes++;
+            expRes++;
 
-            assertEquals(String.valueOf(expectedRes), cache.invoke("key", INCR_PROCESSOR));
+            assertEquals(String.valueOf(expRes), cache.invoke(key, INCR_PROCESSOR));
 
             if (tx != null)
                 tx.commit();
@@ -1050,11 +1052,11 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         Integer exp = (startVal ? 2 : 0) + 3;
 
-        assertEquals(exp, cache.get("key"));
+        assertEquals(exp, cache.get(key));
 
         for (int i = 0; i < gridCount(); i++) {
-            if (ignite(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), "key"))
-                assertEquals(exp, peek(jcache(i), "key"));
+            if (ignite(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key))
+                assertEquals(exp, peek(jcache(i), key));
         }
     }
 


[21/50] incubator-ignite git commit: # i-484-1 review

Posted by vk...@apache.org.
# i-484-1 review


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/42982382
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/42982382
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/42982382

Branch: refs/heads/ignite-1026
Commit: 42982382ee92bc12073beac34d72590bf905ed3e
Parents: 10febf2
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jun 17 18:12:49 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jun 17 18:12:49 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/twostep/GridReduceQueryExecutor.java      | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/42982382/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index c570d24..6c407d9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -438,7 +438,8 @@ public class GridReduceQueryExecutor {
             assert !nodes.isEmpty();
 
             if (cctx.isReplicated() || qry.explain()) {
-                assert qry.explain() || !nodes.contains(ctx.cluster().get().localNode()) : "We must be on a client node.";
+                assert qry.explain() || !nodes.contains(ctx.cluster().get().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));


[07/50] incubator-ignite git commit: ignite-484-1 - minor

Posted by vk...@apache.org.
ignite-484-1 - minor


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/2b7dc3bb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/2b7dc3bb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/2b7dc3bb

Branch: refs/heads/ignite-1026
Commit: 2b7dc3bbe0a65fcdf8e7e50da0e494b07e15ffc7
Parents: 0a5e97c
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 12 10:36:43 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 12 10:36:43 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/query/ScanQuery.java    | 20 --------------------
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |  1 -
 2 files changed, 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2b7dc3bb/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
index 90000e8..11a8c84 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/ScanQuery.java
@@ -119,26 +119,6 @@ public final class ScanQuery<K, V> extends Query<Cache.Entry<K, V>> {
         return this;
     }
 
-    /**
-     * Gets partition number over which this query should iterate. Will return {@code null} if partition was not
-     * set. In this case query will iterate over all partitions in the cache.
-     *
-     * @return Partition number or {@code null}.
-     */
-    @Nullable public Integer getPartition() {
-        return part;
-    }
-
-    /**
-     * Sets partition number over which this query should iterate. If {@code null}, query will iterate over
-     * all partitions in the cache. Must be in the range [0, N) where N is partition number in the cache.
-     *
-     * @param part Partition number over which this query should iterate.
-     */
-    public void setPartition(@Nullable Integer part) {
-        this.part = part;
-    }
-
     /** {@inheritDoc} */
     @Override public ScanQuery<K, V> setPageSize(int pageSize) {
         return (ScanQuery<K, V>)super.setPageSize(pageSize);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2b7dc3bb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
index 9fcd947..3e50443 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.processors.query.h2.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;


[03/50] incubator-ignite git commit: ignite-484-1 - fix

Posted by vk...@apache.org.
ignite-484-1 - fix


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/1bd7f051
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/1bd7f051
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/1bd7f051

Branch: refs/heads/ignite-1026
Commit: 1bd7f05104346618492b4fadeb8767f078f1c598
Parents: d340fe7
Author: S.Vladykin <sv...@gridgain.com>
Authored: Thu Jun 11 15:10:10 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Thu Jun 11 15:10:10 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/distributed/dht/GridDhtLocalPartition.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1bd7f051/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 13c1de1..3670b8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -528,7 +528,7 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
         if (state.getReference() == RENTING && state.getStamp() == 0 && !groupReserved())
             clearAll();
 
-        if (map.isEmpty() && state.compareAndSet(RENTING, EVICTED, 0, 0) && !groupReserved()) {
+        if (map.isEmpty() && !groupReserved() && state.compareAndSet(RENTING, EVICTED, 0, 0)) {
             if (log.isDebugEnabled())
                 log.debug("Evicted partition: " + this);
 


[45/50] incubator-ignite git commit: Fixed test.

Posted by vk...@apache.org.
Fixed test.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/4f3292d5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4f3292d5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4f3292d5

Branch: refs/heads/ignite-1026
Commit: 4f3292d5d8b29c04fcc1bf1fa9b88e5f19ceddfe
Parents: d874b00
Author: nikolay tikhonov <nt...@gridgain.com>
Authored: Thu Jun 18 15:19:03 2015 +0300
Committer: nikolay tikhonov <nt...@gridgain.com>
Committed: Thu Jun 18 15:19:03 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/GridSelfTest.java    | 20 ++++++++++++--------
 1 file changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4f3292d5/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
index eccae34..2de04b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
@@ -112,7 +112,9 @@ public class GridSelfTest extends GridProjectionAbstractTest {
      */
     @SuppressWarnings({"TooBroadScope"})
     public void testAsyncListen() throws Exception {
-        final String msg = "HELLO!";
+        final String hello = "HELLO!";
+
+        final String bye = "BYE!";
 
         final Ignite g = grid(0);
 
@@ -120,10 +122,12 @@ public class GridSelfTest extends GridProjectionAbstractTest {
 
         g.message().remoteListen(null, new MessagingListenActor<String>() {
             @Override protected void receive(UUID nodeId, String rcvMsg) throws Throwable {
-                assertEquals(locNodeId, nodeId);
-                assertEquals(msg, rcvMsg);
+                if (hello.equals(rcvMsg)) {
+                    assertEquals(locNodeId, nodeId);
+                    assertEquals(hello, rcvMsg);
 
-                stop(rcvMsg);
+                    stop(bye);
+                }
             }
         });
 
@@ -131,22 +135,22 @@ public class GridSelfTest extends GridProjectionAbstractTest {
 
         g.message().localListen(null, new P2<UUID, String>() {
             @Override public boolean apply(UUID nodeId, String msg) {
-                if (!locNodeId.equals(nodeId))
+                if (msg.equals(bye))
                     cnt.incrementAndGet();
 
                 return true;
             }
         });
 
-        g.message().send(null, msg);
+        g.message().send(null, hello);
 
         GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
-                return cnt.get() == g.cluster().forRemotes().nodes().size();
+                return cnt.get() == g.cluster().nodes().size();
             }
         }, 5000);
 
-        assertEquals(cnt.get(), g.cluster().forRemotes().nodes().size());
+        assertEquals(cnt.get(), g.cluster().nodes().size());
     }
 
     /**


[34/50] incubator-ignite git commit: # ignite-sprint-6 more info in test assert

Posted by vk...@apache.org.
# ignite-sprint-6 more info in test assert


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/d874b00d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d874b00d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d874b00d

Branch: refs/heads/ignite-1026
Commit: d874b00dcb1687339adc34b6212e8cbc5b9ffe16
Parents: 6c683c2
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 11:07:04 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 11:07:04 2015 +0300

----------------------------------------------------------------------
 .../IgniteCacheAbstractStopBusySelfTest.java    | 30 ++++++++++++--------
 .../IgniteCacheAtomicStopBusySelfTest.java      |  8 +++---
 ...gniteCacheTransactionalStopBusySelfTest.java |  8 +++---
 3 files changed, 26 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d874b00d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
index 10b14cc..074f6ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
@@ -36,9 +36,13 @@ import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
+import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
 /**
  *
  */
@@ -53,7 +57,7 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
     public static final String CACHE_NAME = "StopTest";
 
     /** */
-    public static final TcpDiscoveryIpFinder finder = new TcpDiscoveryVmIpFinder(true);
+    public final TcpDiscoveryIpFinder finder = new TcpDiscoveryVmIpFinder(true);
 
     /** */
     private AtomicBoolean suspended = new AtomicBoolean(false);
@@ -62,7 +66,7 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
     private CountDownLatch blocked;
 
     /** */
-    protected AtomicReference<Class> bannedMessage = new AtomicReference<>();
+    protected AtomicReference<Class> bannedMsg = new AtomicReference<>();
 
     /**
      * @return Cache mode.
@@ -93,9 +97,9 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
         if (gridName.endsWith(String.valueOf(CLN_GRD)))
             cfg.setClientMode(true);
 
-        cacheCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
+        cacheCfg.setRebalanceMode(SYNC);
 
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 
         cacheCfg.setBackups(1);
 
@@ -125,14 +129,14 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
             TimeUnit.MILLISECONDS.sleep(100L);
         }
 
-        assert clientNode().cluster().nodes().size() == 2;
+        assertEquals(2, clientNode().cluster().nodes().size());
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         suspended.set(false);
 
-        bannedMessage.set(null);
+        bannedMsg.set(null);
 
         afterTestsStopped();
 
@@ -140,7 +144,9 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
 
         stopGrid(CLN_GRD);
 
-        assert G.allGrids().isEmpty();
+        List<Ignite> nodes = G.allGrids();
+
+        assertTrue("Unexpected nodes: " + nodes, nodes.isEmpty());
     }
 
     /**
@@ -203,12 +209,11 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
      * @throws Exception If failed.
      */
     public void testGet() throws Exception {
-        bannedMessage.set(GridNearGetRequest.class);
+        bannedMsg.set(GridNearGetRequest.class);
 
         executeTest(new Callable<Integer>() {
             /** {@inheritDoc} */
-            @Override
-            public Integer call() throws Exception {
+            @Override public Integer call() throws Exception {
                 info("Start operation.");
 
                 Integer put = (Integer) clientCache().get(1);
@@ -318,6 +323,7 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
      * @return Cache configuration.
      * @throws Exception In case of error.
      */
+    @SuppressWarnings("unchecked")
     private CacheConfiguration cacheConfiguration(@Nullable String cacheName) throws Exception {
         CacheConfiguration cfg = defaultCacheConfiguration();
 
@@ -339,10 +345,10 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
         /** {@inheritDoc} */
         @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
             if (suspended.get()) {
-                assert bannedMessage.get() != null;
+                assert bannedMsg.get() != null;
 
                 if (msg instanceof GridIoMessage
-                    && ((GridIoMessage)msg).message().getClass().equals(bannedMessage.get())) {
+                    && ((GridIoMessage)msg).message().getClass().equals(bannedMsg.get())) {
                     blocked.countDown();
 
                     return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d874b00d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java
index bae1601..ed15724 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java
@@ -31,28 +31,28 @@ public class IgniteCacheAtomicStopBusySelfTest extends IgniteCacheAbstractStopBu
 
     /** {@inheritDoc} */
     @Override public void testPut() throws Exception {
-        bannedMessage.set(GridNearAtomicUpdateRequest.class);
+        bannedMsg.set(GridNearAtomicUpdateRequest.class);
 
         super.testPut();
     }
 
     /** {@inheritDoc} */
     @Override public void testPutBatch() throws Exception {
-        bannedMessage.set(GridNearAtomicUpdateRequest.class);
+        bannedMsg.set(GridNearAtomicUpdateRequest.class);
 
         super.testPut();
     }
 
     /** {@inheritDoc} */
     @Override public void testPutAsync() throws Exception {
-        bannedMessage.set(GridNearAtomicUpdateRequest.class);
+        bannedMsg.set(GridNearAtomicUpdateRequest.class);
 
         super.testPut();
     }
 
     /** {@inheritDoc} */
     @Override public void testRemove() throws Exception {
-        bannedMessage.set(GridNearAtomicUpdateRequest.class);
+        bannedMsg.set(GridNearAtomicUpdateRequest.class);
 
         super.testPut();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d874b00d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java
index fe1b266..e87a772 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java
@@ -30,28 +30,28 @@ public class IgniteCacheTransactionalStopBusySelfTest extends IgniteCacheAbstrac
 
     /** {@inheritDoc} */
     @Override public void testPut() throws Exception {
-        bannedMessage.set(GridNearTxPrepareRequest.class);
+        bannedMsg.set(GridNearTxPrepareRequest.class);
 
         super.testPut();
     }
 
     /** {@inheritDoc} */
     @Override public void testPutBatch() throws Exception {
-        bannedMessage.set(GridNearTxPrepareRequest.class);
+        bannedMsg.set(GridNearTxPrepareRequest.class);
 
         super.testPut();
     }
 
     /** {@inheritDoc} */
     @Override public void testPutAsync() throws Exception {
-        bannedMessage.set(GridNearTxPrepareRequest.class);
+        bannedMsg.set(GridNearTxPrepareRequest.class);
 
         super.testPut();
     }
 
     /** {@inheritDoc} */
     @Override public void testRemove() throws Exception {
-        bannedMessage.set(GridNearTxPrepareRequest.class);
+        bannedMsg.set(GridNearTxPrepareRequest.class);
 
         super.testPut();
     }


[36/50] incubator-ignite git commit: # minor

Posted by vk...@apache.org.
# minor


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/08d134dd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/08d134dd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/08d134dd

Branch: refs/heads/ignite-1026
Commit: 08d134dd43f143ee8d01f6394de27426c64a96ea
Parents: d874b00
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jun 18 13:46:52 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jun 18 13:46:52 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheUtils.java    | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/08d134dd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 3bd2a45..8c26046 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -391,6 +391,15 @@ public class GridCacheUtils {
     }
 
     /**
+     * Gets public cache name substituting null name by {@code 'default'}.
+     *
+     * @return Public cache name substituting null name by {@code 'default'}.
+     */
+    public static String namexx(@Nullable String name) {
+        return name == null ? "default" : name;
+    }
+
+    /**
      * @return Partition to state transformer.
      */
     @SuppressWarnings({"unchecked"})


[33/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by vk...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/6c683c28
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/6c683c28
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/6c683c28

Branch: refs/heads/ignite-1026
Commit: 6c683c28f5bde0cf22ac7482bc381c861a2c6a62
Parents: cc20ef6 45939ac
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 10:36:28 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 10:36:28 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java    | 2 +-
 .../apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java | 4 ++--
 .../spi/checkpoint/s3/S3CheckpointSpiStartStopSelfTest.java      | 2 +-
 .../ignite/spi/checkpoint/s3/S3SessionCheckpointSelfTest.java    | 2 +-
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java          | 2 +-
 5 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[39/50] incubator-ignite git commit: # ignite-sprint-6 disabled shmem for test

Posted by vk...@apache.org.
# ignite-sprint-6 disabled shmem for test


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/ca81476d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/ca81476d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/ca81476d

Branch: refs/heads/ignite-1026
Commit: ca81476d1ab66773815c7e3f91d61ea5b0651bb9
Parents: b23ea74
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 14:21:34 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 14:21:34 2015 +0300

----------------------------------------------------------------------
 .../datastreamer/DataStreamerMultiThreadedSelfTest.java           | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ca81476d/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
index e0092d4..c37219a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.communication.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -48,6 +49,8 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
         discoSpi.setIpFinder(IP_FINDER);
 


[32/50] incubator-ignite git commit: #Fixed aws tests. (Changed bucket name)

Posted by vk...@apache.org.
#Fixed aws tests. (Changed bucket name)


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/45939ace
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/45939ace
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/45939ace

Branch: refs/heads/ignite-1026
Commit: 45939ace51cd081ac1c755ca0b6076a22f00cfb5
Parents: de53189
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu Jun 18 10:30:33 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Jun 18 10:31:31 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java    | 2 +-
 .../apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java | 4 ++--
 .../spi/checkpoint/s3/S3CheckpointSpiStartStopSelfTest.java      | 2 +-
 .../ignite/spi/checkpoint/s3/S3SessionCheckpointSelfTest.java    | 2 +-
 .../tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java          | 2 +-
 5 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45939ace/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java
index a354e67..b01ba81 100644
--- a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java
+++ b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointManagerSelfTest.java
@@ -39,7 +39,7 @@ public class S3CheckpointManagerSelfTest extends GridCheckpointManagerAbstractSe
 
         spi.setAwsCredentials(cred);
 
-        spi.setBucketNameSuffix("test-bucket");
+        spi.setBucketNameSuffix("unit-test-bucket");
 
         cfg.setCheckpointSpi(spi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45939ace/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
index 5cf0f2e..5894678 100644
--- a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
+++ b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiSelfTest.java
@@ -48,7 +48,7 @@ public class S3CheckpointSpiSelfTest extends GridSpiAbstractTest<S3CheckpointSpi
 
         spi.setAwsCredentials(cred);
 
-        spi.setBucketNameSuffix("test-bucket");
+        spi.setBucketNameSuffix("unit-test-bucket");
 
         super.spiConfigure(spi);
     }
@@ -62,7 +62,7 @@ public class S3CheckpointSpiSelfTest extends GridSpiAbstractTest<S3CheckpointSpi
 
         AmazonS3 s3 = new AmazonS3Client(cred);
 
-        String bucketName = S3CheckpointSpi.BUCKET_NAME_PREFIX + "test-bucket";
+        String bucketName = S3CheckpointSpi.BUCKET_NAME_PREFIX + "unit-test-bucket";
 
         try {
             ObjectListing list = s3.listObjects(bucketName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45939ace/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiStartStopSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiStartStopSelfTest.java
index 7de2124..34f25ad 100644
--- a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiStartStopSelfTest.java
+++ b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3CheckpointSpiStartStopSelfTest.java
@@ -34,7 +34,7 @@ public class S3CheckpointSpiStartStopSelfTest extends GridSpiStartStopAbstractTe
 
         spi.setAwsCredentials(cred);
 
-        spi.setBucketNameSuffix("test-bucket");
+        spi.setBucketNameSuffix("unit-test-bucket");
 
         super.spiConfigure(spi);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45939ace/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3SessionCheckpointSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3SessionCheckpointSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3SessionCheckpointSelfTest.java
index 56f39de..c683cb3 100644
--- a/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3SessionCheckpointSelfTest.java
+++ b/modules/aws/src/test/java/org/apache/ignite/spi/checkpoint/s3/S3SessionCheckpointSelfTest.java
@@ -39,7 +39,7 @@ public class S3SessionCheckpointSelfTest extends GridSessionCheckpointAbstractSe
 
         spi.setAwsCredentials(cred);
 
-        spi.setBucketNameSuffix("test-bucket");
+        spi.setBucketNameSuffix("unit-test-bucket");
 
         cfg.setCheckpointSpi(spi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45939ace/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java
index 709145b..8a7b528 100644
--- a/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java
+++ b/modules/aws/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/s3/TcpDiscoveryS3IpFinderSelfTest.java
@@ -51,7 +51,7 @@ public class TcpDiscoveryS3IpFinderSelfTest
             IgniteS3TestSuite.getSecretKey()));
 
         // Bucket name should be unique for the host to parallel test run on one bucket.
-        finder.setBucketName("ip-finder-test-bucket-" + InetAddress.getLocalHost().getAddress()[3]);
+        finder.setBucketName("ip-finder-unit-test-bucket-" + InetAddress.getLocalHost().getAddress()[3]);
 
         for (int i = 0; i < 5; i++) {
             Collection<InetSocketAddress> addrs = finder.getRegisteredAddresses();


[49/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1026' into ignite-1026

Posted by vk...@apache.org.
Merge remote-tracking branch 'origin/ignite-1026' into ignite-1026


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/2f089f1b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/2f089f1b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/2f089f1b

Branch: refs/heads/ignite-1026
Commit: 2f089f1bda24b0e54735e5f8dccfc372647e1c54
Parents: d6dd707 bb73b66
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 16:03:40 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 16:03:40 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 16 +++++-----
 .../processors/cache/GridCacheProcessor.java    | 31 ++++++++++++--------
 .../datastructures/DataStructuresProcessor.java |  2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  4 +--
 4 files changed, 30 insertions(+), 23 deletions(-)
----------------------------------------------------------------------



[18/50] incubator-ignite git commit: ignite-484-1 - get rid of owning future

Posted by vk...@apache.org.
ignite-484-1 - get rid of owning future


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/68c35e73
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/68c35e73
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/68c35e73

Branch: refs/heads/ignite-1026
Commit: 68c35e734b3754460c4ad807e154ea94066600a0
Parents: 642f1c7
Author: S.Vladykin <sv...@gridgain.com>
Authored: Wed Jun 17 15:24:13 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Wed Jun 17 15:24:13 2015 +0300

----------------------------------------------------------------------
 .../distributed/dht/GridDhtLocalPartition.java   | 19 -------------------
 1 file changed, 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/68c35e73/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 1392f5e..3a577a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -70,10 +70,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     @GridToStringExclude
     private final GridFutureAdapter<?> rent;
 
-    /** Rent future. */
-    @GridToStringExclude
-    private final GridFutureAdapter<?> own;
-
     /** Entries map. */
     private final ConcurrentMap<KeyCacheObject, GridDhtCacheEntry> map;
 
@@ -118,12 +114,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
             }
         };
 
-        own = new GridFutureAdapter<Object>() {
-            @Override public String toString() {
-                return "PartitionOwnFuture [part=" + GridDhtLocalPartition.this + ", map=" + map + ']';
-            }
-        };
-
         map = new ConcurrentHashMap8<>(cctx.config().getStartSize() /
             cctx.affinity().partitions());
 
@@ -426,8 +416,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
                 // No need to keep history any more.
                 evictHist = null;
 
-                own.onDone();
-
                 return true;
             }
         }
@@ -462,13 +450,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     }
 
     /**
-     * @return The future which will be completed when partition will have state {@link GridDhtPartitionState#OWNING}.
-     */
-    public IgniteInternalFuture<?> owningFuture() {
-        return own;
-    }
-
-    /**
      * @param updateSeq Update sequence.
      * @return Future for evict attempt.
      */