You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2017/07/20 12:31:06 UTC

[01/27] ignite git commit: Corrected fix for REST processor wrt authentication

Repository: ignite
Updated Branches:
  refs/heads/ignite-5658 647be5693 -> 9936dd850


Corrected fix for REST processor wrt authentication


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

Branch: refs/heads/ignite-5658
Commit: 707c454ad9c3b4132e2d0a20d15dc1eb2ed295b0
Parents: 478d3b5
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Jul 12 10:53:46 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Jul 12 10:53:46 2017 +0300

----------------------------------------------------------------------
 .../processors/rest/GridRestProcessor.java      | 45 ++++++--------------
 1 file changed, 12 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/707c454a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index fd5583d..9842883 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -243,23 +243,21 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
             SecurityContext secCtx0 = ses.secCtx;
 
-            if (ctx.state().publicApiActiveState() || !isClusterActivateTaskRequest(req)) {
-                try {
-                    if (secCtx0 == null)
-                        ses.secCtx = secCtx0 = authenticate(req);
+            try {
+                if (secCtx0 == null)
+                    ses.secCtx = secCtx0 = authenticate(req);
 
-                    authorize(req, secCtx0);
-                }
-                catch (SecurityException e) {
-                    assert secCtx0 != null;
+                authorize(req, secCtx0);
+            }
+            catch (SecurityException e) {
+                assert secCtx0 != null;
 
-                    GridRestResponse res = new GridRestResponse(STATUS_SECURITY_CHECK_FAILED, e.getMessage());
+                GridRestResponse res = new GridRestResponse(STATUS_SECURITY_CHECK_FAILED, e.getMessage());
 
-                    return new GridFinishedFuture<>(res);
-                }
-                catch (IgniteCheckedException e) {
-                    return new GridFinishedFuture<>(new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage()));
-                }
+                return new GridFinishedFuture<>(res);
+            }
+            catch (IgniteCheckedException e) {
+                return new GridFinishedFuture<>(new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage()));
             }
         }
 
@@ -321,25 +319,6 @@ public class GridRestProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * We skip authentication for activate cluster request.
-     * It's necessary workaround to make possible cluster activation through Visor,
-     * as security checks require working caches.
-     *
-     * @param req Request.
-     */
-    private boolean isClusterActivateTaskRequest(GridRestRequest req) {
-        if (req instanceof GridRestTaskRequest) {
-            GridRestTaskRequest taskReq = (GridRestTaskRequest)req;
-
-            if (VisorGatewayTask.class.getCanonicalName().equals(taskReq.taskName()) &&
-                taskReq.params().contains(VisorChangeGridActiveStateTask.class.getCanonicalName()))
-                return true;
-        }
-
-        return false;
-    }
-
-    /**
      * @param req Request.
      * @return Not null session.
      * @throws IgniteCheckedException If failed.


[21/27] ignite git commit: ignite-5763 Race in concurrent client cache start

Posted by yz...@apache.org.
ignite-5763 Race in concurrent client cache start


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

Branch: refs/heads/ignite-5658
Commit: 644c9f39cd980e8087438519b34a79986c82d8b4
Parents: 624b451
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 17 17:06:55 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 17 17:06:55 2017 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  38 ++++-
 .../cache/IgniteDynamicCacheMultinodeTest.java  | 168 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 3 files changed, 202 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/644c9f39/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 9f5bd3f..347f6fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -192,6 +192,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
     };
 
+    /** */
+    private final Object discoEvtMux = new Object();
+
     /** Discovery event worker. */
     private final DiscoveryWorker discoWrk = new DiscoveryWorker();
 
@@ -551,6 +554,26 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 final ClusterNode node,
                 final Collection<ClusterNode> topSnapshot,
                 final Map<Long, Collection<ClusterNode>> snapshots,
+                @Nullable DiscoverySpiCustomMessage spiCustomMsg) {
+                synchronized (discoEvtMux) {
+                    onDiscovery0(type, topVer, node, topSnapshot, snapshots, spiCustomMsg);
+                }
+            }
+
+            /**
+             * @param type Event type.
+             * @param topVer Event topology version.
+             * @param node Event node.
+             * @param topSnapshot Topology snapsjot.
+             * @param snapshots Topology snapshots history.
+             * @param spiCustomMsg Custom event.
+             */
+            private void onDiscovery0(
+                final int type,
+                final long topVer,
+                final ClusterNode node,
+                final Collection<ClusterNode> topSnapshot,
+                final Map<Long, Collection<ClusterNode>> snapshots,
                 @Nullable DiscoverySpiCustomMessage spiCustomMsg
             ) {
                 DiscoveryCustomMessage customMsg = spiCustomMsg == null ? null
@@ -2062,12 +2085,15 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     public void clientCacheStartEvent(UUID reqId,
         @Nullable Map<String, DynamicCacheChangeRequest> startReqs,
         @Nullable Set<String> cachesToClose) {
-        discoWrk.addEvent(EVT_DISCOVERY_CUSTOM_EVT,
-            AffinityTopologyVersion.NONE,
-            localNode(),
-            null,
-            Collections.<ClusterNode>emptyList(),
-            new ClientCacheChangeDummyDiscoveryMessage(reqId, startReqs, cachesToClose));
+        // Prevent race when discovery message was processed, but was passed to discoWrk.
+        synchronized (discoEvtMux) {
+            discoWrk.addEvent(EVT_DISCOVERY_CUSTOM_EVT,
+                AffinityTopologyVersion.NONE,
+                localNode(),
+                null,
+                Collections.<ClusterNode>emptyList(),
+                new ClientCacheChangeDummyDiscoveryMessage(reqId, startReqs, cachesToClose));
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/644c9f39/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheMultinodeTest.java
new file mode 100644
index 0000000..d362189
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheMultinodeTest.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+
+/**
+ *
+ */
+public class IgniteDynamicCacheMultinodeTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES = 6;
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(NODES - 2);
+
+        client = true;
+
+        startGridsMultiThreaded(NODES - 2, 2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetOrCreateCache() throws Exception {
+        createCacheMultinode(TestOp.GET_OR_CREATE_CACHE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetOrCreateCaches() throws Exception {
+        createCacheMultinode(TestOp.GET_OR_CREATE_CACHES);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void createCacheMultinode(final TestOp op) throws Exception {
+        final int THREADS = NODES * 3;
+
+        for (int i = 0; i < 10; i++) {
+            log.info("Iteration: " + i);
+
+            final CyclicBarrier b = new CyclicBarrier(THREADS);
+
+            final AtomicInteger idx = new AtomicInteger();
+
+            final int iter = i;
+
+            GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    Ignite node = ignite(idx.incrementAndGet() % NODES);
+
+                    b.await();
+
+                    boolean sleep = iter % 2 == 0;
+
+                    if (sleep)
+                        Thread.sleep(ThreadLocalRandom.current().nextLong(100) + 1);
+
+                    switch (op) {
+                        case GET_OR_CREATE_CACHE:
+                            node.getOrCreateCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
+
+                            break;
+
+                        case GET_OR_CREATE_CACHES:
+                            node.getOrCreateCaches(cacheConfigurations());
+
+                            break;
+                    }
+
+                    return null;
+                }
+            }, THREADS, "start-cache");
+
+            for (String cache : ignite(0).cacheNames())
+                ignite(0).destroyCache(cache);
+        }
+    }
+
+    /**
+     * @return Cache configurations.
+     */
+    private List<CacheConfiguration> cacheConfigurations() {
+        List<CacheConfiguration> ccfgs = new ArrayList<>();
+
+        for (int i = 0; i < 10; i++) {
+            CacheConfiguration ccfg = new CacheConfiguration("cache-" + i);
+
+            ccfg.setAtomicityMode(i % 2 == 0 ? ATOMIC : TRANSACTIONAL);
+
+            ccfgs.add(ccfg);
+        }
+
+        return ccfgs;
+    }
+
+    /**
+     *
+     */
+    enum TestOp {
+        /** */
+        GET_OR_CREATE_CACHE,
+
+        /** */
+        GET_OR_CREATE_CACHES
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/644c9f39/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index e7f38be..d931ea9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -74,6 +74,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheTxPreloadNoWriteTe
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxReplicatedPeekModesTest;
 import org.apache.ignite.internal.processors.cache.IgniteClientCacheInitializationFailTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheFilterTest;
+import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheMultinodeTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartNoExchangeTimeoutTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartStopConcurrentTest;
@@ -219,6 +220,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteCacheTxPreloadNoWriteTest.class);
 
         suite.addTestSuite(IgniteDynamicCacheStartSelfTest.class);
+        suite.addTestSuite(IgniteDynamicCacheMultinodeTest.class);
         suite.addTestSuite(IgniteDynamicCacheWithConfigStartSelfTest.class);
         suite.addTestSuite(IgniteCacheDynamicStopSelfTest.class);
         suite.addTestSuite(IgniteDynamicCacheStartStopConcurrentTest.class);


[07/27] ignite git commit: More verbose logging

Posted by yz...@apache.org.
More verbose logging


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

Branch: refs/heads/ignite-5658
Commit: 3787181310597b7a6e633e745ba08209abd038a9
Parents: 0cb6ac0
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Wed Jul 12 18:28:57 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Jul 12 18:28:57 2017 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionsExchangeFuture.java       | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/37871813/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 5760f87..3b6fe91 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -1008,11 +1008,19 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      */
     private void tryToPerformLocalSnapshotOperation() {
         try {
+            long start = U.currentTimeMillis();
+
             IgniteInternalFuture fut = cctx.snapshot()
                 .tryStartLocalSnapshotOperation(discoEvt);
 
             if (fut != null)
                 fut.get();
+
+            long end = U.currentTimeMillis();
+
+            if (log.isInfoEnabled())
+                log.info("Snapshot initialization completed [topVer=" + exchangeId().topologyVersion() +
+                    ", time=" + (end - start) + "ms]");
         }
         catch (IgniteCheckedException e) {
             U.error(log, "Error while starting snapshot operation", e);


[18/27] ignite git commit: IGNITE-5649: Get meta for the specified cache via REST. - Fixes #2292.

Posted by yz...@apache.org.
IGNITE-5649: Get meta for the specified cache via REST. - Fixes #2292.

Signed-off-by: shroman <rs...@yahoo.com>


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

Branch: refs/heads/ignite-5658
Commit: 621b0ede7c75260f820ba5fe43850da871541e98
Parents: b95c261
Author: shroman <rs...@yahoo.com>
Authored: Sat Jul 15 10:11:09 2017 +0900
Committer: shroman <rs...@yahoo.com>
Committed: Sat Jul 15 10:11:09 2017 +0900

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     | 50 ++++++++----
 .../ignite/compute/ComputeJobAdapter.java       |  2 +-
 .../cache/query/GridCacheQueryManager.java      | 11 +--
 .../handlers/cache/GridCacheCommandHandler.java | 82 +++++++++++++++-----
 4 files changed, 105 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/621b0ede/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
index 97321a7..7bc26cd 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/JettyRestProcessorAbstractSelfTest.java
@@ -1040,11 +1040,8 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
      * @param metas Metadata for Ignite caches.
      * @throws Exception If failed.
      */
-    private void testMetadata(Collection<GridCacheSqlMetadata> metas, String ret) throws Exception {
-        JsonNode arr = jsonResponse(ret);
-
+    private void testMetadata(Collection<GridCacheSqlMetadata> metas, JsonNode arr) throws Exception {
         assertTrue(arr.isArray());
-        assertEquals(metas.size(), arr.size());
 
         for (JsonNode item : arr) {
             JsonNode cacheNameNode = item.get("cacheName");
@@ -1127,17 +1124,32 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
 
         Collection<GridCacheSqlMetadata> metas = cache.context().queries().sqlMetadata();
 
-        String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_METADATA.key()));
+        assertEquals(5, metas.size());
+
+        String ret = content(F.asMap("cacheName", "", "cmd", GridRestCommand.CACHE_METADATA.key()));
 
         info("Cache metadata: " + ret);
 
-        testMetadata(metas, ret);
+        JsonNode arrResponse = jsonResponse(ret);
 
-        ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_METADATA.key(), "cacheName", "person"));
+        assertEquals(5, arrResponse.size());
 
-        info("Cache metadata with cacheName parameter: " + ret);
+        testMetadata(metas, arrResponse);
+
+        Collection<GridCacheSqlMetadata> dfltCacheMeta = cache.context().queries().sqlMetadata();
+
+        ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_METADATA.key()));
+
+        info("Cache metadata: " + ret);
+
+        arrResponse = jsonResponse(ret);
 
-        testMetadata(metas, ret);
+        assertEquals(1, arrResponse.size());
+
+        testMetadata(dfltCacheMeta, arrResponse);
+
+        assertResponseContainsError(content(
+            F.asMap("cacheName", "nonExistingCacheName", "cmd", GridRestCommand.CACHE_METADATA.key())));
     }
 
     /**
@@ -1153,17 +1165,29 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
 
         Collection<GridCacheSqlMetadata> metas = c.context().queries().sqlMetadata();
 
-        String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_METADATA.key()));
+        String ret = content(F.asMap("cacheName", "", "cmd", GridRestCommand.CACHE_METADATA.key()));
 
         info("Cache metadata: " + ret);
 
-        testMetadata(metas, ret);
+        JsonNode arrResponse = jsonResponse(ret);
+
+        assertEquals(6, arrResponse.size());
+
+        testMetadata(metas, arrResponse);
 
-        ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_METADATA.key(), "cacheName", "person"));
+        ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME,
+            "cmd", GridRestCommand.CACHE_METADATA.key(), "cacheName", "person"));
 
         info("Cache metadata with cacheName parameter: " + ret);
 
-        testMetadata(metas, ret);
+        arrResponse = jsonResponse(ret);
+
+        assertEquals(1, arrResponse.size());
+
+        testMetadata(metas, arrResponse);
+
+        assertResponseContainsError(content(
+            F.asMap("cacheName", "nonExistingCacheName", "cmd", GridRestCommand.CACHE_METADATA.key())));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/621b0ede/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobAdapter.java
index 43465a6..3d1c308 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobAdapter.java
@@ -123,7 +123,7 @@ public abstract class ComputeJobAdapter implements ComputeJob, Callable<Object>
      *
      * @return Array of job arguments.
      */
-    @Nullable Object[] arguments() {
+    @Nullable protected Object[] arguments() {
         return args;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/621b0ede/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index f8342b3..0f47b7f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -1,4 +1,4 @@
- /*
+/*
  * 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.
@@ -209,7 +209,6 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /** */
     private boolean qryProcEnabled;
 
-
     /** */
     private AffinityTopologyVersion qryTopVer;
 
@@ -355,6 +354,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
     /**
      * Checks if IndexinSPI is enabled.
+     *
      * @return IndexingSPI enabled flag.
      */
     private boolean isIndexingSpiEnabled() {
@@ -412,7 +412,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 cctx.kernalContext().indexing().store(cacheName, key0, val0, expirationTime);
             }
 
-            if(qryProcEnabled)
+            if (qryProcEnabled)
                 qryProc.store(cacheName, key, partId, prevVal, prevVer, val, ver, expirationTime, link);
         }
         finally {
@@ -430,7 +430,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings("SimplifiableIfStatement")
-    public void remove(KeyCacheObject key, int partId, CacheObject val, GridCacheVersion ver) throws IgniteCheckedException {
+    public void remove(KeyCacheObject key, int partId, CacheObject val,
+        GridCacheVersion ver) throws IgniteCheckedException {
         assert key != null;
 
         if (!QueryUtils.isEnabled(cctx.config()) && !(key instanceof GridCacheInternal))
@@ -447,7 +448,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             }
 
             // val may be null if we have no previous value. We should not call processor in this case.
-            if(qryProcEnabled && val != null)
+            if (qryProcEnabled && val != null)
                 qryProc.remove(cacheName, key, partId, val, ver);
         }
         finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/621b0ede/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index bfc5282..c2ca587 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -53,7 +53,6 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
@@ -360,7 +359,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         GridRestCacheRequest req0 = (GridRestCacheRequest)req;
 
-        final String cacheName = req0.cacheName() == null ? DFLT_CACHE_NAME: req0.cacheName();
+        final String cacheName = req0.cacheName() == null ? DFLT_CACHE_NAME : req0.cacheName();
 
         final Object key = req0.key();
 
@@ -408,7 +407,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                 }
 
                 case CACHE_METADATA: {
-                    fut = ctx.task().execute(MetadataTask.class, null);
+                    fut = ctx.task().execute(MetadataTask.class, req0.cacheName());
 
                     break;
                 }
@@ -927,7 +926,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
     /** */
     @GridInternal
-    private static class MetadataTask extends ComputeTaskAdapter<Void, GridRestResponse> {
+    private static class MetadataTask extends ComputeTaskAdapter<String, GridRestResponse> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -937,29 +936,48 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         /** {@inheritDoc} */
         @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
-            @Nullable Void arg) throws IgniteException {
+            String cacheName) throws IgniteException {
 
             GridDiscoveryManager discovery = ignite.context().discovery();
 
-            boolean sameCaches = true;
+            Map<ComputeJob, ClusterNode> map = U.newHashMap(F.isEmpty(cacheName) ? subgrid.size() : 1);
 
-            Set<String> caches = discovery.nodePublicCaches(F.first(subgrid)).keySet();
+            if (!F.isEmpty(cacheName)) {
+                for (int i = 1; i < subgrid.size(); i++) {
+                    if (discovery.nodePublicCaches(subgrid.get(i)).keySet().contains(cacheName)) {
+                        MetadataJob job = new MetadataJob();
 
-            for (int i = 1; i < subgrid.size(); i++) {
-                if (!caches.equals(discovery.nodePublicCaches(subgrid.get(i)).keySet())) {
-                    sameCaches = false;
+                        job.setArguments(cacheName);
 
-                    break;
+                        map.put(job, subgrid.get(i));
+
+                        break;
+                    }
                 }
+
+                if (map.isEmpty())
+                    throw new IgniteException("Failed to request meta data. " + cacheName + " is not found");
             }
+            else {
+                // get meta for all caches.
+                boolean sameCaches = true;
 
-            Map<ComputeJob, ClusterNode> map = U.newHashMap(sameCaches ? 1 : subgrid.size());
+                Set<String> caches = discovery.nodePublicCaches(F.first(subgrid)).keySet();
 
-            if (sameCaches)
-                map.put(new MetadataJob(), ignite.localNode());
-            else {
-                for (ClusterNode node : subgrid)
-                    map.put(new MetadataJob(), node);
+                for (int i = 1; i < subgrid.size(); i++) {
+                    if (!caches.equals(discovery.nodePublicCaches(subgrid.get(i)).keySet())) {
+                        sameCaches = false;
+
+                        break;
+                    }
+                }
+
+                if (sameCaches)
+                    map.put(new MetadataJob(), ignite.localNode());
+                else {
+                    for (ClusterNode node : subgrid)
+                        map.put(new MetadataJob(), node);
+                }
             }
 
             return map;
@@ -1003,13 +1021,35 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         /** {@inheritDoc} */
         @Override public Collection<GridCacheSqlMetadata> execute() {
-            IgniteCacheProxy<?, ?> cache = F.first(ignite.context().cache().publicCaches());
+            String cacheName = null;
+            IgniteInternalCache<?, ?> cache;
 
-            if (cache == null)
-                return Collections.emptyList();
+            if (!F.isEmpty(arguments())) {
+                cacheName = argument(0);
+
+                cache = ignite.context().cache().publicCache(cacheName);
+
+                assert cache != null;
+            }
+            else {
+                cache = F.first(ignite.context().cache().publicCaches()).internalProxy();
+
+                if (cache == null)
+                    return Collections.emptyList();
+            }
 
             try {
-                return cache.context().queries().sqlMetadata();
+                Collection<GridCacheSqlMetadata> metas = cache.context().queries().sqlMetadata();
+
+                if (cacheName != null) {
+                    for (GridCacheSqlMetadata meta : metas)
+                        if (meta.cacheName().equals(cacheName))
+                            return Collections.singleton(meta);
+
+                    throw new IgniteException("No meta data for " + cacheName + " can be found");
+                }
+
+                return metas;
             }
             catch (IgniteCheckedException e) {
                 throw U.convertException(e);


[12/27] ignite git commit: IGNITE-4831: Add an option to disable MBeans. This closes #2265.

Posted by yz...@apache.org.
IGNITE-4831: Add an option to disable MBeans. This closes #2265.


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

Branch: refs/heads/ignite-5658
Commit: 7283edb21dcf3ff4ac22c81f1c4286d719c0044f
Parents: abe8e67
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Jul 14 14:12:29 2017 +0300
Committer: Andrey V. Mashenkov <an...@gmail.com>
Committed: Fri Jul 14 14:12:29 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   8 ++
 .../org/apache/ignite/cache/CacheManager.java   |  10 ++
 .../apache/ignite/internal/IgniteKernal.java    |  71 ++++++-----
 .../org/apache/ignite/internal/IgnitionEx.java  |  10 +-
 .../client/router/impl/GridTcpRouterImpl.java   |  57 ++++++---
 .../processors/cache/GridCacheProcessor.java    |  11 +-
 .../GridCacheDatabaseSharedManager.java         |  28 +++--
 .../IgniteCacheDatabaseSharedManager.java       |  44 ++++---
 .../ignite/internal/util/IgniteUtils.java       |  23 +++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  10 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |   2 +
 .../util/mbeans/GridMBeanDisableSelfTest.java   | 121 +++++++++++++++++++
 12 files changed, 312 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 35b0577..1a2887a 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -404,6 +404,14 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_MBEAN_APPEND_CLASS_LOADER_ID = "IGNITE_MBEAN_APPEND_CLASS_LOADER_ID";
 
     /**
+     * If property is set to {@code true}, then Ignite will disable MBeans registration.
+     * This may be helpful if MBeans are not allowed e.g. for security reasons.
+     *
+     * Default is {@code false}
+     */
+    public static final String IGNITE_MBEANS_DISABLED = "IGNITE_MBEANS_DISABLED";
+
+    /**
      * Property controlling size of buffer holding last exception. Default value of {@code 1000}.
      */
     public static final String IGNITE_EXCEPTION_REGISTRY_MAX_SIZE = "IGNITE_EXCEPTION_REGISTRY_MAX_SIZE";

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java
index 97409fa..351cd0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheManager.java
@@ -44,6 +44,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.mxbean.IgniteStandardMXBean;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -314,6 +315,9 @@ public class CacheManager implements javax.cache.CacheManager {
 
     /** {@inheritDoc} */
     @Override public void enableManagement(String cacheName, boolean enabled) {
+        if(IgniteUtils.IGNITE_MBEANS_DISABLED)
+            return;
+
         kernalGateway.readLock();
 
         try {
@@ -336,6 +340,9 @@ public class CacheManager implements javax.cache.CacheManager {
 
     /** {@inheritDoc} */
     @Override public void enableStatistics(String cacheName, boolean enabled) {
+        if(IgniteUtils.IGNITE_MBEANS_DISABLED)
+            return;
+
         kernalGateway.readLock();
 
         try {
@@ -389,6 +396,9 @@ public class CacheManager implements javax.cache.CacheManager {
      * @param beanType Mxbean name.
      */
     private void unregisterCacheObject(String name, String beanType) {
+        if(IgniteUtils.IGNITE_MBEANS_DISABLED)
+            return;
+
         MBeanServer mBeanSrv = ignite.configuration().getMBeanServer();
 
         Set<ObjectName> registeredObjNames = mBeanSrv.queryNames(getObjectName(name, beanType), null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 0c17b32..00c1d73 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1605,6 +1605,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** @throws IgniteCheckedException If registration failed. */
     private void registerKernalMBean() throws IgniteCheckedException {
+        if(U.IGNITE_MBEANS_DISABLED)
+            return;
+
         try {
             kernalMBean = U.registerMBean(
                 cfg.getMBeanServer(),
@@ -1626,6 +1629,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** @throws IgniteCheckedException If registration failed. */
     private void registerLocalNodeMBean() throws IgniteCheckedException {
+        if(U.IGNITE_MBEANS_DISABLED)
+            return;
+
         ClusterLocalNodeMetricsMXBean mbean = new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery().localNode());
 
         try {
@@ -1662,7 +1668,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ExecutorService mgmtExecSvc,
         ExecutorService restExecSvc,
         ExecutorService qryExecSvc,
-        ExecutorService schemaExecSvc) throws IgniteCheckedException {
+        ExecutorService schemaExecSvc
+    ) throws IgniteCheckedException {if(U.IGNITE_MBEANS_DISABLED)
+            return;
         pubExecSvcMBean = registerExecutorMBean(execSvc, "GridExecutionExecutor");
         sysExecSvcMBean = registerExecutorMBean(sysExecSvc, "GridSystemExecutor");
         mgmtExecSvcMBean = registerExecutorMBean(mgmtExecSvc, "GridManagementExecutor");
@@ -1684,6 +1692,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      */
     private ObjectName registerExecutorMBean(ExecutorService exec, String name) throws IgniteCheckedException {
         assert exec != null;
+        assert !U.IGNITE_MBEANS_DISABLED;
 
         try {
             ObjectName res = U.registerMBean(
@@ -1710,24 +1719,26 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @throws IgniteCheckedException If registration failed.
      */
     private void registerStripedExecutorMBean(StripedExecutor stripedExecSvc) throws IgniteCheckedException {
-        if (stripedExecSvc != null) {
-            String name = "StripedExecutor";
+        if (stripedExecSvc == null || U.IGNITE_MBEANS_DISABLED)
+            return;
 
-            try {
-                stripedExecSvcMBean = U.registerMBean(
-                    cfg.getMBeanServer(),
-                    cfg.getIgniteInstanceName(),
-                    "Thread Pools",
-                    name,
-                    new StripedExecutorMXBeanAdapter(stripedExecSvc),
-                    StripedExecutorMXBean.class);
+        String name = "StripedExecutor";
 
-                if (log.isDebugEnabled())
-                    log.debug("Registered executor service MBean: " + stripedExecSvcMBean);
-            } catch (JMException e) {
-                throw new IgniteCheckedException("Failed to register executor service MBean [name="
-                    + name + ", exec=" + stripedExecSvc + ']', e);
-            }
+        try {
+            stripedExecSvcMBean = U.registerMBean(
+                cfg.getMBeanServer(),
+                cfg.getIgniteInstanceName(),
+                "Thread Pools",
+                name,
+                new StripedExecutorMXBeanAdapter(stripedExecSvc),
+                StripedExecutorMXBean.class);
+
+            if (log.isDebugEnabled())
+                log.debug("Registered executor service MBean: " + stripedExecSvcMBean);
+        }
+        catch (JMException e) {
+            throw new IgniteCheckedException("Failed to register executor service MBean [name="
+                + name + ", exec=" + stripedExecSvc + ']', e);
         }
     }
 
@@ -1738,22 +1749,24 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @return {@code True} if successfully unregistered, {@code false} otherwise.
      */
     private boolean unregisterMBean(@Nullable ObjectName mbean) {
-        if (mbean != null)
-            try {
-                cfg.getMBeanServer().unregisterMBean(mbean);
+        if (mbean == null)
+            return true;
 
-                if (log.isDebugEnabled())
-                    log.debug("Unregistered MBean: " + mbean);
+        assert !U.IGNITE_MBEANS_DISABLED;
 
-                return true;
-            }
-            catch (JMException e) {
-                U.error(log, "Failed to unregister MBean.", e);
+        try {
+            cfg.getMBeanServer().unregisterMBean(mbean);
 
-                return false;
-            }
+            if (log.isDebugEnabled())
+                log.debug("Unregistered MBean: " + mbean);
 
-        return true;
+            return true;
+        }
+        catch (JMException e) {
+            U.error(log, "Failed to unregister MBean.", e);
+
+            return false;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 187d4af..d219333 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -2120,7 +2120,7 @@ public class IgnitionEx {
             if (myCfg.getUserAttributes() == null)
                 myCfg.setUserAttributes(Collections.<String, Object>emptyMap());
 
-            if (myCfg.getMBeanServer() == null)
+            if (myCfg.getMBeanServer() == null && !U.IGNITE_MBEANS_DISABLED)
                 myCfg.setMBeanServer(ManagementFactory.getPlatformMBeanServer());
 
             Marshaller marsh = myCfg.getMarshaller();
@@ -2570,6 +2570,11 @@ public class IgnitionEx {
          * @throws IgniteCheckedException If registration failed.
          */
         private void registerFactoryMbean(MBeanServer srv) throws IgniteCheckedException {
+            if(U.IGNITE_MBEANS_DISABLED)
+                return;
+
+            assert srv != null;
+
             synchronized (mbeans) {
                 GridMBeanServerData data = mbeans.get(srv);
 
@@ -2620,6 +2625,9 @@ public class IgnitionEx {
          * Unregister delegate Mbean instance for {@link Ignition}.
          */
         private void unregisterFactoryMBean() {
+            if(U.IGNITE_MBEANS_DISABLED)
+                return;
+
             synchronized (mbeans) {
                 Iterator<Entry<MBeanServer, GridMBeanServerData>> iter = mbeans.entrySet().iterator();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
index cb8325b..da55ec7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
@@ -169,6 +169,32 @@ public class GridTcpRouterImpl implements GridTcpRouter, GridTcpRouterMBean, Lif
                 "are in use) [firstPort=" + cfg.getPort() + ", lastPort=" + (cfg.getPort() + cfg.getPortRange()) +
                 ", addr=" + hostAddr + ']');
 
+        registerMBean();
+    }
+
+    /**
+     * Stops this router.
+     */
+    @Override public void stop() {
+        if (srv != null)
+            srv.stop();
+
+        if (client != null)
+            client.stop(true);
+
+        unregisterMBean();
+
+        if (log.isInfoEnabled())
+            log.info("TCP router successfully stopped.");
+    }
+
+    /**
+     * Try to register MBean.
+     */
+    private void registerMBean() {
+        if (U.IGNITE_MBEANS_DISABLED)
+            return;
+
         try {
             ObjectName objName = U.registerMBean(
                 ManagementFactory.getPlatformMBeanServer(),
@@ -189,28 +215,23 @@ public class GridTcpRouterImpl implements GridTcpRouter, GridTcpRouterMBean, Lif
     }
 
     /**
-     * Stops this router.
+     * Unregister MBean.
      */
-    @Override public void stop() {
-        if (srv != null)
-            srv.stop();
-
-        if (client != null)
-            client.stop(true);
+    private void unregisterMBean() {
+        if (mbeanName == null)
+            return;
 
-        if (mbeanName != null)
-            try {
-                ManagementFactory.getPlatformMBeanServer().unregisterMBean(mbeanName);
+        assert !U.IGNITE_MBEANS_DISABLED;
 
-                if (log.isDebugEnabled())
-                    log.debug("Unregistered MBean: " + mbeanName);
-            }
-            catch (JMException e) {
-                U.error(log, "Failed to unregister MBean.", e);
-            }
+        try {
+            ManagementFactory.getPlatformMBeanServer().unregisterMBean(mbeanName);
 
-        if (log.isInfoEnabled())
-            log.info("TCP router successfully stopped.");
+            if (log.isDebugEnabled())
+                log.debug("Unregistered MBean: " + mbeanName);
+        }
+        catch (JMException e) {
+            U.error(log, "Failed to unregister MBean.", e);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/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 5149d4b..5b709b3 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
@@ -37,7 +37,6 @@ import java.util.concurrent.CountDownLatch;
 import javax.cache.configuration.Factory;
 import javax.cache.integration.CacheLoader;
 import javax.cache.integration.CacheWriter;
-import javax.management.JMException;
 import javax.management.MBeanServer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -3584,6 +3583,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     private void registerMbean(Object obj, @Nullable String cacheName, boolean near)
         throws IgniteCheckedException {
+        if(U.IGNITE_MBEANS_DISABLED)
+            return;
+
         assert obj != null;
 
         MBeanServer srvr = ctx.config().getMBeanServer();
@@ -3602,7 +3604,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     U.registerCacheMBean(srvr, ctx.igniteInstanceName(), cacheName, obj.getClass().getName(), mbeanImpl,
                         (Class<Object>)itf);
                 }
-                catch (JMException e) {
+                catch (Throwable e) {
                     throw new IgniteCheckedException("Failed to register MBean for component: " + obj, e);
                 }
 
@@ -3619,6 +3621,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param near Near flag.
      */
     private void unregisterMbean(Object o, @Nullable String cacheName, boolean near) {
+        if(U.IGNITE_MBEANS_DISABLED)
+            return;
+
         assert o != null;
 
         MBeanServer srvr = ctx.config().getMBeanServer();
@@ -3645,7 +3650,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             try {
                 srvr.unregisterMBean(U.makeCacheMBeanName(ctx.igniteInstanceName(), cacheName, o.getClass().getName()));
             }
-            catch (JMException e) {
+            catch (Throwable e) {
                 U.error(log, "Failed to unregister MBean for component: " + o, e);
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index d147f36..39038ba 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -477,9 +477,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
+     * Try to register Metrics MBean.
      * @throws IgniteCheckedException If failed.
      */
     private void registrateMetricsMBean() throws IgniteCheckedException {
+        if (U.IGNITE_MBEANS_DISABLED)
+            return;
+
         try {
             persistenceMetricsMbeanName = U.registerMBean(
                 cctx.kernalContext().config().getMBeanServer(),
@@ -489,25 +493,25 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 persStoreMetrics,
                 PersistenceMetricsMXBean.class);
         }
-        catch (JMException e) {
+        catch (Throwable e) {
             throw new IgniteCheckedException("Failed to register " + MBEAN_NAME + " MBean.", e);
         }
     }
 
     /**
-     *
+     * Unregister metrics MBean.
      */
     private void unRegistrateMetricsMBean() {
-        if (persistenceMetricsMbeanName != null) {
-            try {
-                cctx.kernalContext().config().getMBeanServer().unregisterMBean(persistenceMetricsMbeanName);
-            }
-            catch (InstanceNotFoundException ignore) {
-                // No-op, nothing to unregister.
-            }
-            catch (MBeanRegistrationException e) {
-                U.error(log, "Failed to unregister " + MBEAN_NAME + " MBean.", e);
-            }
+        if (persistenceMetricsMbeanName == null)
+            return;
+
+        assert !U.IGNITE_MBEANS_DISABLED;
+
+        try {
+            cctx.kernalContext().config().getMBeanServer().unregisterMBean(persistenceMetricsMbeanName);
+        }
+        catch (Throwable e) {
+            U.error(log, "Failed to unregister " + MBEAN_NAME + " MBean.", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index c503fb4..eec3b85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -115,6 +115,9 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * Registers MBeans for all MemoryMetrics configured in this instance.
      */
     private void registerMetricsMBeans() {
+        if(U.IGNITE_MBEANS_DISABLED)
+            return;
+
         IgniteConfiguration cfg = cctx.gridConfig();
 
         for (MemoryMetrics memMetrics : memMetricsMap.values()) {
@@ -134,6 +137,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         MemoryPolicyConfiguration memPlcCfg,
         IgniteConfiguration cfg
     ) {
+        assert !U.IGNITE_MBEANS_DISABLED;
+
         try {
             U.registerMBean(
                 cfg.getMBeanServer(),
@@ -143,7 +148,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                 new MemoryMetricsMXBeanImpl(memMetrics, memPlcCfg),
                 MemoryMetricsMXBean.class);
         }
-        catch (JMException e) {
+        catch (Throwable e) {
             U.error(log, "Failed to register MBean for MemoryMetrics with name: '" + memMetrics.getName() + "'", e);
         }
     }
@@ -647,19 +652,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
                 memPlc.evictionTracker().stop();
 
-                IgniteConfiguration cfg = cctx.gridConfig();
-
-                try {
-                    cfg.getMBeanServer().unregisterMBean(
-                        U.makeMBeanName(
-                            cfg.getIgniteInstanceName(),
-                            "MemoryMetrics",
-                            memPlc.memoryMetrics().getName()));
-                }
-                catch (JMException e) {
-                    U.error(log, "Failed to unregister MBean for memory metrics: " +
-                        memPlc.memoryMetrics().getName(), e);
-                }
+                unregisterMBean(memPlc.memoryMetrics().getName());
             }
 
             memPlcMap.clear();
@@ -669,6 +662,29 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
+     * Unregister MBean.
+     * @param name Name of mbean.
+     */
+    private void unregisterMBean(String name) {
+        if(U.IGNITE_MBEANS_DISABLED)
+            return;
+
+        IgniteConfiguration cfg = cctx.gridConfig();
+
+        try {
+            cfg.getMBeanServer().unregisterMBean(
+                U.makeMBeanName(
+                    cfg.getIgniteInstanceName(),
+                    "MemoryMetrics", name
+                    ));
+        }
+        catch (Throwable e) {
+            U.error(log, "Failed to unregister MBean for memory metrics: " +
+                name, e);
+        }
+    }
+
+    /**
      *
      */
     public boolean persistenceEnabled() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 6b94d09..6a3be55 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -139,6 +139,7 @@ import java.util.zip.ZipInputStream;
 import java.util.zip.ZipOutputStream;
 import javax.management.DynamicMBean;
 import javax.management.JMException;
+import javax.management.MBeanRegistrationException;
 import javax.management.MBeanServer;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
@@ -155,6 +156,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteDeploymentException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
@@ -515,11 +517,14 @@ public abstract class IgniteUtils {
         }
     };
 
+    /** Ignite MBeans disabled flag. */
+    public static boolean IGNITE_MBEANS_DISABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_MBEANS_DISABLED);
+
     /** */
     private static final boolean assertionsEnabled;
 
     /*
-     *
+     * Initializes enterprise check.
      */
     static {
         boolean assertionsEnabled0 = true;
@@ -4486,10 +4491,12 @@ public abstract class IgniteUtils {
      * @param impl MBean implementation.
      * @param itf MBean interface.
      * @return JMX object name.
+     * @throws MBeanRegistrationException if MBeans are disabled.
      * @throws JMException If MBean creation failed.
      */
-    public static <T> ObjectName registerMBean(MBeanServer mbeanSrv, @Nullable String igniteInstanceName,
-        @Nullable String grp, String name, T impl, @Nullable Class<T> itf) throws JMException {
+    public static <T> ObjectName registerMBean(MBeanServer mbeanSrv, @Nullable String igniteInstanceName, @Nullable String grp,
+        String name, T impl, @Nullable Class<T> itf) throws JMException {if(IGNITE_MBEANS_DISABLED)
+            throw new MBeanRegistrationException(new IgniteIllegalStateException("No MBeans are allowed."));
         assert mbeanSrv != null;
         assert name != null;
         assert itf != null;
@@ -4510,10 +4517,15 @@ public abstract class IgniteUtils {
      * @param impl MBean implementation.
      * @param itf MBean interface.
      * @return JMX object name.
+     * @throws MBeanRegistrationException if MBeans are disabled.
      * @throws JMException If MBean creation failed.
+     * @throws IgniteException If MBean creation are not allowed.
      */
     public static <T> ObjectName registerMBean(MBeanServer mbeanSrv, ObjectName name, T impl, Class<T> itf)
         throws JMException {
+        if(IGNITE_MBEANS_DISABLED)
+            throw new MBeanRegistrationException(new IgniteIllegalStateException("MBeans are disabled."));
+
         assert mbeanSrv != null;
         assert name != null;
         assert itf != null;
@@ -4536,10 +4548,15 @@ public abstract class IgniteUtils {
      * @param impl MBean implementation.
      * @param itf MBean interface.
      * @return JMX object name.
+     * @throws MBeanRegistrationException if MBeans are disabled.
      * @throws JMException If MBean creation failed.
+     * @throws IgniteException If MBean creation are not allowed.
      */
     public static <T> ObjectName registerCacheMBean(MBeanServer mbeanSrv, @Nullable String igniteInstanceName,
         @Nullable String cacheName, String name, T impl, Class<T> itf) throws JMException {
+        if(IGNITE_MBEANS_DISABLED)
+            throw new MBeanRegistrationException(new IgniteIllegalStateException("MBeans are disabled."));
+
         assert mbeanSrv != null;
         assert name != null;
         assert itf != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 59c2656..07ba214 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -401,9 +401,11 @@ public abstract class IgniteSpiAdapter implements IgniteSpi {
      * @param <T> Type of the MBean
      * @throws IgniteSpiException If registration failed.
      */
-    protected final <T extends IgniteSpiManagementMBean> void registerMBean(
-        String igniteInstanceName, T impl, Class<T> mbeanItf
-    ) throws IgniteSpiException {
+    protected final <T extends IgniteSpiManagementMBean> void registerMBean(String igniteInstanceName, T impl, Class<T> mbeanItf
+       ) throws IgniteSpiException {
+        if(ignite == null || U.IGNITE_MBEANS_DISABLED)
+            return;
+
         MBeanServer jmx = ignite.configuration().getMBeanServer();
 
         assert mbeanItf == null || mbeanItf.isInterface();
@@ -428,6 +430,8 @@ public abstract class IgniteSpiAdapter implements IgniteSpi {
     protected final void unregisterMBean() throws IgniteSpiException {
         // Unregister SPI MBean.
         if (spiMBean != null && ignite != null) {
+            assert !U.IGNITE_MBEANS_DISABLED;
+
             MBeanServer jmx = ignite.configuration().getMBeanServer();
 
             assert jmx != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
index 1298a75..20f37f7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
@@ -46,6 +46,7 @@ import org.apache.ignite.util.GridPartitionMapSelfTest;
 import org.apache.ignite.util.GridQueueSelfTest;
 import org.apache.ignite.util.GridSpinReadWriteLockSelfTest;
 import org.apache.ignite.util.GridStringBuilderFactorySelfTest;
+import org.apache.ignite.util.mbeans.GridMBeanDisableSelfTest;
 import org.apache.ignite.util.mbeans.GridMBeanSelfTest;
 
 /**
@@ -77,6 +78,7 @@ public class IgniteUtilSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridToStringBuilderSelfTest.class);
         suite.addTestSuite(GridByteArrayListSelfTest.class);
         suite.addTestSuite(GridMBeanSelfTest.class);
+        suite.addTestSuite(GridMBeanDisableSelfTest.class);
         suite.addTestSuite(GridLongListSelfTest.class);
         suite.addTestSuite(GridIntListSelfTest.class);
         suite.addTestSuite(GridArraysSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7283edb2/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java
new file mode 100644
index 0000000..f08f58b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/util/mbeans/GridMBeanDisableSelfTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.util.mbeans;
+
+import java.util.concurrent.Callable;
+import javax.management.MBeanRegistrationException;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Disabling MBeans test.
+ */
+public class GridMBeanDisableSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        IgniteUtils.IGNITE_MBEANS_DISABLED = true;
+
+        super.beforeTestsStarted();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        IgniteUtils.IGNITE_MBEANS_DISABLED = false;
+    }
+
+    /**
+     * Test MBean registration.
+     *
+     * @throws Exception Thrown if test fails.
+     */
+    public void testCorrectMBeanInfo() throws Exception {
+        // Node should start and stopped with no errors.
+        try (final Ignite ignite = startGrid(0)) {
+
+            // Cache should be created and closed with no errors.
+            try (IgniteCache cache = ignite.getOrCreateCache(new CacheConfiguration("MyCache"))) {
+
+                final MBeanServer server = ignite.configuration().getMBeanServer();
+
+                GridTestUtils.assertThrowsWithCause(
+                    new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            U.registerMBean(server, ignite.name(), "dummy", "DummyMbean1", new DummyMBeanImpl(), DummyMBean.class);
+
+                            return null;
+
+                        }
+                    }, MBeanRegistrationException.class);
+
+                GridTestUtils.assertThrowsWithCause(
+                    new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            ObjectName objName = U.makeMBeanName(
+                                ignite.name(),
+                                "dummy",
+                                "DummyMbean2"
+                            );
+
+                            U.registerMBean(server, objName, new DummyMBeanImpl(), DummyMBean.class);
+
+                            return null;
+
+                        }
+                    }, MBeanRegistrationException.class);
+
+                GridTestUtils.assertThrowsWithCause(
+                    new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            U.registerCacheMBean(server, ignite.name(), "MyCache", "DummyMbean3",
+                                new DummyMBeanImpl(), DummyMBean.class);
+
+                            return null;
+
+                        }
+                    }, MBeanRegistrationException.class);
+            }
+        }
+    }
+
+    /**
+     * MBean dummy interface.
+     */
+    interface DummyMBean {
+        /** */
+        void noop();
+    }
+
+    /**
+     * MBean stub.
+     */
+    static class DummyMBeanImpl implements DummyMBean {
+        /** {@inheritDoc} */
+        @Override public void noop() {
+            // No op.
+        }
+    }
+}
\ No newline at end of file


[09/27] ignite git commit: IGNITE-5479 - Cleanup public API for PersistentStoreConfiguration

Posted by yz...@apache.org.
IGNITE-5479 - Cleanup public API for PersistentStoreConfiguration


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

Branch: refs/heads/ignite-5658
Commit: 689b1b6e9c3e723cf394c7ff2427097b21d96ce3
Parents: 21964fb
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Jul 13 10:12:01 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Jul 13 10:12:01 2017 +0300

----------------------------------------------------------------------
 .../configuration/MemoryConfiguration.java      |  4 +-
 .../MemoryPolicyConfiguration.java              |  4 +-
 .../PersistentStoreConfiguration.java           | 61 ++++++++++----------
 .../GridCacheDatabaseSharedManager.java         | 23 +++++---
 .../IgniteCacheDatabaseSharedManager.java       | 21 ++++---
 .../wal/FileWriteAheadLogManager.java           |  6 +-
 .../utils/PlatformConfigurationUtils.java       |  4 +-
 .../node/VisorPersistentStoreConfiguration.java | 36 ++++++------
 .../persistence/db/wal/IgnitePdsWalTlbTest.java |  2 +-
 .../Cache/PersistentStoreTest.cs                |  8 ++-
 .../Config/spring-test.xml                      |  4 ++
 .../IgniteConfigurationTest.cs                  |  8 ++-
 .../PersistentStoreConfiguration.cs             | 31 ++++++----
 modules/web-console/backend/app/mongo.js        |  2 +-
 .../generator/ConfigurationGenerator.js         |  8 +--
 .../generator/defaults/Cluster.service.js       |  2 +-
 .../configuration/clusters/persistence.pug      |  2 +-
 17 files changed, 128 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
index 5cf6cb7..f83d7d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryConfiguration.java
@@ -64,7 +64,7 @@ public class MemoryConfiguration implements Serializable {
 
     /** Default memory policy start size (256 MB). */
     @SuppressWarnings("UnnecessaryBoxing")
-    public static final Long DFLT_MEMORY_POLICY_INITIAL_SIZE = new Long(256L * 1024 * 1024);
+    public static final long DFLT_MEMORY_POLICY_INITIAL_SIZE = 256L * 1024 * 1024;
 
     /** Fraction of available memory to allocate for default MemoryPolicy. */
     private static final double DFLT_MEMORY_POLICY_FRACTION = 0.8;
@@ -222,6 +222,8 @@ public class MemoryConfiguration implements Serializable {
 
         if (maxSize < DFLT_MEMORY_POLICY_INITIAL_SIZE)
             memPlc.setInitialSize(maxSize);
+        else
+            memPlc.setInitialSize(DFLT_MEMORY_POLICY_INITIAL_SIZE);
 
         memPlc.setMaxSize(maxSize);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
index b496901..dff8b2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
@@ -78,7 +78,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
     private String name = DFLT_MEM_PLC_DEFAULT_NAME;
 
     /** Memory policy start size. */
-    private Long initialSize = MemoryConfiguration.DFLT_MEMORY_POLICY_INITIAL_SIZE;
+    private long initialSize;
 
     /** Memory policy maximum size. */
     private long maxSize = MemoryConfiguration.DFLT_MEMORY_POLICY_MAX_SIZE;
@@ -172,7 +172,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
      *
      * @return Memory policy start size.
      */
-    public Long getInitialSize() {
+    public long getInitialSize() {
         return initialSize;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index 4792483..e8a0ff4 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -44,10 +44,6 @@ public class PersistentStoreConfiguration implements Serializable {
     /** Default length of interval over which rate-based metric is calculated. */
     public static final int DFLT_RATE_TIME_INTERVAL_MILLIS = 60_000;
 
-    /** */
-    @SuppressWarnings("UnnecessaryBoxing")
-    public static final Long DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE = new Long(256L * 1024 * 1024);
-
     /** Default number of checkpointing threads. */
     public static final int DFLT_CHECKPOINTING_THREADS = 1;
 
@@ -70,7 +66,7 @@ public class PersistentStoreConfiguration implements Serializable {
     public static final int DFLT_WAL_FLUSH_FREQ = 2000;
 
     /** Default wal fsync delay. */
-    public static final int DFLT_WAL_FSYNC_DELAY = 1;
+    public static final int DFLT_WAL_FSYNC_DELAY = 1000;
 
     /** Default wal record iterator buffer size. */
     public static final int DFLT_WAL_RECORD_ITERATOR_BUFFER_SIZE = 64 * 1024 * 1024;
@@ -90,11 +86,11 @@ public class PersistentStoreConfiguration implements Serializable {
     /** Checkpointing frequency. */
     private long checkpointingFreq = DFLT_CHECKPOINTING_FREQ;
 
-    /** Lock wait time. */
-    private int lockWaitTime = DFLT_LOCK_WAIT_TIME;
+    /** Lock wait time, in milliseconds. */
+    private long lockWaitTime = DFLT_LOCK_WAIT_TIME;
 
     /** */
-    private Long checkpointingPageBufSize = DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
+    private long checkpointingPageBufSize;
 
     /** */
     private int checkpointingThreads = DFLT_CHECKPOINTING_THREADS;
@@ -124,10 +120,10 @@ public class PersistentStoreConfiguration implements Serializable {
     private int tlbSize = DFLT_TLB_SIZE;
 
     /** Wal flush frequency in milliseconds. */
-    private int walFlushFreq = DFLT_WAL_FLUSH_FREQ;
+    private long walFlushFreq = DFLT_WAL_FLUSH_FREQ;
 
     /** Wal fsync delay. */
-    private int walFsyncDelay = DFLT_WAL_FSYNC_DELAY;
+    private long walFsyncDelay = DFLT_WAL_FSYNC_DELAY;
 
     /** Wal record iterator buffer size. */
     private int walRecordIterBuffSize = DFLT_WAL_RECORD_ITERATOR_BUFFER_SIZE;
@@ -200,9 +196,10 @@ public class PersistentStoreConfiguration implements Serializable {
     /**
      * Gets amount of memory allocated for a checkpointing temporary buffer.
      *
-     * @return checkpointing page buffer size in bytes.
+     * @return Checkpointing page buffer size in bytes or {@code 0} for Ignite
+     *      to choose the buffer size automatically.
      */
-    public Long getCheckpointingPageBufferSize() {
+    public long getCheckpointingPageBufferSize() {
         return checkpointingPageBufSize;
     }
 
@@ -211,7 +208,8 @@ public class PersistentStoreConfiguration implements Serializable {
      * copies of pages that are being written to disk and being update in parallel while the checkpointing is in
      * progress.
      *
-     * @param checkpointingPageBufSize checkpointing page buffer size in bytes.
+     * @param checkpointingPageBufSize Checkpointing page buffer size in bytes or {@code 0} for Ignite to
+     *      choose the buffer size automatically.
      * @return {@code this} for chaining.
      */
     public PersistentStoreConfiguration setCheckpointingPageBufferSize(long checkpointingPageBufSize) {
@@ -243,21 +241,23 @@ public class PersistentStoreConfiguration implements Serializable {
     }
 
     /**
-     * Time out in second, while wait and try get file lock for start persist manager.
+     * Time out in milliseonds to wait when acquiring persistence store lock file before failing the
+     * local node.
      *
-     * @return Time for wait.
+     * @return Lock wait time in milliseconds.
      */
-    public int getLockWaitTime() {
+    public long getLockWaitTime() {
         return lockWaitTime;
     }
 
     /**
-     * Time out in milliseconds, while wait and try get file lock for start persist manager.
+     * Time out in milliseconds  to wait when acquiring persistence store lock file before failing the
+     * local node.
      *
-     * @param lockWaitTime Lock wait time.
+     * @param lockWaitTime Lock wait time in milliseconds.
      * @return {@code this} for chaining.
      */
-    public PersistentStoreConfiguration setLockWaitTime(int lockWaitTime) {
+    public PersistentStoreConfiguration setLockWaitTime(long lockWaitTime) {
         this.lockWaitTime = lockWaitTime;
 
         return this;
@@ -474,19 +474,22 @@ public class PersistentStoreConfiguration implements Serializable {
     }
 
     /**
-     *  Property define how often will be fsync, in milliseconds.
-     *  In background mode, exist thread which do fsync by timeout.
+     *  This property define how often WAL will be fsync-ed in {@code BACKGROUND} mode. Ignored for
+     *  all other WAL modes.
      *
-     * @return Flush frequency.
+     * @return WAL flush frequency, in milliseconds.
      */
-    public int getWalFlushFrequency() {
+    public long getWalFlushFrequency() {
         return walFlushFreq;
     }
 
     /**
-     * @param walFlushFreq Wal flush frequency, in milliseconds.
+     *  This property define how often WAL will be fsync-ed in {@code BACKGROUND} mode. Ignored for
+     *  all other WAL modes.
+     *
+     * @param walFlushFreq WAL flush frequency, in milliseconds.
      */
-    public PersistentStoreConfiguration setWalFlushFrequency(int walFlushFreq) {
+    public PersistentStoreConfiguration setWalFlushFrequency(long walFlushFreq) {
         this.walFlushFreq = walFlushFreq;
 
         return this;
@@ -495,15 +498,15 @@ public class PersistentStoreConfiguration implements Serializable {
     /**
      * Gets the fsync delay, in nanoseconds.
      */
-    public int getWalFsyncDelay() {
+    public long getWalFsyncDelayNanos() {
         return walFsyncDelay <= 0 ? DFLT_WAL_FSYNC_DELAY : walFsyncDelay;
     }
 
     /**
-     * @param walFsyncDelay Wal fsync delay, in nanoseconds.
+     * @param walFsyncDelayNanos Wal fsync delay, in nanoseconds.
      */
-    public PersistentStoreConfiguration setWalFsyncDelay(int walFsyncDelay) {
-        this.walFsyncDelay = walFsyncDelay;
+    public PersistentStoreConfiguration setWalFsyncDelayNanos(long walFsyncDelayNanos) {
+        walFsyncDelay = walFsyncDelayNanos;
 
         return this;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 9f2067a..d147f36 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -151,6 +151,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** */
     public static final String IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC = "IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC";
 
+    /** Default checkpointing page buffer size (may be adjusted by Ignite). */
+    public static final Long DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE = 256L * 1024 * 1024;
+
     /** Skip sync. */
     private final boolean skipSync = IgniteSystemProperties.getBoolean(IGNITE_PDS_CHECKPOINT_TEST_SKIP_SYNC);
 
@@ -277,7 +280,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     private FileLockHolder fileLockHolder;
 
     /** Lock wait time. */
-    private final int lockWaitTime;
+    private final long lockWaitTime;
 
     /** */
     private Map<Integer, Map<Integer, T2<Long, WALPointer>>> reservedForExchange;
@@ -375,10 +378,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
-     * @throws IgniteCheckedException If failed.
+     *
      */
-    private void initDataBase() throws IgniteCheckedException {
-        Long cpBufSize = persistenceCfg.getCheckpointingPageBufferSize();
+    private void initDataBase() {
+        long cpBufSize = persistenceCfg.getCheckpointingPageBufferSize();
 
         if (persistenceCfg.getCheckpointingThreads() > 1)
             asyncRunner = new ThreadPoolExecutor(
@@ -391,7 +394,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         // Intentionally use identity comparison to check if configuration default has changed.
         //noinspection NumberEquality
-        if (cpBufSize == PersistentStoreConfiguration.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE) {
+        if (cpBufSize == 0L) {
+            cpBufSize = DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
+
             MemoryConfiguration memCfg = cctx.kernalContext().config().getMemoryConfiguration();
 
             assert memCfg != null;
@@ -2928,10 +2933,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
 
         /**
-         * @param lockWaitTime During which time thread will try capture file lock.
+         * @param lockWaitTimeMillis During which time thread will try capture file lock.
          * @throws IgniteCheckedException If failed to capture file lock.
          */
-        public void tryLock(int lockWaitTime) throws IgniteCheckedException {
+        public void tryLock(long lockWaitTimeMillis) throws IgniteCheckedException {
             assert lockFile != null;
 
             FileChannel ch = lockFile.getChannel();
@@ -2965,7 +2970,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 String content = null;
 
                 // Try to get lock, if not available wait 1 sec and re-try.
-                for (int i = 0; i < lockWaitTime; i += 1000) {
+                for (int i = 0; i < lockWaitTimeMillis; i += 1000) {
                     try {
                         lock = ch.tryLock(0, 1, false);
                         if (lock != null && lock.isValid()) {
@@ -2989,7 +2994,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 if (content == null)
                     content = readContent();
 
-                failMsg = "Failed to acquire file lock during " + (lockWaitTime / 1000) +
+                failMsg = "Failed to acquire file lock during " + (lockWaitTimeMillis / 1000) +
                     " sec, (locked by " + content + "): " + file.getAbsolutePath();
             }
             catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index c5f174c..c503fb4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -295,15 +295,6 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
-     * @param dbCfg Database configuration.
-     * @param memPlcCfg MemoryPolicy configuration.
-     * @param memMetrics MemoryMetrics instance.
-     */
-    private MemoryPolicy createDefaultMemoryPolicy(MemoryConfiguration dbCfg, MemoryPolicyConfiguration memPlcCfg, MemoryMetricsImpl memMetrics) {
-        return initMemory(dbCfg, memPlcCfg, memMetrics);
-    }
-
-    /**
      * @param sysCacheInitSize Initial size of PageMemory to be created for system cache.
      * @param sysCacheMaxSize Maximum size of PageMemory to be created for system cache.
      *
@@ -454,6 +445,14 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      * @throws IgniteCheckedException If config is invalid.
      */
     private void checkPolicySize(MemoryPolicyConfiguration plcCfg) throws IgniteCheckedException {
+        boolean dfltInitSize = false;
+
+        if (plcCfg.getInitialSize() == 0) {
+            plcCfg.setInitialSize(DFLT_MEMORY_POLICY_INITIAL_SIZE);
+
+            dfltInitSize = true;
+        }
+
         if (plcCfg.getInitialSize() < MIN_PAGE_MEMORY_SIZE)
             throw new IgniteCheckedException("MemoryPolicy must have size more than 10MB (use " +
                 "MemoryPolicyConfiguration.initialSize property to set correct size in bytes) " +
@@ -461,8 +460,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
             );
 
         if (plcCfg.getMaxSize() < plcCfg.getInitialSize()) {
-            // We will know for sure if initialSize has been changed if we compare Longs by "==".
-            if (plcCfg.getInitialSize() == DFLT_MEMORY_POLICY_INITIAL_SIZE) {
+            // If initial size was not set, use the max size.
+            if (dfltInitSize) {
                 plcCfg.setInitialSize(plcCfg.getMaxSize());
 
                 LT.warn(log, "MemoryPolicy maxSize=" + U.readableSize(plcCfg.getMaxSize(), true) +

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 5c112fb..897f903 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -124,7 +124,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     private final int tlbSize;
 
     /** WAL flush frequency. Makes sense only for {@link WALMode#BACKGROUND} log WALMode. */
-    private final int flushFreq;
+    private final long flushFreq;
 
     /** Fsync delay. */
     private final long fsyncDelay;
@@ -229,7 +229,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         mode = psCfg.getWalMode();
         tlbSize = psCfg.getTlbSize();
         flushFreq = psCfg.getWalFlushFrequency();
-        fsyncDelay = psCfg.getWalFsyncDelay();
+        fsyncDelay = psCfg.getWalFsyncDelayNanos();
         alwaysWriteFullPages = psCfg.isAlwaysWriteFullPages();
         ioFactory = psCfg.getFileIOFactory();
         walAutoArchiveAfterInactivity = psCfg.getWalAutoArchiveAfterInactivity();
@@ -1666,7 +1666,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         /** Condition activated each time writeBuffer() completes. Used to wait previously flushed write to complete */
         private final Condition writeComplete = lock.newCondition();
 
-        /** Condition for timed wait of several threads, see {@link PersistentStoreConfiguration#getWalFsyncDelay()} */
+        /** Condition for timed wait of several threads, see {@link PersistentStoreConfiguration#getWalFsyncDelayNanos()} */
         private final Condition fsync = lock.newCondition();
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 03b30db..8c9f5e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -1499,7 +1499,7 @@ public class PlatformConfigurationUtils {
                 .setWalMode(WALMode.fromOrdinal(in.readInt()))
                 .setTlbSize(in.readInt())
                 .setWalFlushFrequency((int) in.readLong())
-                .setWalFsyncDelay(in.readInt())
+                .setWalFsyncDelayNanos(in.readLong())
                 .setWalRecordIteratorBufferSize(in.readInt())
                 .setAlwaysWriteFullPages(in.readBoolean())
                 .setMetricsEnabled(in.readBoolean())
@@ -1531,7 +1531,7 @@ public class PlatformConfigurationUtils {
             w.writeInt(cfg.getWalMode().ordinal());
             w.writeInt(cfg.getTlbSize());
             w.writeLong(cfg.getWalFlushFrequency());
-            w.writeInt(cfg.getWalFsyncDelay());
+            w.writeLong(cfg.getWalFsyncDelayNanos());
             w.writeInt(cfg.getWalRecordIteratorBufferSize());
             w.writeBoolean(cfg.isAlwaysWriteFullPages());
             w.writeBoolean(cfg.isMetricsEnabled());

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
index 3559845..128f43a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPersistentStoreConfiguration.java
@@ -39,10 +39,10 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     private long checkpointingFreq;
 
     /** Lock wait time. */
-    private int lockWaitTime;
+    private long lockWaitTime;
 
     /** */
-    private Long checkpointingPageBufSize;
+    private long checkpointingPageBufSize;
 
     /** */
     private int checkpointingThreads;
@@ -72,10 +72,10 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     private int tlbSize;
 
     /** Wal flush frequency. */
-    private int walFlushFreq;
+    private long walFlushFreq;
 
-    /** Wal fsync delay. */
-    private int walFsyncDelay;
+    /** Wal fsync delay in nanoseconds. */
+    private long walFsyncDelay;
 
     /** Wal record iterator buffer size. */
     private int walRecordIterBuffSize;
@@ -114,7 +114,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         walMode = cfg.getWalMode();
         tlbSize = cfg.getTlbSize();
         walFlushFreq = cfg.getWalFlushFrequency();
-        walFsyncDelay = cfg.getWalFsyncDelay();
+        walFsyncDelay = cfg.getWalFsyncDelayNanos();
         walRecordIterBuffSize = cfg.getWalRecordIteratorBufferSize();
         alwaysWriteFullPages = cfg.isAlwaysWriteFullPages();
         subIntervals = cfg.getSubIntervals();
@@ -138,7 +138,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     /**
      * @return Checkpointing page buffer size in bytes.
      */
-    public Long getCheckpointingPageBufferSize() {
+    public long getCheckpointingPageBufferSize() {
         return checkpointingPageBufSize;
     }
 
@@ -152,7 +152,7 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     /**
      * @return Time for wait.
      */
-    public int getLockWaitTime() {
+    public long getLockWaitTime() {
         return lockWaitTime;
     }
 
@@ -229,14 +229,14 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     /**
      * @return Flush frequency.
      */
-    public int getWalFlushFrequency() {
+    public long getWalFlushFrequency() {
         return walFlushFreq;
     }
 
     /**
      * Gets the fsync delay, in nanoseconds.
      */
-    public int getWalFsyncDelay() {
+    public long getWalFsyncDelayNanos() {
         return walFsyncDelay;
     }
 
@@ -258,8 +258,8 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     @Override protected void writeExternalData(ObjectOutput out) throws IOException {
         U.writeString(out, persistenceStorePath);
         out.writeLong(checkpointingFreq);
-        out.writeInt(lockWaitTime);
-        out.writeObject(checkpointingPageBufSize);
+        out.writeLong(lockWaitTime);
+        out.writeLong(checkpointingPageBufSize);
         out.writeInt(checkpointingThreads);
         out.writeInt(walHistSize);
         out.writeInt(walSegments);
@@ -269,8 +269,8 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         out.writeBoolean(metricsEnabled);
         U.writeEnum(out, walMode);
         out.writeInt(tlbSize);
-        out.writeInt(walFlushFreq);
-        out.writeInt(walFsyncDelay);
+        out.writeLong(walFlushFreq);
+        out.writeLong(walFsyncDelay);
         out.writeInt(walRecordIterBuffSize);
         out.writeBoolean(alwaysWriteFullPages);
         out.writeInt(subIntervals);
@@ -281,8 +281,8 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
     @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
         persistenceStorePath = U.readString(in);
         checkpointingFreq = in.readLong();
-        lockWaitTime = in.readInt();
-        checkpointingPageBufSize = (Long)in.readObject();
+        lockWaitTime = in.readLong();
+        checkpointingPageBufSize = in.readLong();
         checkpointingThreads = in.readInt();
         walHistSize = in.readInt();
         walSegments = in.readInt();
@@ -292,8 +292,8 @@ public class VisorPersistentStoreConfiguration extends VisorDataTransferObject {
         metricsEnabled = in.readBoolean();
         walMode = WALMode.fromOrdinal(in.readByte());
         tlbSize = in.readInt();
-        walFlushFreq = in.readInt();
-        walFsyncDelay = in.readInt();
+        walFlushFreq = in.readLong();
+        walFsyncDelay = in.readLong();
         walRecordIterBuffSize = in.readInt();
         alwaysWriteFullPages = in.readBoolean();
         subIntervals = in.readInt();

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
index a1a7286..fd0fd34 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgnitePdsWalTlbTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import static org.apache.ignite.configuration.PersistentStoreConfiguration.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
+import static org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager.DFLT_CHECKPOINTING_PAGE_BUFFER_SIZE;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
index 3320dd7..d321639 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/PersistentStoreTest.cs
@@ -59,7 +59,8 @@ namespace Apache.Ignite.Core.Tests.Cache
                     PersistentStorePath = Path.Combine(_tempDir, "Store"),
                     WalStorePath = Path.Combine(_tempDir, "WalStore"),
                     WalArchivePath = Path.Combine(_tempDir, "WalArchive"),
-                    MetricsEnabled = true
+                    MetricsEnabled = true,
+                    CheckpointingPageBufferSize = 1024 * 1024  // TODO: Use default (IGNITE-5717)
                 }
             };
 
@@ -116,7 +117,10 @@ namespace Apache.Ignite.Core.Tests.Cache
         {
             var cfg = new IgniteConfiguration(GetTestConfiguration())
             {
-                PersistentStoreConfiguration = new PersistentStoreConfiguration()
+                PersistentStoreConfiguration = new PersistentStoreConfiguration
+                {
+                    CheckpointingPageBufferSize = 1024 * 1024  // TODO: Use default (IGNITE-5717)
+                }
             };
 
             using (var ignite = Ignition.Start(cfg))

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml
index dd0669a..31fa3b3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/spring-test.xml
@@ -56,5 +56,9 @@
                 </property>
             </bean>
         </property>
+
+        <property name="persistentStoreConfiguration">
+            <bean class="org.apache.ignite.configuration.PersistentStoreConfiguration"/>
+        </property>
     </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 81fd226..93d6af3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -316,6 +316,9 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(MemoryPolicyConfiguration.DefaultMaxSize, plc.MaxSize);
                 Assert.AreEqual(MemoryPolicyConfiguration.DefaultSubIntervals, plc.SubIntervals);
                 Assert.AreEqual(MemoryPolicyConfiguration.DefaultRateTimeInterval, plc.RateTimeInterval);
+
+                // Check PersistentStoreConfiguration defaults.
+                CheckDefaultProperties(resCfg.PersistentStoreConfiguration);
             }
         }
 
@@ -532,8 +535,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(PersistentStoreConfiguration.DefaultTlbSize, cfg.TlbSize);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultCheckpointingFrequency, cfg.CheckpointingFrequency);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultCheckpointingThreads, cfg.CheckpointingThreads);
-            Assert.AreEqual(PersistentStoreConfiguration.DefaultCheckpointingPageBufferSize, 
-                cfg.CheckpointingPageBufferSize);
+            Assert.AreEqual(default(long), cfg.CheckpointingPageBufferSize);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultLockWaitTime, cfg.LockWaitTime);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultWalFlushFrequency, cfg.WalFlushFrequency);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultWalFsyncDelayNanos, cfg.WalFsyncDelayNanos);
@@ -546,6 +548,8 @@ namespace Apache.Ignite.Core.Tests
             Assert.IsFalse(cfg.MetricsEnabled);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultSubIntervals, cfg.SubIntervals);
             Assert.AreEqual(PersistentStoreConfiguration.DefaultRateTimeInterval, cfg.RateTimeInterval);
+            Assert.AreEqual(PersistentStoreConfiguration.DefaultWalStorePath, cfg.WalStorePath);
+            Assert.AreEqual(PersistentStoreConfiguration.DefaultWalArchivePath, cfg.WalArchivePath);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
index 43b17ac..cac6cc8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/PersistentStore/PersistentStoreConfiguration.cs
@@ -30,11 +30,6 @@ namespace Apache.Ignite.Core.PersistentStore
     public class PersistentStoreConfiguration
     {
         /// <summary>
-        /// Default value for <see cref="CheckpointingPageBufferSize"/>.
-        /// </summary>
-        public const long DefaultCheckpointingPageBufferSize = 256L * 1024 * 1024;
-
-        /// <summary>
         /// Default value for <see cref="CheckpointingThreads"/>.
         /// </summary>
         public const int DefaultCheckpointingThreads = 1;
@@ -82,7 +77,7 @@ namespace Apache.Ignite.Core.PersistentStore
         /// <summary>
         /// Default value for <see cref="WalFsyncDelayNanos"/>.
         /// </summary>
-        public const int DefaultWalFsyncDelayNanos = 1;
+        public const long DefaultWalFsyncDelayNanos = 1000;
 
         /// <summary>
         /// The default sub intervals.
@@ -97,11 +92,20 @@ namespace Apache.Ignite.Core.PersistentStore
         public static readonly TimeSpan DefaultRateTimeInterval = TimeSpan.FromSeconds(60);
 
         /// <summary>
+        /// Default value for <see cref="WalStorePath"/>.
+        /// </summary>
+        public const string DefaultWalStorePath = "db/wal";
+
+        /// <summary>
+        /// Default value for <see cref="WalArchivePath"/>.
+        /// </summary>
+        public const string DefaultWalArchivePath = "db/wal/archive";
+
+        /// <summary>
         /// Initializes a new instance of the <see cref="PersistentStoreConfiguration"/> class.
         /// </summary>
         public PersistentStoreConfiguration()
         {
-            CheckpointingPageBufferSize = DefaultCheckpointingPageBufferSize;
             CheckpointingThreads = DefaultCheckpointingThreads;
             CheckpointingFrequency = DefaultCheckpointingFrequency;
             LockWaitTime = DefaultLockWaitTime;
@@ -114,6 +118,8 @@ namespace Apache.Ignite.Core.PersistentStore
             WalFsyncDelayNanos = DefaultWalFsyncDelayNanos;
             RateTimeInterval = DefaultRateTimeInterval;
             SubIntervals = DefaultSubIntervals;
+            WalArchivePath = DefaultWalArchivePath;
+            WalStorePath = DefaultWalStorePath;
         }
 
         /// <summary>
@@ -137,7 +143,7 @@ namespace Apache.Ignite.Core.PersistentStore
             WalMode = (WalMode)reader.ReadInt();
             TlbSize = reader.ReadInt();
             WalFlushFrequency = reader.ReadLongAsTimespan();
-            WalFsyncDelayNanos = reader.ReadInt();
+            WalFsyncDelayNanos = reader.ReadLong();
             WalRecordIteratorBufferSize = reader.ReadInt();
             AlwaysWriteFullPages = reader.ReadBoolean();
             MetricsEnabled = reader.ReadBoolean();
@@ -166,7 +172,7 @@ namespace Apache.Ignite.Core.PersistentStore
             writer.WriteInt((int)WalMode);
             writer.WriteInt(TlbSize);
             writer.WriteTimeSpanAsLong(WalFlushFrequency);
-            writer.WriteInt(WalFsyncDelayNanos);
+            writer.WriteLong(WalFsyncDelayNanos);
             writer.WriteInt(WalRecordIteratorBufferSize);
             writer.WriteBoolean(AlwaysWriteFullPages);
             writer.WriteBoolean(MetricsEnabled);
@@ -188,8 +194,9 @@ namespace Apache.Ignite.Core.PersistentStore
 
         /// <summary>
         /// Gets or sets the size of the checkpointing page buffer.
+        /// <para />
+        /// Default is <c>0</c>: Ignite will choose buffer size automatically.
         /// </summary>
-        [DefaultValue(DefaultCheckpointingPageBufferSize)]
         public long CheckpointingPageBufferSize { get; set; }
 
         /// <summary>
@@ -227,12 +234,14 @@ namespace Apache.Ignite.Core.PersistentStore
         /// <summary>
         /// Gets or sets the path to the directory where WAL (Write Ahead Log) is stored.
         /// </summary>
+        [DefaultValue(DefaultWalStorePath)]
         public string WalStorePath { get; set; }
 
         /// <summary>
         /// Gets or sets the path to the directory where WAL (Write Ahead Log) archive is stored.
         /// Every WAL segment will be fully copied to this directory before it can be reused for WAL purposes.
         /// </summary>
+        [DefaultValue(DefaultWalArchivePath)]
         public string WalArchivePath { get; set; }
 
         /// <summary>
@@ -256,7 +265,7 @@ namespace Apache.Ignite.Core.PersistentStore
         /// Gets or sets the WAL (Write Ahead Log) fsync (disk sync) delay, in nanoseconds
         /// </summary>
         [DefaultValue(DefaultWalFsyncDelayNanos)]
-        public int WalFsyncDelayNanos { get; set; }
+        public long WalFsyncDelayNanos { get; set; }
 
         /// <summary>
         /// Gets or sets the size of the WAL (Write Ahead Log) record iterator buffer, in bytes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/web-console/backend/app/mongo.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js
index aa11e0b..57af928 100644
--- a/modules/web-console/backend/app/mongo.js
+++ b/modules/web-console/backend/app/mongo.js
@@ -1006,7 +1006,7 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
             walSegmentSize: Number,
             walHistorySize: Number,
             walFlushFrequency: Number,
-            walFsyncDelay: Number,
+            walFsyncDelayNanos: Number,
             walRecordIteratorBufferSize: Number,
             lockWaitTime: Number,
             rateTimeInterval: Number,

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
index 8c9b14c..8299b9b 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
@@ -1477,17 +1477,17 @@ export default class IgniteConfigurationGenerator {
             .boolProperty('metricsEnabled')
             .boolProperty('alwaysWriteFullPages')
             .intProperty('checkpointingFrequency')
-            .intProperty('checkpointingPageBufferSize')
+            .longProperty('checkpointingPageBufferSize')
             .intProperty('checkpointingThreads')
             .stringProperty('walStorePath')
             .stringProperty('walArchivePath')
             .intProperty('walSegments')
             .intProperty('walSegmentSize')
             .intProperty('walHistorySize')
-            .intProperty('walFlushFrequency')
-            .intProperty('walFsyncDelay')
+            .longProperty('walFlushFrequency')
+            .longProperty('walFsyncDelayNanos')
             .intProperty('walRecordIteratorBufferSize')
-            .intProperty('lockWaitTime')
+            .longProperty('lockWaitTime')
             .intProperty('rateTimeInterval')
             .intProperty('tlbSize')
             .intProperty('subIntervals');

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
index a4680bf..1c7fc44 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
@@ -340,7 +340,7 @@ const DFLT_CLUSTER = {
         walSegmentSize: 67108864,
         walHistorySize: 20,
         walFlushFrequency: 2000,
-        walFsyncDelay: 1,
+        walFsyncDelayNanos: 1000,
         walRecordIteratorBufferSize: 67108864,
         lockWaitTime: 10000,
         rateTimeInterval: 60000,

http://git-wip-us.apache.org/repos/asf/ignite/blob/689b1b6e/modules/web-console/frontend/app/modules/states/configuration/clusters/persistence.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/persistence.pug b/modules/web-console/frontend/app/modules/states/configuration/clusters/persistence.pug
index 7fd0966..50a569b 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/persistence.pug
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/persistence.pug
@@ -62,7 +62,7 @@ include /app/helpers/jade/mixins
                     +number('WAL flush frequency:', `${model}.walFlushFrequency`, '"PersistenceWalFlushFrequency"', enabled, '2000', '1',
                     'How often will be fsync, in milliseconds. In background mode, exist thread which do fsync by timeout')
                 .settings-row
-                    +number('WAL fsync delay:', `${model}.walFsyncDelay`, '"PersistenceWalFsyncDelay"', enabled, '1', '1', 'WAL fsync delay, in nanoseconds')
+                    +number('WAL fsync delay:', `${model}.walFsyncDelayNanos`, '"PersistenceWalFsyncDelay"', enabled, '1000', '1', 'WAL fsync delay, in nanoseconds')
                 .settings-row
                     +number('WAL record iterator buffer size:', `${model}.walRecordIteratorBufferSize`, '"PersistenceWalRecordIteratorBufferSize"', enabled, '67108864', '1',
                     'How many bytes iterator read from disk(for one reading), during go ahead WAL')


[03/27] ignite git commit: AI 2.1 release notes.

Posted by yz...@apache.org.
AI 2.1 release notes.


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

Branch: refs/heads/ignite-5658
Commit: 5859b192ba28d53e1bccb01ce3005821e26b5347
Parents: f382826
Author: devozerov <vo...@gridgain.com>
Authored: Wed Jul 12 12:46:42 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Jul 12 12:46:42 2017 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 33 +++++++++++++++++++++++++++++++++
 1 file changed, 33 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5859b192/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index b64aced..92286cf 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -1,6 +1,39 @@
 Apache Ignite Release Notes
 ===========================
 
+Apache Ignite In-Memory Data Fabric 2.1
+---------------------------------------
+Ignite:
+* Persistent cache store
+* Added IgniteFuture.listenAsync() and IgniteFuture.chainAsync() mehtods
+* Deprecated IgniteConfiguration.marshaller
+* Updated Lucene dependency to version 5.5.2
+* Machine learning: implemented K-means clusterization algorithm optimized for distributed storages
+* SQL: CREATE TABLE and DROP TABLE commands support
+* SQL: New thin JDBC driver
+* SQL: Improved performance of certain queries, when affinity node can be calculated in advance
+* SQL: Fixed return type of AVG() function
+* SQL: BLOB type support added to thick JDBC driver
+* SQL: Improved LocalDate, LocalTime and LocalDateTime support for Java 8
+* SQL: Added FieldsQueryCursor interface to get fields metadata for SqlFieldsQuery
+* ODBC: Implemented DML statement batching
+* Massive performance and stability improvements
+
+Ignite.NET:
+* Automatic remote assembly loading
+* NuGet-based standalone node deployment
+* Added conditional data removeal via LINQ DeleteAll
+* Added TimestampAttribute to control DateTime serialization mode
+* Added local collections joins support to LINQ.
+
+Ignite CPP:
+* Added Compute::Call and Compute::Broadcast methods
+
+Web Console:
+* Implemented support for UNIQUE indexes for key fields on import model from RDBMS
+* Added option to show full stack trace on Queries screen
+* Added PK alias generation on Models screen.
+
 Apache Ignite In-Memory Data Fabric 2.0
 ---------------------------------------
 Ignite:


[08/27] ignite git commit: Optimized snapshot progress tracking

Posted by yz...@apache.org.
Optimized snapshot progress tracking


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

Branch: refs/heads/ignite-5658
Commit: 21964fb5f6fb6fee891283332202cbc9ed5ac3f3
Parents: 3787181
Author: Dmitry Pavlov <dp...@gmail.com>
Authored: Wed Jul 12 18:59:10 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Jul 12 18:59:10 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/pagemem/FullPageId.java     |   6 +-
 .../ignite/internal/pagemem/PageIdUtils.java    |  14 +-
 .../pagemem/store/IgnitePageStoreManager.java   |   1 +
 .../internal/pagemem/store/PageStore.java       |   2 +
 .../delta/MetaPageUpdateLastAllocatedIndex.java |   2 +-
 .../cache/persistence/DbCheckpointListener.java |   7 +-
 .../FullPageIdIterableComparator.java           |  51 -------
 .../GridCacheDatabaseSharedManager.java         |  63 +++++---
 .../persistence/GridCacheOffheapManager.java    |  56 +++----
 .../cache/persistence/file/FilePageStore.java   |   2 +-
 .../persistence/file/FilePageStoreManager.java  |   1 +
 .../cache/persistence/pagemem/PageMemoryEx.java |   8 +-
 .../persistence/pagemem/PageMemoryImpl.java     |  10 +-
 .../persistence/partstate/GroupPartitionId.java | 145 +++++++++++++++++++
 .../partstate/PagesAllocationRange.java         |  68 +++++++++
 .../partstate/PartitionAllocationMap.java       | 113 +++++++++++++++
 .../snapshot/IgniteCacheSnapshotManager.java    |  17 ++-
 .../cache/persistence/tree/io/PageMetaIO.java   |  27 ++--
 .../persistence/tree/io/TrackingPageIO.java     |   8 +-
 .../persistence/tree/util/PageHandler.java      |   3 +-
 20 files changed, 466 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java
index 00f52c1..9e24943 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/FullPageId.java
@@ -21,7 +21,7 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
-  * Compound object used to address a page in the global page space.
+ * Compound object used to address a page in the global page space.
  * <h3>Page ID structure</h3>
  * <p>
  * Generally, a full page ID consists of a cache ID and page ID. A page ID consists of
@@ -49,13 +49,13 @@ import org.apache.ignite.internal.util.typedef.internal.U;
  * Effective page ID is page ID with zeroed bits used for page ID rotation.
  */
 public class FullPageId {
-    /** */
+    /** Page ID. */
     private final long pageId;
 
     /** */
     private final long effectivePageId;
 
-    /** */
+    /** Cache group ID. */
     private final int grpId;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageIdUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageIdUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageIdUtils.java
index 92f427a..6f4ba93 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageIdUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageIdUtils.java
@@ -49,10 +49,10 @@ public final class PageIdUtils {
     /** */
     public static final long TAG_MASK = ~(-1L << TAG_SIZE);
 
-    /** */
+    /** Page Index is a monotonically growing number within each partition */
     public static final long PART_ID_MASK = ~(-1L << PART_ID_SIZE);
 
-    /** */
+    /** Flags mask. Flags consists from a number of reserved bits, and page type (data/index page) */
     public static final long FLAG_MASK = ~(-1L << FLAG_SIZE);
 
     /** */
@@ -92,10 +92,10 @@ public final class PageIdUtils {
     }
 
     /**
-     * Extracts a page index from the given pageId.
+     * Extracts a page index from the given page ID.
      *
-     * @param pageId Page id.
-     * @return Page ID.
+     * @param pageId Page ID.
+     * @return Page index.
      */
     public static int pageIndex(long pageId) {
         return (int)(pageId & PAGE_IDX_MASK); // 4 bytes
@@ -150,7 +150,9 @@ public final class PageIdUtils {
 
     /**
      * @param partId Partition ID.
-     * @return Part ID constructed from the given cache ID and partition ID.
+     * @param flag Flags (a number of reserved bits, and page type (data/index page))
+     * @param pageIdx Page index, monotonically growing number within each partition
+     * @return Page ID constructed from the given pageIdx and partition ID, see {@link FullPageId}
      */
     public static long pageId(int partId, byte flag, int pageIdx) {
         long pageId = flag & FLAG_MASK;

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
index a1b766f..eaa85ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
@@ -77,6 +77,7 @@ public interface IgnitePageStoreManager extends GridCacheSharedManager, IgniteCh
      *
      * @param grpId Cache group ID of the evicted partition.
      * @param partId Partition ID.
+     * @param tag Partition tag (growing 1-based partition file version).
      * @throws IgniteCheckedException If failed to handle partition destroy callback.
      */
     public void onPartitionDestroyed(int grpId, int partId, int tag) throws IgniteCheckedException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java
index be83704..4698a6b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/PageStore.java
@@ -70,6 +70,8 @@ public interface PageStore {
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to write.
+     * @param tag Partition file version, 1-based incrementing counter. For outdated pages {@code tag} has lower value,
+     * and write does nothing
      * @throws IgniteCheckedException If page writing failed (IO error occurred).
      */
     public void write(long pageId, ByteBuffer pageBuf, int tag) throws IgniteCheckedException;

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
index 60aebde..11b2a67 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
@@ -44,7 +44,7 @@ public class MetaPageUpdateLastAllocatedIndex extends PageDeltaRecord {
 
         PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(pageAddr));
 
-        io.setLastPageCount(pageAddr, lastAllocatedIdx);
+        io.setLastAllocatedPageCount(pageAddr, lastAllocatedIdx);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java
index 0b28b6a..1c438b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DbCheckpointListener.java
@@ -17,9 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.persistence;
 
-import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap;
 
 /**
  *
@@ -35,9 +34,9 @@ public interface DbCheckpointListener {
         public boolean nextSnapshot();
 
         /**
-         *
+         * @return Partition allocation statistic map
          */
-        public Map<T2<Integer, Integer>, T2<Integer, Integer>> partitionStatMap();
+        public PartitionAllocationMap partitionStatMap();
 
         /**
          * @param cacheOrGrpName Cache or group name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FullPageIdIterableComparator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FullPageIdIterableComparator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FullPageIdIterableComparator.java
deleted file mode 100644
index c056c52..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/FullPageIdIterableComparator.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.persistence;
-
-import java.io.Serializable;
-import java.util.Comparator;
-import org.apache.ignite.internal.util.typedef.T2;
-
-/**
- *
- */
-public class FullPageIdIterableComparator implements Comparator<T2<Integer, Integer>>, Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    public static final FullPageIdIterableComparator INSTANCE = new FullPageIdIterableComparator();
-
-    /** {@inheritDoc} */
-    @Override public int compare(T2<Integer, Integer> o1, T2<Integer, Integer> o2) {
-        if (o1.get1() < o2.get1())
-            return -1;
-
-        if (o1.get1() > o2.get1())
-            return 1;
-
-        if (o1.get2() < o2.get2())
-            return -1;
-
-        if (o1.get2() > o2.get2())
-            return 1;
-
-        return 0;
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 5136731..9f2067a 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -40,7 +40,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
-import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -52,6 +51,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.regex.Matcher;
@@ -109,6 +109,7 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStor
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.CheckpointMetricsTracker;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryImpl;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperation;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
@@ -263,8 +264,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** */
     private boolean stopping;
 
-    /** Checkpoint runner thread pool. */
-    private ExecutorService asyncRunner;
+    /** Checkpoint runner thread pool. If null tasks are to be run in single thread */
+    @Nullable private ExecutorService asyncRunner;
 
     /** Buffer for the checkpoint threads. */
     private ThreadLocal<ByteBuffer> threadBuf;
@@ -1916,6 +1917,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             asyncRunner == null ? 1 : chp.cpPages.collectionsSize());
 
                         tracker.onPagesWriteStart();
+                        final AtomicInteger writtenPagesCtr = new AtomicInteger();
+                        final int totalPagesToWriteCnt = chp.cpPages.size();
 
                         if (asyncRunner != null) {
                             for (int i = 0; i < chp.cpPages.collectionsSize(); i++) {
@@ -1923,7 +1926,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                                     tracker,
                                     chp.cpPages.innerCollection(i),
                                     updStores,
-                                    doneWriteFut
+                                    doneWriteFut,
+                                    writtenPagesCtr,
+                                    totalPagesToWriteCnt
                                 );
 
                                 try {
@@ -1937,7 +1942,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                         }
                         else {
                             // Single-threaded checkpoint.
-                            Runnable write = new WriteCheckpointPages(tracker, chp.cpPages, updStores, doneWriteFut);
+                            Runnable write = new WriteCheckpointPages(tracker,
+                                chp.cpPages,
+                                updStores,
+                                doneWriteFut,
+                                writtenPagesCtr,
+                                totalPagesToWriteCnt);
 
                             write.run();
                         }
@@ -2092,15 +2102,15 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     curCpProgress = curr;
                 }
 
-                final NavigableMap<T2<Integer, Integer>, T2<Integer, Integer>> map =
-                    new TreeMap<>(FullPageIdIterableComparator.INSTANCE);
+                final PartitionAllocationMap map = new PartitionAllocationMap();
 
                 DbCheckpointListener.Context ctx0 = new DbCheckpointListener.Context() {
                     @Override public boolean nextSnapshot() {
                         return curr.nextSnapshot;
                     }
 
-                    @Override public Map<T2<Integer, Integer>, T2<Integer, Integer>> partitionStatMap() {
+                    /** {@inheritDoc} */
+                    @Override public PartitionAllocationMap partitionStatMap() {
                         return map;
                     }
 
@@ -2278,14 +2288,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
     }
 
-    /**
-     *
-     */
+    /** Pages write task */
     private class WriteCheckpointPages implements Runnable {
         /** */
         private CheckpointMetricsTracker tracker;
 
-        /** */
+        /** Collection of page IDs to write under this task. Overall pages to write may be greater than this collection*/
         private Collection<FullPageId> writePageIds;
 
         /** */
@@ -2294,19 +2302,34 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         /** */
         private CountDownFuture doneFut;
 
+        /** Counter for all written pages. May be shared between several workers */
+        private AtomicInteger writtenPagesCntr;
+
+        /** Total pages to write, counter may be greater than {@link #writePageIds} size*/
+        private final int totalPagesToWrite;
+
         /**
-         * @param writePageIds Write page IDs.
+         * Creates task for write pages
+         * @param tracker
+         * @param writePageIds Collection of page IDs to write.
+         * @param updStores
+         * @param doneFut
+         * @param writtenPagesCntr all written pages counter, may be shared between several write tasks
+         * @param totalPagesToWrite total pages to be written under this checkpoint
          */
         private WriteCheckpointPages(
-            CheckpointMetricsTracker tracker,
-            Collection<FullPageId> writePageIds,
-            GridConcurrentHashSet<PageStore> updStores,
-            CountDownFuture doneFut
-        ) {
+            final CheckpointMetricsTracker tracker,
+            final Collection<FullPageId> writePageIds,
+            final GridConcurrentHashSet<PageStore> updStores,
+            final CountDownFuture doneFut,
+            @NotNull final AtomicInteger writtenPagesCntr,
+            final int totalPagesToWrite) {
             this.tracker = tracker;
             this.writePageIds = writePageIds;
             this.updStores = updStores;
             this.doneFut = doneFut;
+            this.writtenPagesCntr = writtenPagesCntr;
+            this.totalPagesToWrite = totalPagesToWrite;
         }
 
         /** {@inheritDoc} */
@@ -2354,7 +2377,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             tmpWriteBuf.rewind();
                         }
 
-                        snapshotMgr.onPageWrite(fullId, tmpWriteBuf);
+                        int curWrittenPages = writtenPagesCntr.incrementAndGet();
+
+                        snapshotMgr.onPageWrite(fullId, tmpWriteBuf, curWrittenPages, totalPagesToWrite);
 
                         tmpWriteBuf.rewind();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index bd902fb..6e6b7df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -52,6 +52,9 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeListImpl;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PagesAllocationRange;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionCountersIO;
@@ -63,7 +66,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.lang.GridCursor;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -207,9 +209,9 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                 long partMetaPage = pageMem.acquirePage(grpId, partMetaId);
 
                 try {
-                    long pageAddr = pageMem.writeLock(grpId, partMetaId, partMetaPage);
+                    long partMetaPageAddr = pageMem.writeLock(grpId, partMetaId, partMetaPage);
 
-                    if (pageAddr == 0L) {
+                    if (partMetaPageAddr == 0L) {
                         U.warn(log, "Failed to acquire write lock for meta page [metaPage=" + partMetaPage +
                             ", saveMeta=" + saveMeta + ", beforeDestroy=" + beforeDestroy + ", size=" + size +
                             ", updCntr=" + updCntr + ", state=" + state + ']');
@@ -220,21 +222,21 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                     boolean changed = false;
 
                     try {
-                        PagePartitionMetaIO io = PageIO.getPageIO(pageAddr);
+                        PagePartitionMetaIO io = PageIO.getPageIO(partMetaPageAddr);
 
-                        changed |= io.setUpdateCounter(pageAddr, updCntr);
-                        changed |= io.setGlobalRemoveId(pageAddr, rmvId);
-                        changed |= io.setSize(pageAddr, size);
+                        changed |= io.setUpdateCounter(partMetaPageAddr, updCntr);
+                        changed |= io.setGlobalRemoveId(partMetaPageAddr, rmvId);
+                        changed |= io.setSize(partMetaPageAddr, size);
 
                         if (state != null)
-                            changed |= io.setPartitionState(pageAddr, (byte)state.ordinal());
+                            changed |= io.setPartitionState(partMetaPageAddr, (byte)state.ordinal());
                         else
                             assert grp.isLocal() : grp.cacheOrGroupName();
 
                         long cntrsPageId;
 
                         if (grp.sharedGroup()) {
-                            cntrsPageId = io.getCountersPageId(pageAddr);
+                            cntrsPageId = io.getCountersPageId(partMetaPageAddr);
 
                             byte[] data = serializeCacheSizes(store.cacheSizes());
 
@@ -247,7 +249,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                             if (init && items > 0) {
                                 cntrsPageId = pageMem.allocatePage(grpId, store.partId(), PageIdAllocator.FLAG_DATA);
 
-                                io.setCountersPageId(pageAddr, cntrsPageId);
+                                io.setCountersPageId(partMetaPageAddr, cntrsPageId);
 
                                 changed = true;
                             }
@@ -301,7 +303,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
                         if (needSnapshot) {
                             pageCnt = this.ctx.pageStore().pages(grpId, store.partId());
-                            io.setCandidatePageCount(pageAddr, pageCnt);
+                            io.setCandidatePageCount(partMetaPageAddr, pageCnt);
 
                             if (saveMeta) {
                                 long metaPageId = pageMem.metaPageId(grpId);
@@ -345,13 +347,13 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
                             if (partMap.containsKey(store.partId()) &&
                                 partMap.get(store.partId()) == GridDhtPartitionState.OWNING)
-                                addPartition(ctx.partitionStatMap(), pageAddr, io, grpId, store.partId(),
+                                addPartition(ctx.partitionStatMap(), partMetaPageAddr, io, grpId, store.partId(),
                                     this.ctx.pageStore().pages(grpId, store.partId()));
 
                             changed = true;
                         }
                         else
-                            pageCnt = io.getCandidatePageCount(pageAddr);
+                            pageCnt = io.getCandidatePageCount(partMetaPageAddr);
 
                         if (PageHandler.isWalDeltaRecordNeeded(pageMem, grpId, partMetaId, partMetaPage, wal, null))
                             wal.log(new MetaPageUpdatePartitionDataRecord(
@@ -397,27 +399,29 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
 
     /**
      * @param map Map to add values to.
-     * @param pageAddr page address
+     * @param metaPageAddr Meta page address
      * @param io Page Meta IO
      * @param cacheId Cache ID.
-     * @param partition Partition ID.
-     * @param pages Number of pages to add.
+     * @param partId Partition ID. Or {@link PageIdAllocator#INDEX_PARTITION} for index partition
+     * @param currAllocatedPageCnt total number of pages allocated for partition <code>[partition, cacheId]</code>
      */
     private static void addPartition(
-        Map<T2<Integer, Integer>, T2<Integer, Integer>> map,
-        long pageAddr,
-        PageMetaIO io,
-        int cacheId,
-        int partition,
-        int pages
+        final PartitionAllocationMap map,
+        final long metaPageAddr,
+        final PageMetaIO io,
+        final int cacheId,
+        final int partId,
+        final int currAllocatedPageCnt
     ) {
-        if (pages <= 1)
+        if (currAllocatedPageCnt <= 1)
             return;
 
-        assert PageIO.getPageId(pageAddr) != 0;
+        assert PageIO.getPageId(metaPageAddr) != 0;
 
-        int lastAllocatedIdx = io.getLastPageCount(pageAddr);
-        map.put(new T2<>(cacheId, partition), new T2<>(lastAllocatedIdx, pages));
+        int lastAllocatedPageCnt = io.getLastAllocatedPageCount(metaPageAddr);
+        map.put(
+            new GroupPartitionId(cacheId, partId),
+            new PagesAllocationRange(lastAllocatedPageCnt, currAllocatedPageCnt));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
index c827e96..a7ca13c 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStore.java
@@ -77,7 +77,7 @@ public class FilePageStore implements PageStore {
     /** */
     private volatile boolean recover;
 
-    /** */
+    /** Partition file version, 1-based incrementing counter. For outdated pages tag has low value, and write does nothing */
     private volatile int tag;
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index af20136..e2ad070 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -318,6 +318,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
      * @param cacheId Cache ID to write.
      * @param pageId Page ID.
      * @param pageBuf Page buffer.
+     * @param tag Partition tag (growing 1-based partition file version). Used to validate page is not outdated
      * @return PageStore to which the page has been written.
      * @throws IgniteCheckedException If IO error occurred.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
index 7c63d41..53e21b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
@@ -115,19 +115,19 @@ public interface PageMemoryEx extends PageMemory {
      *
      * @param pageId Page ID to get byte buffer for. The page ID must be present in the collection returned by
      *      the {@link #beginCheckpoint()} method call.
-     * @param tmpBuf Temporary buffer to write changes into.
+     * @param outBuf Temporary buffer to write changes into.
      * @param tracker Checkpoint metrics tracker.
-     * @return {@code True} if data were read, {@code false} otherwise (data already saved to storage).
+     * @return {@code Partition tag} if data was read, {@code null} otherwise (data already saved to storage).
      * @throws IgniteException If failed to obtain page data.
      */
-    @Nullable public Integer getForCheckpoint(FullPageId pageId, ByteBuffer tmpBuf, CheckpointMetricsTracker tracker);
+    @Nullable public Integer getForCheckpoint(FullPageId pageId, ByteBuffer outBuf, CheckpointMetricsTracker tracker);
 
     /**
      * Marks partition as invalid / outdated.
      *
      * @param cacheId Cache ID.
      * @param partId Partition ID.
-     * @return New partition tag.
+     * @return New partition tag (growing 1-based partition file version).
      */
     public int invalidate(int cacheId, int partId);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 47381d7..1b4cf81 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -792,8 +792,8 @@ public class PageMemoryImpl implements PageMemoryEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Integer getForCheckpoint(FullPageId fullId, ByteBuffer tmpBuf, CheckpointMetricsTracker tracker) {
-        assert tmpBuf.remaining() == pageSize();
+    @Override public Integer getForCheckpoint(FullPageId fullId, ByteBuffer outBuf, CheckpointMetricsTracker tracker) {
+        assert outBuf.remaining() == pageSize();
 
         Segment seg = segment(fullId.groupId(), fullId.pageId());
 
@@ -876,7 +876,7 @@ public class PageMemoryImpl implements PageMemoryEx {
             }
         }
         else
-            return copyPageForCheckpoint(absPtr, fullId, tmpBuf, tmpBuffer, tracker) ? tag : null;
+            return copyPageForCheckpoint(absPtr, fullId, outBuf, tmpBuffer, tracker) ? tag : null;
     }
 
     /**
@@ -1565,7 +1565,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         /** */
         private final int maxDirtyPages;
 
-        /** */
+        /** Maps partition (cacheId, partId) to its tag. Tag is 1-based incrementing partition file counter */
         private final Map<T2<Integer, Integer>, Integer> partTagMap = new HashMap<>();
 
         /**
@@ -1903,7 +1903,7 @@ public class PageMemoryImpl implements PageMemoryEx {
         /**
          * @param grpId Cache group ID.
          * @param partId Partition ID.
-         * @return Partition tag.
+         * @return Partition tag. Growing 1 based partition file version
          */
         private int partTag(int grpId, int partId) {
             assert getReadHoldCount() > 0 || getWriteHoldCount() > 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/GroupPartitionId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/GroupPartitionId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/GroupPartitionId.java
new file mode 100644
index 0000000..dbdf670
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/GroupPartitionId.java
@@ -0,0 +1,145 @@
+/*
+ * 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.persistence.partstate;
+
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Pair of cache group ID with partition ID. Immutable, comparable class, may be used as key in maps
+ */
+public class GroupPartitionId implements Comparable<GroupPartitionId> {
+    /** Index for super(meta) page. There is always such page for iterated cache partition  */
+    private static final int METAPAGE_IDX = 0;
+
+    /** Cache group ID. */
+    private final int grpId;
+
+    /** Partition ID. */
+    private final int partId;
+
+    /**
+     * Creates group-partition tuple.
+     *
+     * @param grpId Group ID.
+     * @param partId Partition ID.
+     */
+    public GroupPartitionId(final int grpId, final int partId) {
+        this.grpId = grpId;
+        this.partId = partId;
+    }
+
+    /**
+     * @param partId Partition ID.
+     * @return flag to be used for partition
+     */
+    private static byte getFlagByPartId(final int partId) {
+        return partId == PageIdAllocator.INDEX_PARTITION ? PageMemory.FLAG_IDX : PageMemory.FLAG_DATA;
+    }
+
+    /**
+     * @return cache ID
+     */
+    public int getGroupId() {
+        return grpId;
+    }
+
+    /**
+     * @return Partition ID
+     */
+    public int getPartitionId() {
+        return partId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GroupPartitionId.class, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        GroupPartitionId key = (GroupPartitionId)o;
+
+        if (grpId != key.grpId)
+            return false;
+
+        return partId == key.partId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = grpId;
+
+        result = 31 * result + partId;
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compareTo(@NotNull GroupPartitionId o) {
+        if (getGroupId() < o.getGroupId())
+            return -1;
+
+        if (getGroupId() > o.getGroupId())
+            return 1;
+
+        if (getPartitionId() < o.getPartitionId())
+            return -1;
+
+        if (getPartitionId() > o.getPartitionId())
+            return 1;
+        return 0;
+    }
+
+    /**
+     * @param pageIdx Page Index, monotonically growing number within each partition
+     * @return page ID (64 bits) constructed from partition ID and given index
+     */
+    private long createPageId(final int pageIdx) {
+        final int partId = getPartitionId();
+
+        return PageIdUtils.pageId(partId, getFlagByPartId(partId), pageIdx);
+    }
+
+    /**
+     * Returns Full page ID. For index 0 will return super-page of next partition
+     *
+     * @param pageIdx Page Index, monotonically growing number within each partition
+     * @return FullPageId consists of cache ID (32 bits) and page ID (64 bits).
+     */
+    @NotNull private FullPageId createFullPageId(final int pageIdx) {
+        return new FullPageId(createPageId(pageIdx), getGroupId());
+    }
+
+    /**
+     * @return will return super-page (metapage) of this partition
+     */
+    @NotNull public FullPageId createFirstPageFullId() {
+        return createFullPageId(METAPAGE_IDX);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PagesAllocationRange.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PagesAllocationRange.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PagesAllocationRange.java
new file mode 100644
index 0000000..e7170c3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PagesAllocationRange.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.partstate;
+
+/**
+ * Range of pages allocated.
+ * Contains currently allocated page count and previously observed page count.
+ * May be used for tracking history of recent allocation for partition <code>[partition, cacheId]</code>
+ */
+public class PagesAllocationRange {
+    /**
+     * Previously observed total number of allocated pages. May be stored using PageMetaIO.
+     * Used to separate newly allocated pages with previously observed state
+     * Minimum value is 0. Can't be greater than {@link #currAllocatedPageCnt}
+     */
+    private final int lastAllocatedPageCnt;
+
+    /** Total current number of pages allocated, minimum value is 0. */
+    private final int currAllocatedPageCnt;
+
+    /**
+     * Creates pages range
+     *
+     * @param lastAllocatedPageCnt Last allocated pages count.
+     * @param currAllocatedPageCnt Currently allocated pages count.
+     */
+    public PagesAllocationRange(final int lastAllocatedPageCnt, final int currAllocatedPageCnt) {
+        this.lastAllocatedPageCnt = lastAllocatedPageCnt;
+        this.currAllocatedPageCnt = currAllocatedPageCnt;
+    }
+
+    /**
+     * @return Total current number of pages allocated, minimum value is 0.
+     */
+    public int getCurrAllocatedPageCnt() {
+        return currAllocatedPageCnt;
+    }
+
+    /**
+     * @return Previously observed total number of allocated pages.
+     */
+    public int getLastAllocatedPageCnt() {
+        return lastAllocatedPageCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "PagesAllocationRange{" +
+            "lastAllocatedPageCnt=" + lastAllocatedPageCnt +
+            ", currAllocatedPageCnt=" + currAllocatedPageCnt +
+            '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PartitionAllocationMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PartitionAllocationMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PartitionAllocationMap.java
new file mode 100644
index 0000000..9ed4000
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstate/PartitionAllocationMap.java
@@ -0,0 +1,113 @@
+/*
+ * 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.persistence.partstate;
+
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Information structure with partitions state.
+ * Page counts map.
+ */
+public class PartitionAllocationMap {
+    /** Maps following pairs: (groupId, partId) -> (lastAllocatedCount, allocatedCount) */
+    private final NavigableMap<GroupPartitionId, PagesAllocationRange> map = new TreeMap<>();
+
+    /**
+     * Returns the value to which the specified key is mapped,
+     * or {@code null} if this map contains no mapping for the key.
+     *
+     * @param key to get
+     * @return value or null
+     */
+    @Nullable public PagesAllocationRange get(GroupPartitionId key) {
+        return map.get(key);
+    }
+
+    /**
+     * Extracts partition information from full page ID
+     *
+     * @param fullId page related to some cache
+     * @return pair of cache ID and partition ID
+     */
+    @NotNull public static GroupPartitionId createCachePartId(@NotNull final FullPageId fullId) {
+        return new GroupPartitionId(fullId.groupId(), PageIdUtils.partId(fullId.pageId()));
+    }
+
+    /** @return <tt>true</tt> if this map contains no key-value mappings */
+    public boolean isEmpty() {
+        return map.isEmpty();
+    }
+
+    /** @return the number of key-value mappings in this map. */
+    public int size() {
+        return map.size();
+    }
+
+    /** @return keys (all caches partitions) */
+    public Set<GroupPartitionId> keySet() {
+        return map.keySet();
+    }
+
+    /** @return values (allocation ranges) */
+    public Iterable<PagesAllocationRange> values() {
+        return map.values();
+    }
+
+    /** @return Returns the first (lowest) key currently in this map. */
+    public GroupPartitionId firstKey() {
+        return map.firstKey();
+    }
+
+    /**
+     * Returns next (higher) key for provided cache and partition or null
+     *
+     * @param key cache and partition to search
+     * @return first found key which is greater than provided
+     */
+    @Nullable public GroupPartitionId nextKey(@NotNull final GroupPartitionId key) {
+        return map.navigableKeySet().higher(key);
+    }
+
+    /** @return set view of the mappings contained in this map, sorted in ascending key order */
+    public Set<Map.Entry<GroupPartitionId, PagesAllocationRange>> entrySet() {
+        return map.entrySet();
+    }
+
+    /** @return <tt>true</tt> if this map contains a mapping for the specified key */
+    public boolean containsKey(GroupPartitionId key) {
+        return map.containsKey(key);
+    }
+
+    /**
+     * @param key key with which the specified value is to be associated
+     * @param val value to be associated with the specified key
+     * @return the previous value associated with <tt>key</tt>, or <tt>null</tt> if there was no mapping for
+     * <tt>key</tt>.
+     */
+    public PagesAllocationRange put(GroupPartitionId key, PagesAllocationRange val) {
+        return map.put(key, val);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java
index 0a27bcd..50e6515 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteCacheSnapshotManager.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.persistence.snapshot;
 
 import java.nio.ByteBuffer;
-import java.util.NavigableMap;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -29,8 +28,8 @@ import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.partstate.PartitionAllocationMap;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -64,12 +63,13 @@ public class IgniteCacheSnapshotManager<T extends SnapshotOperation> extends Gri
 
     /**
      * @param snapshotOperation current snapshot operation.
+     * @param map  (cacheId, partId) -> (lastAllocatedIndex, count)
      *
      * @return {@code true} if next operation must be snapshot, {@code false} if checkpoint must be executed.
      */
     public boolean onMarkCheckPointBegin(
         T snapshotOperation,
-        NavigableMap<T2<Integer, Integer>, T2<Integer, Integer>> map
+        PartitionAllocationMap map
     ) throws IgniteCheckedException {
         return false;
     }
@@ -107,9 +107,16 @@ public class IgniteCacheSnapshotManager<T extends SnapshotOperation> extends Gri
     }
 
     /**
-     * @param fullId Full id.
+     * @param fullId Full page id.
+     * @param tmpWriteBuf buffer
+     * @param writtenPages Overall pages written, negative value means there is no progress tracked
+     * @param totalPages Overall pages count to be written, should be positive
      */
-    public void onPageWrite(FullPageId fullId, ByteBuffer tmpWriteBuf) {
+    public void onPageWrite(
+        final FullPageId fullId,
+        final ByteBuffer tmpWriteBuf,
+        final int writtenPages,
+        final int totalPages) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java
index ac482e8..becd3e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageMetaIO.java
@@ -43,10 +43,10 @@ public class PageMetaIO extends PageIO {
     /** Last successful full snapshot tag offset. */
     private static final int LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF = NEXT_SNAPSHOT_TAG_OFF + 8;
 
-    /** Last allocated index offset. */
+    /** Last allocated pages count offset. */
     private static final int LAST_PAGE_COUNT_OFF = LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF + 8;
 
-    /** Candidate allocated index offset. */
+    /** Candidate allocated page count offset. */
     private static final int CANDIDATE_PAGE_COUNT_OFF = LAST_PAGE_COUNT_OFF + 4;
 
     /** End of page meta. */
@@ -82,7 +82,7 @@ public class PageMetaIO extends PageIO {
         setLastSuccessfulSnapshotId(pageAddr, 0);
         setNextSnapshotTag(pageAddr, 1);
         setLastSuccessfulSnapshotTag(pageAddr, 0);
-        setLastPageCount(pageAddr, 0);
+        setLastAllocatedPageCount(pageAddr, 0);
         setCandidatePageCount(pageAddr, 0);
     }
 
@@ -179,24 +179,31 @@ public class PageMetaIO extends PageIO {
     }
 
     /**
-     * @param pageAddr Page address.
-     * @param pageCnt Last allocated index.
+     * Sets last allocated pages count, used to save and observe previous allocated count
+     *
+     * @param pageAddr Meta Page address.
+     * @param pageCnt Last allocated pages count to set
      */
-    public void setLastPageCount(long pageAddr, int pageCnt) {
+    public void setLastAllocatedPageCount(final long pageAddr, final int pageCnt) {
         PageUtils.putInt(pageAddr, LAST_PAGE_COUNT_OFF, pageCnt);
     }
 
     /**
-     * @param buf Buffer.
+     * Gets last allocated pages count from given buffer
+     *
+     * @param buf Buffer to read data from.
      */
-    public int getLastPageCount(@NotNull ByteBuffer buf) {
+    public int getLastAllocatedPageCount(@NotNull final ByteBuffer buf) {
         return buf.getInt(LAST_PAGE_COUNT_OFF);
     }
 
     /**
-     * @param pageAddr Page address.
+     * Gets last allocated pages count by provided address
+     *
+     * @param pageAddr Meta page address.
+     * @return Last allocated page count
      */
-    public int getLastPageCount(long pageAddr) {
+    public int getLastAllocatedPageCount(final long pageAddr) {
         return PageUtils.getInt(pageAddr, LAST_PAGE_COUNT_OFF);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java
index 2263130..2051778 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/TrackingPageIO.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.persistence.tree.io;
 import java.nio.ByteBuffer;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * We use dedicated page for tracking pages updates.
@@ -182,6 +183,7 @@ public class TrackingPageIO extends PageIO {
      * @param buf Buffer.
      * @param pageId Page id.
      * @param curSnapshotTag Snapshot tag.
+     * @param lastSuccessfulSnapshotTag Last successful snapshot id.
      * @param pageSize Page size.
      */
     public boolean wasChanged(ByteBuffer buf, long pageId, long curSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
@@ -265,10 +267,12 @@ public class TrackingPageIO extends PageIO {
      * @param buf Buffer.
      * @param start Start.
      * @param curSnapshotTag Snapshot id.
+     * @param lastSuccessfulSnapshotTag  Last successful snapshot id.
      * @param pageSize Page size.
-     * @return set pageId if it was changed or next closest one, if there is no changed page null will be returned
+     * @return set pageId if it was changed or next closest one, if there is no changed page {@code null} will be returned
      */
-    public Long findNextChangedPage(ByteBuffer buf, long start, long curSnapshotTag, long lastSuccessfulSnapshotTag, int pageSize) {
+    @Nullable public Long findNextChangedPage(ByteBuffer buf, long start, long curSnapshotTag,
+        long lastSuccessfulSnapshotTag, int pageSize) {
         validateSnapshotId(buf, curSnapshotTag + 1, lastSuccessfulSnapshotTag, pageSize);
 
         int cntOfPage = countOfPageToTrack(pageSize);

http://git-wip-us.apache.org/repos/asf/ignite/blob/21964fb5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java
index a87525a..3316980 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.persistence.tree.util;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageSupport;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
@@ -440,7 +441,7 @@ public abstract class PageHandler<X, R> {
      * @return {@code true} If we need to make a delta WAL record for the change in this page.
      */
     public static boolean isWalDeltaRecordNeeded(
-        PageMemory pageMem,
+        PageSupport pageMem,
         int cacheId,
         long pageId,
         long page,


[05/27] ignite git commit: Fixed page acquire during checkpoint

Posted by yz...@apache.org.
Fixed page acquire during checkpoint


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

Branch: refs/heads/ignite-5658
Commit: c6ee085b8a1321ce7fa15f8adf74fa7a01f7a445
Parents: 8afdc7b
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Wed Jul 12 14:22:03 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Jul 12 14:22:03 2017 +0300

----------------------------------------------------------------------
 .../persistence/pagemem/PageMemoryImpl.java     | 30 ++++++++++++++++----
 1 file changed, 24 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c6ee085b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index e4428a2..47381d7 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -803,9 +803,14 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         int tag;
 
+        boolean tmpBuffer = false;
+
         seg.readLock().lock();
 
         try {
+            if (!isInCheckpoint(fullId))
+                return null;
+
             tag = seg.partTag(fullId.groupId(), PageIdUtils.partId(fullId.pageId()));
 
             relPtr = seg.loadedPages.get(
@@ -826,6 +831,8 @@ public class PageMemoryImpl implements PageMemoryEx {
                 // Pin the page until page will not be copied.
                 if (PageHeader.tempBufferPointer(absPtr) == INVALID_REL_PTR)
                     PageHeader.acquirePage(absPtr);
+                else
+                    tmpBuffer = true;
             }
         }
         finally {
@@ -869,7 +876,7 @@ public class PageMemoryImpl implements PageMemoryEx {
             }
         }
         else
-            return copyPageForCheckpoint(absPtr, fullId, tmpBuf, tracker) ? tag : null;
+            return copyPageForCheckpoint(absPtr, fullId, tmpBuf, tmpBuffer, tracker) ? tag : null;
     }
 
     /**
@@ -877,19 +884,24 @@ public class PageMemoryImpl implements PageMemoryEx {
      * @param fullId Full id.
      * @param tmpBuf Tmp buffer.
      */
-    private boolean copyPageForCheckpoint(long absPtr, FullPageId fullId, ByteBuffer tmpBuf, CheckpointMetricsTracker tracker) {
+    private boolean copyPageForCheckpoint(
+        long absPtr,
+        FullPageId fullId,
+        ByteBuffer tmpBuf,
+        boolean tmpBuffer,
+        CheckpointMetricsTracker tracker
+    ) {
         assert absPtr != 0;
+        assert PageHeader.isAcquired(absPtr);
 
         rwLock.writeLock(absPtr + PAGE_LOCK_OFFSET, OffheapReadWriteLock.TAG_LOCK_ALWAYS);
 
         try {
             long tmpRelPtr = PageHeader.tempBufferPointer(absPtr);
 
-            if (!clearCheckpoint(fullId)){
-                assert tmpRelPtr == INVALID_REL_PTR;
+            boolean success = clearCheckpoint(fullId);
 
-                return false;
-            }
+            assert success : "Page was pin when we resolve abs pointer, it can not be evicted";
 
             if (tmpRelPtr != INVALID_REL_PTR){
                 PageHeader.tempBufferPointer(absPtr, INVALID_REL_PTR);
@@ -907,6 +919,12 @@ public class PageMemoryImpl implements PageMemoryEx {
 
                 // We pinned the page when allocated the temp buffer, release it now.
                 PageHeader.releasePage(absPtr);
+
+                // Need release again because we pin page when resolve abs pointer,
+                // and page did not have tmp buffer page.
+                if (!tmpBuffer)
+                    PageHeader.releasePage(absPtr);
+
             }
             else {
                 copyInBuffer(absPtr, tmpBuf);


[06/27] ignite git commit: IGNITE-5740 - Added transaction load timing benchmark

Posted by yz...@apache.org.
IGNITE-5740 - Added transaction load timing benchmark


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

Branch: refs/heads/ignite-5658
Commit: 0cb6ac06adddd43ac72c707b29d7216bd4cb711a
Parents: c6ee085
Author: Oleg Ostanin <oo...@gridgain.com>
Authored: Wed Jul 12 15:57:40 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Jul 12 15:57:40 2017 +0300

----------------------------------------------------------------------
 .../yardstick/IgniteBenchmarkArguments.java     |  33 +++++
 .../cache/IgnitePutTxLoadBenchmark.java         | 119 +++++++++++++++++++
 2 files changed, 152 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0cb6ac06/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index 5ec6c54..594fa1f 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -209,6 +209,18 @@ public class IgniteBenchmarkArguments {
     private int pageSize = MemoryConfiguration.DFLT_PAGE_SIZE;
 
     /** */
+    @Parameter(names = {"-sl", "--stringLength"}, description = "Test string length")
+    private int stringLength = 500;
+
+    /** */
+    @Parameter(names = {"-wt", "--warningTime"}, description = "Warning time interval for printing log")
+    private long warningTime = 500;
+
+    /** */
+    @Parameter(names = {"-prb", "--printRollBacks"}, description = "Print rollBacks")
+    private boolean printRollBacks;
+
+    /** */
     @Parameter(names = {"-prt", "--partitions"}, description = "Number of cache partitions")
     private int partitions = 10;
 
@@ -507,6 +519,27 @@ public class IgniteBenchmarkArguments {
     }
 
     /**
+     * @return Test string length.
+     */
+    public int getStringLength() {
+        return stringLength;
+    }
+
+    /**
+     * @return Warning time interval.
+     */
+    public long getWarningTime() {
+        return warningTime;
+    }
+
+    /**
+     * @return Flag for printing rollbacks.
+     */
+    public boolean printRollBacks() {
+        return printRollBacks;
+    }
+
+    /**
      * @return Number of partitioned caches.
      */
     public int partitionedCachesNumber() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cb6ac06/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxLoadBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxLoadBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxLoadBenchmark.java
new file mode 100644
index 0000000..7ac7c3a
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgnitePutTxLoadBenchmark.java
@@ -0,0 +1,119 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Random;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionMetrics;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ * Ignite benchmark that performs transactional load put operations.
+ */
+public class IgnitePutTxLoadBenchmark extends IgniteCacheAbstractBenchmark<Integer, Object> {
+    /** */
+    private ArrayList<IgniteCache<Object, Object>> cacheList;
+
+    /** */
+    private String val;
+
+    /** */
+    private Random random;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        if (!IgniteSystemProperties.getBoolean("SKIP_MAP_CHECK"))
+            ignite().compute().broadcast(new WaitMapExchangeFinishCallable());
+
+        cacheList = new ArrayList<>(args.cachesCount());
+
+        for (int i = 0; i < args.cachesCount(); i++)
+            cacheList.add(ignite().cache("tx-" + i));
+
+        val = createVal(args.getStringLength());
+
+        random = new Random();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        IgniteTransactions transactions = ignite().transactions();
+
+        long startTime;
+        long endTime;
+
+        try (Transaction tx = transactions.txStart(args.txConcurrency(), args.txIsolation())) {
+            ArrayList<Long> keyList = new ArrayList<>(args.scaleFactor());
+
+            for (int i = 0; i < args.scaleFactor(); i++)
+                keyList.add(random.nextLong());
+
+            Collections.sort(keyList);
+
+            for (int i = 0; i < args.scaleFactor(); i++){
+                IgniteCache<Object, Object> curCache = cacheList.get(random.nextInt(cacheList.size()));
+                curCache.put(keyList.get(i), val);
+            }
+
+            startTime = System.currentTimeMillis();
+
+            tx.commit();
+
+            endTime = System.currentTimeMillis();
+
+        }
+
+        TransactionMetrics tm = transactions.metrics();
+
+        if (endTime - startTime > args.getWarningTime())
+            BenchmarkUtils.println("Transaction commit time = " + (tm.commitTime() - startTime));
+
+        if (tm.txRollbacks() > 0 && args.printRollBacks())
+            BenchmarkUtils.println("Transaction rollbacks = " + tm.txRollbacks());
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache<Integer, Object> cache() {
+        return ignite().cache("tx");
+    }
+
+    /**
+     * Creates String val
+     * @param lgth String length
+     * @return String for inserting in cache
+     */
+    private String createVal(int lgth){
+        StringBuilder sb = new StringBuilder(lgth);
+
+        for(int i = 0; i < lgth; i++)
+            sb.append('x');
+
+        return sb.toString();
+    }
+}


[25/27] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by yz...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-5658
Commit: bd5e8a02b25b8fa39d5a9c479498cf14bdb935cd
Parents: 54585ab 2a0d9d6
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jul 18 17:53:08 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jul 18 17:53:08 2017 +0300

----------------------------------------------------------------------
 .../processors/rest/handlers/cache/GridCacheCommandHandler.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[26/27] ignite git commit: Merge branches 'ignite-5658' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-5658

Posted by yz...@apache.org.
Merge branches 'ignite-5658' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-5658


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

Branch: refs/heads/ignite-5658
Commit: 269ca024e83416acb5b36d261406e8144a324f07
Parents: 647be56 bd5e8a0
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Jul 18 19:29:27 2017 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Jul 18 19:29:27 2017 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  33 +++
 examples/config/example-default.xml             |   7 -
 examples/pom.xml                                |   7 +
 .../store/auto/CacheBinaryAutoStoreExample.java | 170 -----------
 .../datagrid/store/auto/package-info.java       |  22 --
 .../JettyRestProcessorAbstractSelfTest.java     |  50 +++-
 .../apache/ignite/IgniteSystemProperties.java   |   8 +
 .../org/apache/ignite/cache/CacheManager.java   |  10 +
 .../ignite/compute/ComputeJobAdapter.java       |   2 +-
 .../configuration/MemoryConfiguration.java      |   4 +-
 .../MemoryPolicyConfiguration.java              |   4 +-
 .../PersistentStoreConfiguration.java           |  61 ++--
 .../apache/ignite/internal/IgniteKernal.java    |  71 +++--
 .../org/apache/ignite/internal/IgnitionEx.java  |  10 +-
 .../ignite/internal/binary/BinaryContext.java   |   1 +
 .../ignite/internal/binary/BinaryUtils.java     |  19 +-
 .../internal/binary/GridBinaryMarshaller.java   |   3 +
 .../client/router/impl/GridTcpRouterImpl.java   |  57 ++--
 .../discovery/GridDiscoveryManager.java         |  38 ++-
 .../ignite/internal/pagemem/FullPageId.java     |   6 +-
 .../ignite/internal/pagemem/PageIdUtils.java    |  14 +-
 .../pagemem/store/IgnitePageStoreManager.java   |   1 +
 .../internal/pagemem/store/PageStore.java       |   2 +
 .../delta/MetaPageUpdateLastAllocatedIndex.java |   2 +-
 .../processors/cache/CacheObjectUtils.java      |   4 +-
 .../processors/cache/GridCacheProcessor.java    |  17 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   3 +-
 .../GridDhtPartitionsExchangeFuture.java        |   8 +
 .../cache/persistence/DbCheckpointListener.java |   7 +-
 .../FullPageIdIterableComparator.java           |  51 ----
 .../GridCacheDatabaseSharedManager.java         | 114 +++++---
 .../persistence/GridCacheOffheapManager.java    |  56 ++--
 .../IgniteCacheDatabaseSharedManager.java       |  65 +++--
 .../cache/persistence/file/FilePageStore.java   |   2 +-
 .../persistence/file/FilePageStoreManager.java  |   1 +
 .../cache/persistence/pagemem/PageMemoryEx.java |   8 +-
 .../persistence/pagemem/PageMemoryImpl.java     |  38 ++-
 .../persistence/partstate/GroupPartitionId.java | 145 +++++++++
 .../partstate/PagesAllocationRange.java         |  68 +++++
 .../partstate/PartitionAllocationMap.java       | 113 +++++++
 .../snapshot/IgniteCacheSnapshotManager.java    |  17 +-
 .../cache/persistence/tree/io/PageMetaIO.java   |  27 +-
 .../persistence/tree/io/TrackingPageIO.java     |   8 +-
 .../persistence/tree/util/PageHandler.java      |   3 +-
 .../wal/FileWriteAheadLogManager.java           |   6 +-
 .../cache/query/GridCacheQueryManager.java      |  17 +-
 .../cache/query/GridCacheQueryResponse.java     |   4 +-
 .../store/GridCacheStoreManagerAdapter.java     |   7 +-
 .../utils/PlatformConfigurationUtils.java       |   4 +-
 .../platform/utils/PlatformUtils.java           |   3 +-
 .../processors/rest/GridRestProcessor.java      |  45 +--
 .../handlers/cache/GridCacheCommandHandler.java |  86 ++++--
 .../timeout/GridTimeoutProcessor.java           |  18 +-
 .../ignite/internal/util/IgniteUtils.java       |  34 ++-
 .../internal/util/MutableSingletonList.java     |  53 ++++
 .../node/VisorPersistentStoreConfiguration.java |  36 +--
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  10 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   2 +-
 .../cache/CacheConnectionLeakStoreTxTest.java   | 291 +++++++++++++++++++
 .../IgniteClusterActivateDeactivateTest.java    |   9 +-
 .../cache/IgniteDynamicCacheMultinodeTest.java  | 168 +++++++++++
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  28 ++
 .../IgniteTxRemoveTimeoutObjectsTest.java       | 194 +++++++++++++
 .../persistence/db/wal/IgnitePdsWalTlbTest.java |   2 +-
 .../ScanQueryOffheapExpiryPolicySelfTest.java   | 114 ++++++++
 .../timeout/GridTimeoutProcessorSelfTest.java   |  68 ++++-
 .../testsuites/IgniteCacheTestSuite3.java       |   4 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 .../ignite/testsuites/IgnitePdsTestSuite.java   |   5 +-
 .../testsuites/IgniteUtilSelfTestSuite.java     |   2 +
 .../util/mbeans/GridMBeanDisableSelfTest.java   | 121 ++++++++
 modules/geospatial/pom.xml                      |  24 ++
 modules/hibernate-4.2/pom.xml                   |  24 ++
 modules/hibernate-5.1/pom.xml                   |  24 ++
 .../IgniteCacheQuerySelfTestSuite2.java         |   3 +
 .../core/include/ignite/cache/query/query_sql.h |   4 +
 .../ignite/cache/query/query_sql_fields.h       |   4 +
 .../Cache/PersistentStoreTest.cs                |   8 +-
 .../Config/spring-test.xml                      |   4 +
 .../IgniteConfigurationTest.cs                  |   8 +-
 .../PersistentStoreConfiguration.cs             |  31 +-
 modules/schedule/pom.xml                        |  24 ++
 modules/web-console/backend/app/mongo.js        |   2 +-
 .../generator/ConfigurationGenerator.js         |   8 +-
 .../generator/defaults/Cluster.service.js       |   2 +-
 .../configuration/clusters/persistence.pug      |   2 +-
 .../web-console/frontend/views/sql/sql.tpl.pug  |   4 +-
 .../yardstick/IgniteBenchmarkArguments.java     |  33 +++
 .../cache/IgnitePutTxLoadBenchmark.java         | 119 ++++++++
 89 files changed, 2381 insertions(+), 639 deletions(-)
----------------------------------------------------------------------



[19/27] ignite git commit: IGNITE-5762: LGPL modules are no longer deployed into Maven repository.

Posted by yz...@apache.org.
IGNITE-5762: LGPL modules are no longer deployed into Maven repository.


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

Branch: refs/heads/ignite-5658
Commit: 1a2dde91083a2d733e2ccc1c03bc9959caa921af
Parents: 3c1749d
Author: Oleg Ostanin <oo...@gridgain.com>
Authored: Mon Jul 17 15:10:36 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Jul 17 15:10:36 2017 +0300

----------------------------------------------------------------------
 examples/pom.xml              |  7 +++++++
 modules/geospatial/pom.xml    | 24 ++++++++++++++++++++++++
 modules/hibernate-4.2/pom.xml | 24 ++++++++++++++++++++++++
 modules/hibernate-5.1/pom.xml | 24 ++++++++++++++++++++++++
 modules/schedule/pom.xml      | 24 ++++++++++++++++++++++++
 5 files changed, 103 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1a2dde91/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 58da342..6042781 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -312,6 +312,13 @@
                     </execution>
                 </executions>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a2dde91/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index c715ffa..488a4a4 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -83,6 +83,23 @@
         </dependency>
     </dependencies>
 
+    <profiles>
+        <profile>
+            <id>lgpl</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-deploy-plugin</artifactId>
+                        <configuration>
+                            <skip>false</skip>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
     <build>
         <plugins>
             <!-- Generate the OSGi MANIFEST.MF for this bundle.
@@ -99,6 +116,13 @@
                     </instructions>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a2dde91/modules/hibernate-4.2/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate-4.2/pom.xml b/modules/hibernate-4.2/pom.xml
index adcf1b1..e40a550 100644
--- a/modules/hibernate-4.2/pom.xml
+++ b/modules/hibernate-4.2/pom.xml
@@ -132,6 +132,23 @@
         </dependency>
     </dependencies>
 
+    <profiles>
+        <profile>
+            <id>lgpl</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-deploy-plugin</artifactId>
+                        <configuration>
+                            <skip>false</skip>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
     <build>
         <testResources>
             <testResource>
@@ -154,6 +171,13 @@
                 <groupId>org.apache.felix</groupId>
                 <artifactId>maven-bundle-plugin</artifactId>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a2dde91/modules/hibernate-5.1/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate-5.1/pom.xml b/modules/hibernate-5.1/pom.xml
index 8424806..a99408e 100644
--- a/modules/hibernate-5.1/pom.xml
+++ b/modules/hibernate-5.1/pom.xml
@@ -132,6 +132,23 @@
         </dependency>
     </dependencies>
 
+    <profiles>
+        <profile>
+            <id>lgpl</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-deploy-plugin</artifactId>
+                        <configuration>
+                            <skip>false</skip>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
     <build>
         <testResources>
             <testResource>
@@ -154,6 +171,13 @@
                 <groupId>org.apache.felix</groupId>
                 <artifactId>maven-bundle-plugin</artifactId>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>

http://git-wip-us.apache.org/repos/asf/ignite/blob/1a2dde91/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 22c2571..2686701 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -83,6 +83,23 @@
         </dependency>
     </dependencies>
 
+    <profiles>
+        <profile>
+            <id>lgpl</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-deploy-plugin</artifactId>
+                        <configuration>
+                            <skip>false</skip>
+                        </configuration>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+
     <build>
         <plugins>
             <!-- Generate the OSGi MANIFEST.MF for this fragment. It is a fragment because it contains internal processors
@@ -96,6 +113,13 @@
                     </instructions>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
         </plugins>
     </build>
 </project>


[15/27] ignite git commit: IGNITE-5444: Collections.singletonList is not properly serialized by binary marshaller. This closes #2217.

Posted by yz...@apache.org.
IGNITE-5444: Collections.singletonList is not properly serialized by binary marshaller. This closes #2217.


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

Branch: refs/heads/ignite-5658
Commit: 038a90a98e3d6dce4514d55710955789ea30d2cb
Parents: c0c6c2f
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Jul 14 16:41:41 2017 +0300
Committer: Andrey V. Mashenkov <an...@gmail.com>
Committed: Fri Jul 14 16:41:41 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/binary/BinaryContext.java   |  1 +
 .../ignite/internal/binary/BinaryUtils.java     | 19 +++++--
 .../internal/binary/GridBinaryMarshaller.java   |  3 ++
 .../processors/cache/CacheObjectUtils.java      |  4 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |  3 +-
 .../platform/utils/PlatformUtils.java           |  3 +-
 .../ignite/internal/util/IgniteUtils.java       | 11 ++++
 .../internal/util/MutableSingletonList.java     | 53 ++++++++++++++++++++
 .../BinaryObjectBuilderAdditionalSelfTest.java  |  2 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java | 28 +++++++++++
 10 files changed, 120 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index fa051f5..eaac7b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -275,6 +275,7 @@ public class BinaryContext {
 
         colTypes.put(ArrayList.class, GridBinaryMarshaller.ARR_LIST);
         colTypes.put(LinkedList.class, GridBinaryMarshaller.LINKED_LIST);
+        colTypes.put(BinaryUtils.SINGLETON_LIST_CLS, GridBinaryMarshaller.SINGLETON_LIST);
         colTypes.put(HashSet.class, GridBinaryMarshaller.HASH_SET);
         colTypes.put(LinkedHashSet.class, GridBinaryMarshaller.LINKED_HASH_SET);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
index 969f3e1..74d1730 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
@@ -32,6 +32,7 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -59,6 +60,7 @@ import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.internal.binary.builder.BinaryLazyValue;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+import org.apache.ignite.internal.util.MutableSingletonList;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -92,6 +94,9 @@ public class BinaryUtils {
     /** Binary classes. */
     private static final Collection<Class<?>> BINARY_CLS = new HashSet<>();
 
+    /** Class for SingletonList obtained at runtime. */
+    public static final Class<? extends Collection> SINGLETON_LIST_CLS = Collections.singletonList(null).getClass();
+
     /** Flag: user type. */
     public static final short FLAG_USR_TYP = 0x0001;
 
@@ -697,7 +702,8 @@ public class BinaryUtils {
             (!wrapTrees() && cls == TreeSet.class) ||
             cls == ConcurrentSkipListSet.class ||
             cls == ArrayList.class ||
-            cls == LinkedList.class;
+            cls == LinkedList.class ||
+            cls == SINGLETON_LIST_CLS;
     }
 
     /**
@@ -738,6 +744,8 @@ public class BinaryUtils {
             return new ArrayList<>(((Collection)col).size());
         else if (cls == LinkedList.class)
             return new LinkedList<>();
+        else if (cls == SINGLETON_LIST_CLS)
+            return new MutableSingletonList<>();
 
         return null;
     }
@@ -1132,7 +1140,7 @@ public class BinaryUtils {
      * @return {@code True} if this is a special collection class.
      */
     public static boolean isSpecialCollection(Class cls) {
-        return ArrayList.class.equals(cls) || LinkedList.class.equals(cls) ||
+        return ArrayList.class.equals(cls) || LinkedList.class.equals(cls) || SINGLETON_LIST_CLS.equals(cls) ||
             HashSet.class.equals(cls) || LinkedHashSet.class.equals(cls);
     }
 
@@ -2023,6 +2031,11 @@ public class BinaryUtils {
 
                     break;
 
+                case GridBinaryMarshaller.SINGLETON_LIST:
+                    col = new MutableSingletonList<>();
+
+                    break;
+
                 case GridBinaryMarshaller.HASH_SET:
                     col = U.newHashSet(size);
 
@@ -2053,7 +2066,7 @@ public class BinaryUtils {
         for (int i = 0; i < size; i++)
             col.add(deserializeOrUnmarshal(in, ctx, ldr, handles, deserialize));
 
-        return col;
+        return colType == GridBinaryMarshaller.SINGLETON_LIST ? U.convertToSingletonList(col) : col;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
index 291c638..d6c8abd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/GridBinaryMarshaller.java
@@ -184,6 +184,9 @@ public class GridBinaryMarshaller {
     public static final byte LINKED_HASH_SET = 4;
 
     /** */
+    public static final byte SINGLETON_LIST = 5;
+
+    /** */
     public static final byte HASH_MAP = 1;
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java
index f9c76df..5afa751 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectUtils.java
@@ -18,7 +18,9 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.util.MutableSingletonList;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -66,7 +68,7 @@ public class CacheObjectUtils {
         for (Object obj : col)
             col0.add(unwrapBinary(ctx, obj, keepBinary, cpy));
 
-        return col0;
+        return (col0 instanceof MutableSingletonList) ? U.convertToSingletonList(col0) : col0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 0065e41..c0f3515 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessorImpl;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.MutableSingletonList;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -349,7 +350,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
                 for (Object item : col)
                     pCol.add(marshalToBinary(item));
 
-                return pCol;
+                return (pCol instanceof MutableSingletonList) ? U.convertToSingletonList(pCol) : pCol;
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
index 654a2a1..dbd65ed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.util.MutableSingletonList;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -941,7 +942,7 @@ public class PlatformUtils {
         for (Object obj : col)
             col0.add(unwrapBinary(obj));
 
-        return col0;
+        return (col0 instanceof MutableSingletonList) ? U.convertToSingletonList(col0) : col0;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 6a3be55..54ffe41 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -9343,6 +9343,17 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * @param col non-null collection with one element
+     * @return a SingletonList containing the element in the original collection
+     */
+    public static <T> Collection<T> convertToSingletonList(Collection<T> col) {
+        if (col.size() != 1) {
+            throw new IllegalArgumentException("Unexpected collection size for singleton list, expecting 1 but was: " + col.size());
+        }
+        return Collections.singletonList(col.iterator().next());
+    }
+
+    /**
      * Returns comparator that sorts remote node addresses. If remote node resides on the same host, then put
      * loopback addresses first, last otherwise.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/main/java/org/apache/ignite/internal/util/MutableSingletonList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/MutableSingletonList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/MutableSingletonList.java
new file mode 100644
index 0000000..87b96de
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/MutableSingletonList.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util;
+
+
+import java.util.AbstractList;
+
+/**
+ * List that can contain maximum of one element. Does not allow null element to be added.
+ */
+public class MutableSingletonList<E> extends AbstractList<E> {
+
+    /** The only element of collection. */
+    private E element;
+
+    /** {@inheritDoc} */
+    @Override public E get(int index) {
+        if (index != 0 || element == null)
+            throw new IndexOutOfBoundsException("Index: " + index + ", Size: " + size());
+
+        return element;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void add(int index, E element) {
+        if (element == null)
+            throw new IllegalArgumentException("Cannot add null element to list");
+        else if (index != 0)
+            throw new IllegalStateException("Element already added to singleton list");
+        else
+            this.element = element;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        return element == null ? 0 : 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
index 145dbb4..82ff383 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryObjectBuilderAdditionalSelfTest.java
@@ -1511,7 +1511,7 @@ public class BinaryObjectBuilderAdditionalSelfTest extends GridCommonAbstractTes
 
         assert MAP.equals(binaryObj.type().fieldTypeName("singletonMap"));
 
-        assert OBJ.equals(binaryObj.type().fieldTypeName("asList"));
+        assert COL.equals(binaryObj.type().fieldTypeName("asList"));
         assert OBJ.equals(binaryObj.type().fieldTypeName("asSet"));
         assert OBJ.equals(binaryObj.type().fieldTypeName("asMap"));
         assert OBJ.equals(binaryObj.type().fieldTypeName("asListHint"));

http://git-wip-us.apache.org/repos/asf/ignite/blob/038a90a9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
index 5dace92..b98615c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
@@ -21,6 +21,7 @@ import java.math.BigDecimal;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -424,6 +425,33 @@ public abstract class GridCacheBinaryObjectsAbstractSelfTest extends GridCommonA
     /**
      * @throws Exception If failed.
      */
+    public void testSingletonList() throws Exception {
+        IgniteCache<Integer, Collection<TestObject>> c = jcache(0);
+
+        c.put(0, Collections.singletonList(new TestObject(123)));
+
+        Collection<TestObject> cFromCache = c.get(0);
+
+        assertEquals(1, cFromCache.size());
+        assertEquals(123, cFromCache.iterator().next().val);
+
+        IgniteCache<Integer, Collection<BinaryObject>> kpc = keepBinaryCache();
+
+        Collection<?> cBinary = kpc.get(0);
+
+        assertEquals(1, cBinary.size());
+
+        Object bObj = cBinary.iterator().next();
+
+        assertTrue(bObj instanceof BinaryObject);
+        assertEquals(Collections.singletonList(null).getClass(), cBinary.getClass());
+
+        assertEquals(Integer.valueOf(123), ((BinaryObject) bObj).field("val"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetAsync() throws Exception {
         IgniteCache<Integer, TestObject> c = jcache(0);
 


[27/27] ignite git commit: ignite-5658 review

Posted by yz...@apache.org.
ignite-5658 review


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

Branch: refs/heads/ignite-5658
Commit: 9936dd850e1caa022064cef16e680e32f1f2007d
Parents: 269ca02
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jul 20 15:30:52 2017 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jul 20 15:30:52 2017 +0300

----------------------------------------------------------------------
 .../yardstick/IgniteBenchmarkArguments.java     |  11 +
 .../IgniteSingleCacheStreamerBenchmark.java     | 209 +++++++++++++++++++
 .../cache/IgniteStreamerBenchmark.java          |  13 +-
 3 files changed, 231 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9936dd85/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index 594fa1f..355b1b0 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -252,6 +252,10 @@ public class IgniteBenchmarkArguments {
     @Parameter(names = {"-stbs", "--streamerBufSize"}, description = "Data streamer buffer size")
     private int streamerBufSize = IgniteDataStreamer.DFLT_PER_NODE_BUFFER_SIZE;
 
+    /** */
+    @Parameter(names = {"-stpo", "--streamerParallelOps"}, description = "Data streamer max parallel ops")
+    private int streamerPerNodeParallelOps = IgniteDataStreamer.DFLT_MAX_PARALLEL_OPS;
+
     /**
      * @return {@code True} if need set {@link PersistentStoreConfiguration}.
      */
@@ -631,6 +635,13 @@ public class IgniteBenchmarkArguments {
         return streamerBufSize;
     }
 
+    /**
+     * @return Streamer per node parallel ops.
+     */
+    public int getStreamerPerNodeParallelOps() {
+        return streamerPerNodeParallelOps;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(IgniteBenchmarkArguments.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9936dd85/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSingleCacheStreamerBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSingleCacheStreamerBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSingleCacheStreamerBenchmark.java
new file mode 100644
index 0000000..1a35c5d
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteSingleCacheStreamerBenchmark.java
@@ -0,0 +1,209 @@
+/*
+ * 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.yardstick.cache;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
+import org.apache.ignite.yardstick.cache.model.SampleValue;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ *
+ */
+public class IgniteSingleCacheStreamerBenchmark extends IgniteAbstractBenchmark {
+    /** */
+    private List<String> cacheNames;
+
+    /** */
+    private ExecutorService executor;
+
+    /** */
+    private int entries;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        entries = args.range();
+
+        if (entries <= 0)
+            throw new IllegalArgumentException("Invalid number of entries: " + entries);
+
+        if (cfg.threads() != 1)
+            throw new IllegalArgumentException("IgniteStreamerBenchmark should be run with single thread. " +
+                "Internally it starts multiple threads.");
+
+        String cacheNamePrefix = args.streamerCachesPrefix();
+
+        if (cacheNamePrefix == null || cacheNamePrefix.isEmpty())
+            throw new IllegalArgumentException("Streamer caches prefix not set.");
+
+        List<String> caches = new ArrayList<>();
+
+        for (String cacheName : ignite().cacheNames()) {
+            if (cacheName.startsWith(cacheNamePrefix))
+                caches.add(cacheName);
+        }
+
+        if (caches.isEmpty())
+            throw new IllegalArgumentException("Failed to find for IgniteStreamerBenchmark caches " +
+                "starting with '" + cacheNamePrefix + "'");
+
+        BenchmarkUtils.println("Found " + caches.size() + " caches for IgniteStreamerBenchmark: " + caches);
+
+        if (args.streamerCacheIndex() >= caches.size()) {
+            throw new IllegalArgumentException("Invalid streamer cache index: " + args.streamerCacheIndex() +
+                ", there are only " + caches.size() + " caches.");
+        }
+
+        if (args.streamerCacheIndex() + args.streamerConcurrentCaches() > caches.size()) {
+            throw new IllegalArgumentException("There are no enough caches [cacheIndex=" + args.streamerCacheIndex() +
+                ", concurrentCaches=" + args.streamerConcurrentCaches() +
+                ", totalCaches=" + caches.size() + ']');
+        }
+
+        Collections.sort(caches);
+
+        cacheNames = new ArrayList<>(caches.subList(args.streamerCacheIndex(),
+            args.streamerCacheIndex() + args.streamerConcurrentCaches()));
+
+        if (cacheNames.size() > 1)
+            throw new IllegalArgumentException("IgniteSingleCacheStreamerBenchmark can run only with single cache " +
+                "[cacheNames=" + cacheNames + ']');
+
+        executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
+
+        BenchmarkUtils.println("IgniteStreamerBenchmark start [cacheIndex=" + args.streamerCacheIndex() +
+            ", concurrentCaches=" + args.streamerConcurrentCaches() +
+            ", entries=" + entries +
+            ", bufferSize=" + args.streamerBufferSize() +
+            ", cachesToUse=" + cacheNames + ']');
+
+        if (cfg.warmup() > 0)
+            throw new IllegalArgumentException("IgniteSingleCacheStreamerBenchmark can run only without warmup " +
+                "[warmup=" + cfg.warmup() + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> map) throws Exception {
+        BenchmarkUtils.println("IgniteStreamerBenchmark start test.");
+
+        long start = System.currentTimeMillis();
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        final String cacheName = cacheNames.get(0);
+
+        try (IgniteDataStreamer<Object, Object> streamer = ignite().dataStreamer(cacheName)) {
+            streamer.perNodeParallelOperations(args.getStreamerPerNodeParallelOps());
+            streamer.perNodeBufferSize(args.streamerBufferSize());
+
+            final List<Future<Void>> futs = new ArrayList<>();
+
+            int availableCpus = Runtime.getRuntime().availableProcessors();
+
+            final AtomicInteger cnt = new AtomicInteger();
+            final int delta = entries / availableCpus;
+
+            for (int i = 0; i < availableCpus; i++) {
+                futs.add(executor.submit(
+                    new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            int min = cnt.getAndAdd(delta);
+                            int max = min + delta;
+
+                            long start = System.currentTimeMillis();
+
+                            BenchmarkUtils.println("IgniteStreamerBenchmark start load cache " +
+                                "[name=" + cacheName +
+                                ", min=" + min +
+                                ", max=" + max + ']');
+
+                            for (int i = 0; i < delta; i++) {
+                                streamer.addData(min + i, new SampleValue(min + i));
+
+                                if (i > 0 && i % 1000 == 0) {
+                                    if (stop.get())
+                                        break;
+
+                                    if (i % 100_000 == 0) {
+                                        BenchmarkUtils.println("IgniteStreamerBenchmark cache load progress " +
+                                            "[name=" + cacheName +
+                                            ", entries=" + i +
+                                            ", delta=" + delta +
+                                            ", timeMillis=" + (System.currentTimeMillis() - start) + ']');
+                                    }
+                                }
+                            }
+
+                            long time = System.currentTimeMillis() - start;
+
+                            BenchmarkUtils.println("Thread finished loading cache [name=" + cacheName +
+                                ", min=" + min +
+                                ", max=" + max +
+                                ", bufferSize=" + args.streamerBufferSize() +
+                                ", totalTimeMillis=" + time + ']');
+
+                            return null;
+                        }
+                    }
+                ));
+            }
+
+            for (Future<Void> fut : futs)
+                fut.get();
+        }
+        finally {
+            stop.set(true);
+        }
+
+        long time = System.currentTimeMillis() - start;
+
+        BenchmarkUtils.println("IgniteStreamerBenchmark finished [totalTimeMillis=" + time +
+            ", entries=" + entries +
+            ", bufferSize=" + args.streamerBufferSize() + ']');
+
+        BenchmarkUtils.println("Cache size [cacheName=" + cacheName +
+            ", size=" + ignite().cache(cacheName).size() + ']');
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void tearDown() throws Exception {
+        if (executor != null) {
+            executor.shutdownNow();
+
+            executor.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        }
+
+        super.tearDown();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9936dd85/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteStreamerBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteStreamerBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteStreamerBenchmark.java
index 9e253e1..9914eac 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteStreamerBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteStreamerBenchmark.java
@@ -25,6 +25,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
@@ -175,6 +176,11 @@ public class IgniteStreamerBenchmark extends IgniteAbstractBenchmark {
                         BenchmarkUtils.println("IgniteStreamerBenchmark start load cache [name=" + cacheName + ']');
 
                         try (IgniteDataStreamer<Object, Object> streamer = ignite().dataStreamer(cacheName)) {
+                            streamer.perNodeParallelOperations(Runtime.getRuntime().availableProcessors() * 4);
+                            streamer.perNodeBufferSize(args.streamerBufferSize());
+
+                            BenchmarkUtils.println("Data streamer: " + streamer);
+
                             for (int i = 0; i < entries; i++) {
                                 streamer.addData(i, new SampleValue(i));
 
@@ -226,8 +232,11 @@ public class IgniteStreamerBenchmark extends IgniteAbstractBenchmark {
 
     /** {@inheritDoc} */
     @Override public void tearDown() throws Exception {
-        if (executor != null)
-            executor.shutdown();
+        if (executor != null) {
+            executor.shutdownNow();
+
+            executor.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        }
 
         super.tearDown();
     }


[14/27] ignite git commit: ignite-5722 Cache entries stay in onheap after scan query execution for OFFHEAP_TIRED cache with expiry policy

Posted by yz...@apache.org.
ignite-5722 Cache entries stay in onheap after scan query execution for OFFHEAP_TIRED cache with expiry policy


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

Branch: refs/heads/ignite-5658
Commit: c0c6c2f9f41605820f2a9a226c6847daee1ba95d
Parents: 1b2b26a
Author: agura <ag...@gridgain.com>
Authored: Tue Jul 11 16:24:54 2017 +0300
Committer: agura <ag...@gridgain.com>
Committed: Fri Jul 14 16:11:51 2017 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryManager.java      |   6 +-
 .../ScanQueryOffheapExpiryPolicySelfTest.java   | 114 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite2.java         |   3 +
 3 files changed, 122 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c0c6c2f9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 4307d26..f8342b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -3035,7 +3035,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                     entry.unswap();
 
-                    return entry.peek(true, true, topVer, expiryPlc);
+                    CacheObject cacheObj = entry.peek(true, true, topVer, expiryPlc);
+
+                    cctx.evicts().touch(entry, topVer);
+
+                    return cacheObj;
                 }
                 catch (GridCacheEntryRemovedException ignore) {
                     // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c0c6c2f9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryOffheapExpiryPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryOffheapExpiryPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryOffheapExpiryPolicySelfTest.java
new file mode 100644
index 0000000..080558e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/ScanQueryOffheapExpiryPolicySelfTest.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import javax.cache.expiry.CreatedExpiryPolicy;
+import javax.cache.expiry.Duration;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.ignite.cache.CachePeekMode.OFFHEAP;
+import static org.apache.ignite.cache.CachePeekMode.ONHEAP;
+
+/**
+ *
+ */
+public class ScanQueryOffheapExpiryPolicySelfTest extends GridCommonAbstractTest {
+
+    /** Nodes count. */
+    private static final int NODES_CNT = 2;
+
+    /** Entries count */
+    private static final int ENTRIES_CNT = 1024;
+
+    /** CAche name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setName(CACHE_NAME);
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setExpiryPolicyFactory(CreatedExpiryPolicy.factoryOf(new Duration(TimeUnit.MINUTES, 10)));
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(NODES_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEntriesMovedFromOnHeap() throws Exception {
+        Ignite ignite0 = grid(0);
+        Ignite ignite1 = grid(1);
+
+        IgniteCache<Integer, Integer> cache0 = ignite0.cache(CACHE_NAME);
+        IgniteCache<Integer, Integer> cache1 = ignite1.cache(CACHE_NAME);
+
+        populateCache(cache0);
+
+        assertEquals(0, cache0.localSize(ONHEAP));
+        assertEquals(0, cache1.localSize(ONHEAP));
+
+        assertEquals(ENTRIES_CNT, cache0.localSize(OFFHEAP) + cache1.localSize(OFFHEAP));
+
+        cache0.query(new ScanQuery<>()).getAll();
+        cache1.query(new ScanQuery<>()).getAll();
+
+        assertEquals(0, cache0.localSize(ONHEAP));
+        assertEquals(0, cache1.localSize(ONHEAP));
+
+        assertEquals(ENTRIES_CNT, cache0.localSize(OFFHEAP) + cache1.localSize(OFFHEAP));
+    }
+
+    /**
+     * @param cache Cache instance.
+     */
+    private static void populateCache(IgniteCache<Integer, Integer> cache) {
+        for (int i = 0; i < ENTRIES_CNT; i++)
+            cache.put(i, i);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c0c6c2f9/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
index 4d82bf9..8fe7f77 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.index.DynamicIndexPartitioned
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexPartitionedTransactionalConcurrentSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexReplicatedAtomicConcurrentSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexReplicatedTransactionalConcurrentSelfTest;
+import org.apache.ignite.internal.processors.cache.query.ScanQueryOffheapExpiryPolicySelfTest;
 import org.apache.ignite.internal.processors.query.IgniteCacheGroupsCompareQueryTest;
 import org.apache.ignite.internal.processors.query.IgniteCacheGroupsSqlDistributedJoinSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteCacheGroupsSqlSegmentedIndexMultiNodeSelfTest;
@@ -62,6 +63,8 @@ public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
 
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
 
+        suite.addTestSuite(ScanQueryOffheapExpiryPolicySelfTest.class);
+
         suite.addTestSuite(IgniteCacheCrossCacheJoinRandomTest.class);
         suite.addTestSuite(IgniteCacheClientQueryReplicatedNodeRestartSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeFailTest.class);


[11/27] ignite git commit: Merge branch 'ignite-2.1'

Posted by yz...@apache.org.
Merge branch 'ignite-2.1'


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

Branch: refs/heads/ignite-5658
Commit: abe8e67e63018305cf0c2065282380b0cf9b762c
Parents: 7338445 3c1749d
Author: devozerov <vo...@gridgain.com>
Authored: Fri Jul 14 13:22:54 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Jul 14 13:22:54 2017 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  33 ++++
 examples/config/example-default.xml             |   7 -
 .../store/auto/CacheBinaryAutoStoreExample.java | 170 -------------------
 .../datagrid/store/auto/package-info.java       |  22 ---
 .../configuration/MemoryConfiguration.java      |   4 +-
 .../MemoryPolicyConfiguration.java              |   4 +-
 .../PersistentStoreConfiguration.java           |  61 +++----
 .../ignite/internal/pagemem/FullPageId.java     |   6 +-
 .../ignite/internal/pagemem/PageIdUtils.java    |  14 +-
 .../pagemem/store/IgnitePageStoreManager.java   |   1 +
 .../internal/pagemem/store/PageStore.java       |   2 +
 .../delta/MetaPageUpdateLastAllocatedIndex.java |   2 +-
 .../processors/cache/GridCacheProcessor.java    |   6 +
 .../GridDhtPartitionsExchangeFuture.java        |   8 +
 .../cache/persistence/DbCheckpointListener.java |   7 +-
 .../FullPageIdIterableComparator.java           |  51 ------
 .../GridCacheDatabaseSharedManager.java         |  86 +++++++---
 .../persistence/GridCacheOffheapManager.java    |  56 +++---
 .../IgniteCacheDatabaseSharedManager.java       |  21 ++-
 .../cache/persistence/file/FilePageStore.java   |   2 +-
 .../persistence/file/FilePageStoreManager.java  |   1 +
 .../cache/persistence/pagemem/PageMemoryEx.java |   8 +-
 .../persistence/pagemem/PageMemoryImpl.java     |  38 +++--
 .../persistence/partstate/GroupPartitionId.java | 145 ++++++++++++++++
 .../partstate/PagesAllocationRange.java         |  68 ++++++++
 .../partstate/PartitionAllocationMap.java       | 113 ++++++++++++
 .../snapshot/IgniteCacheSnapshotManager.java    |  17 +-
 .../cache/persistence/tree/io/PageMetaIO.java   |  27 +--
 .../persistence/tree/io/TrackingPageIO.java     |   8 +-
 .../persistence/tree/util/PageHandler.java      |   3 +-
 .../wal/FileWriteAheadLogManager.java           |   6 +-
 .../utils/PlatformConfigurationUtils.java       |   4 +-
 .../processors/rest/GridRestProcessor.java      |  45 ++---
 .../node/VisorPersistentStoreConfiguration.java |  36 ++--
 .../persistence/db/wal/IgnitePdsWalTlbTest.java |   2 +-
 .../Cache/PersistentStoreTest.cs                |   8 +-
 .../Config/spring-test.xml                      |   4 +
 .../IgniteConfigurationTest.cs                  |   8 +-
 .../PersistentStoreConfiguration.cs             |  31 ++--
 modules/web-console/backend/app/mongo.js        |   2 +-
 .../generator/ConfigurationGenerator.js         |   8 +-
 .../generator/defaults/Cluster.service.js       |   2 +-
 .../configuration/clusters/persistence.pug      |   2 +-
 .../web-console/frontend/views/sql/sql.tpl.pug  |   4 +-
 .../yardstick/IgniteBenchmarkArguments.java     |  33 ++++
 .../cache/IgnitePutTxLoadBenchmark.java         | 119 +++++++++++++
 46 files changed, 830 insertions(+), 475 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/abe8e67e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/abe8e67e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------


[02/27] ignite git commit: IGNITE-5639 Added duration for empty result set.

Posted by yz...@apache.org.
IGNITE-5639 Added duration for empty result set.


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

Branch: refs/heads/ignite-5658
Commit: f3828261b30c12d5aa181914033afe46c787f87e
Parents: 707c454
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Jul 12 14:57:50 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Jul 12 14:57:50 2017 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/views/sql/sql.tpl.pug | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f3828261/modules/web-console/frontend/views/sql/sql.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/sql/sql.tpl.pug b/modules/web-console/frontend/views/sql/sql.tpl.pug
index f215c43..0b8011a 100644
--- a/modules/web-console/frontend/views/sql/sql.tpl.pug
+++ b/modules/web-console/frontend/views/sql/sql.tpl.pug
@@ -196,7 +196,7 @@ mixin paragraph-scan
 
         .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
             .error(ng-switch-when='error') Error: {{paragraph.error.message}}
-            .empty(ng-switch-when='empty') Result set is empty
+            .empty(ng-switch-when='empty') Result set is empty. Duration: #[b {{paragraph.duration | duration}}]
             .table(ng-switch-when='table')
                 +table-result-heading-scan
                 +table-result-body
@@ -247,7 +247,7 @@ mixin paragraph-query
                 label Error: {{paragraph.error.message}}
                 br
                 a(ng-show='paragraph.resultType() === "error"' ng-click='showStackTrace(paragraph)') Show more
-            .empty(ng-switch-when='empty') Result set is empty
+            .empty(ng-switch-when='empty') Result set is empty. Duration: #[b {{paragraph.duration | duration}}]
             .table(ng-switch-when='table')
                 +table-result-heading-query
                 +table-result-body


[24/27] ignite git commit: Fixed test IgniteClusterActivateDeactivateTestWithPersistence.

Posted by yz...@apache.org.
Fixed test IgniteClusterActivateDeactivateTestWithPersistence.


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

Branch: refs/heads/ignite-5658
Commit: 54585ab3ea98b3236ac2635440ed496fb24742d8
Parents: 32f5cbc
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jul 18 17:52:51 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jul 18 17:52:51 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/query/GridCacheQueryResponse.java      | 4 ++--
 .../cache/IgniteClusterActivateDeactivateTest.java          | 9 +++++++--
 2 files changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/54585ab3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
index 521aacf..13e0915 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java
@@ -130,10 +130,10 @@ public class GridCacheQueryResponse extends GridCacheIdMessage implements GridCa
         if (err != null && errBytes == null)
             errBytes = U.marshal(ctx, err);
 
-        if (metaDataBytes == null)
+        if (metaDataBytes == null && metadata != null)
             metaDataBytes = marshalCollection(metadata, cctx);
 
-        if (dataBytes == null)
+        if (dataBytes == null && data != null)
             dataBytes = marshalCollection(data, cctx);
 
         if (addDepInfo && !F.isEmpty(data)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/54585ab3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
index 8a604be..cb7c274 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -366,13 +366,18 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
 
         startWithCaches1(srvs, clients);
 
-        if (initiallyActive && persistenceEnabled())
+        int minorVer = 1;
+
+        if (initiallyActive && persistenceEnabled()) {
             ignite(0).active(true);
 
+            minorVer++;
+        }
+
         if (blockMsgNodes.length == 0)
             blockMsgNodes = new int[]{1};
 
-        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(srvs + clients, 1);
+        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(srvs + clients, minorVer);
 
         List<TestRecordingCommunicationSpi> spis = new ArrayList<>();
 


[16/27] ignite git commit: IGNITE-5753: CPP Memory leak fixed.

Posted by yz...@apache.org.
IGNITE-5753: CPP Memory leak fixed.


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

Branch: refs/heads/ignite-5658
Commit: 07cc05f51666b19598abf0e8dcb2d2e534369df3
Parents: 038a90a
Author: Igor Sapego <is...@gridgain.com>
Authored: Fri Jul 14 18:23:46 2017 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Fri Jul 14 18:23:46 2017 +0300

----------------------------------------------------------------------
 .../platforms/cpp/core/include/ignite/cache/query/query_sql.h    | 4 ++++
 .../cpp/core/include/ignite/cache/query/query_sql_fields.h       | 4 ++++
 2 files changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/07cc05f5/modules/platforms/cpp/core/include/ignite/cache/query/query_sql.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_sql.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_sql.h
index eb0606a..365dcaa 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_sql.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_sql.h
@@ -249,6 +249,10 @@ namespace ignite
                  */
                 void ClearArguments()
                 {
+                    std::vector<impl::cache::query::QueryArgumentBase*>::iterator iter;
+                    for (iter = args.begin(); iter != args.end(); ++iter)
+                        delete *iter;
+
                     args.clear();
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/07cc05f5/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h b/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
index bf8d7ac..e00da28 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
@@ -278,6 +278,10 @@ namespace ignite
                  */
                 void ClearArguments()
                 {
+                    std::vector<impl::cache::query::QueryArgumentBase*>::iterator iter;
+                    for (iter = args.begin(); iter != args.end(); ++iter)
+                        delete *iter;
+
                     args.clear();
                 }
 


[17/27] ignite git commit: IGNITE-5452: GridTimeoutProcessor can hang on stop. This closes #2279.

Posted by yz...@apache.org.
IGNITE-5452: GridTimeoutProcessor can hang on stop. This closes #2279.


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

Branch: refs/heads/ignite-5658
Commit: b95c261f0b1376e8523dd1d89f253a5874dbf63b
Parents: 07cc05f
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Jul 14 20:14:47 2017 +0300
Committer: Andrey V. Mashenkov <an...@gmail.com>
Committed: Fri Jul 14 20:14:47 2017 +0300

----------------------------------------------------------------------
 .../timeout/GridTimeoutProcessor.java           |  18 +-
 .../IgniteTxRemoveTimeoutObjectsTest.java       | 194 +++++++++++++++++++
 .../timeout/GridTimeoutProcessorSelfTest.java   |  68 +++++--
 .../testsuites/IgniteCacheTestSuite3.java       |   4 +-
 4 files changed, 265 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b95c261f/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
index 9deca9a..8c71f76 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
@@ -37,7 +37,7 @@ import org.apache.ignite.thread.IgniteThread;
  */
 public class GridTimeoutProcessor extends GridProcessorAdapter {
     /** */
-    private final IgniteThread timeoutWorker;
+    private final TimeoutWorker timeoutWorker;
 
     /** Time-based sorted set for timeout objects. */
     private final GridConcurrentSkipListSet<GridTimeoutObject> timeoutObjs =
@@ -62,13 +62,12 @@ public class GridTimeoutProcessor extends GridProcessorAdapter {
     public GridTimeoutProcessor(GridKernalContext ctx) {
         super(ctx);
 
-        timeoutWorker = new IgniteThread(ctx.config().getIgniteInstanceName(), "grid-timeout-worker",
-            new TimeoutWorker());
+        timeoutWorker = new TimeoutWorker();
     }
 
     /** {@inheritDoc} */
     @Override public void start() {
-        timeoutWorker.start();
+        new IgniteThread(timeoutWorker).start();
 
         if (log.isDebugEnabled())
             log.debug("Timeout processor started.");
@@ -76,7 +75,7 @@ public class GridTimeoutProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        U.interrupt(timeoutWorker);
+        timeoutWorker.cancel();
         U.join(timeoutWorker);
 
         if (log.isDebugEnabled())
@@ -159,6 +158,13 @@ public class GridTimeoutProcessor extends GridProcessorAdapter {
                             timeoutObj.onTimeout();
                         }
                         catch (Throwable e) {
+                            if (isCancelled() && !(e instanceof Error)){
+                                if (log.isDebugEnabled())
+                                    log.debug("Error when executing timeout callback: " + timeoutObj);
+
+                                return;
+                            }
+
                             U.error(log, "Error when executing timeout callback: " + timeoutObj, e);
 
                             if (e instanceof Error)
@@ -170,7 +176,7 @@ public class GridTimeoutProcessor extends GridProcessorAdapter {
                 }
 
                 synchronized (mux) {
-                    while (true) {
+                    while (!isCancelled()) {
                         // Access of the first element must be inside of
                         // synchronization block, so we don't miss out
                         // on thread notification events sent from

http://git-wip-us.apache.org/repos/asf/ignite/blob/b95c261f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxRemoveTimeoutObjectsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxRemoveTimeoutObjectsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxRemoveTimeoutObjectsTest.java
new file mode 100644
index 0000000..c0f9940
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxRemoveTimeoutObjectsTest.java
@@ -0,0 +1,194 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionTimeoutException;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
+
+/**
+ * Test correctness of rollback a transaction with timeout during the grid stop.
+ */
+public class IgniteTxRemoveTimeoutObjectsTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final int PUT_CNT = 1000;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 60_000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxRemoveTimeoutObjects() throws Exception {
+        IgniteCache<Integer, Integer> cache0 = grid(0).cache(DEFAULT_CACHE_NAME);
+        IgniteCache<Integer, Integer> cache1 = grid(1).cache(DEFAULT_CACHE_NAME);
+
+        // start additional grid to be closed.
+        IgniteCache<Integer, Integer> cacheAdditional = startGrid(gridCount()).cache(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < PUT_CNT; i++)
+            cache0.put(i, Integer.MAX_VALUE);
+
+        logTimeoutObjectsFrequency();
+
+        info("Tx1 started");
+        try (Transaction tx = grid(gridCount()).transactions().txStart(PESSIMISTIC, SERIALIZABLE, 100, PUT_CNT)) {
+            try {
+                for (int i = 0; i < PUT_CNT; i++) {
+                    cacheAdditional.put(i, Integer.MIN_VALUE);
+
+                    if (i % 100 == 0)
+                        logTimeoutObjectsFrequency();
+                }
+
+                U.sleep(200);
+
+                tx.commit();
+
+                fail("A timeout should have happened.");
+            }
+            catch (Exception e) {
+                assertTrue(X.hasCause(e, TransactionTimeoutException.class));
+            }
+        }
+
+        assertDoesNotContainLockTimeoutObjects();
+
+        logTimeoutObjectsFrequency();
+
+        stopGrid(gridCount());
+
+        awaitPartitionMapExchange();
+
+        info("Grid2 closed.");
+
+        assertDoesNotContainLockTimeoutObjects();
+
+        logTimeoutObjectsFrequency();
+
+        // Check that the values have not changed and lock can be acquired.
+        try (Transaction tx2 = grid(1).transactions().txStart(PESSIMISTIC, SERIALIZABLE)) {
+            info("Tx2 started");
+
+            for (int i = 0; i < PUT_CNT; i++) {
+                assertEquals(cache1.get(i).intValue(), Integer.MAX_VALUE);
+                cache1.put(i, i);
+
+                if (i % (PUT_CNT / 5) == 0)
+                    logTimeoutObjectsFrequency();
+            }
+
+            tx2.commit();
+        }
+
+        info("Tx2 stopped");
+
+        // Check that that changes committed.
+        for (int i = 0; i < PUT_CNT; i++)
+            assertEquals(cache0.get(i).intValue(), i);
+    }
+
+    /**
+     * Fails if at least one grid contains LockTimeoutObjects.
+     */
+    private void assertDoesNotContainLockTimeoutObjects() {
+        for (Ignite ignite : G.allGrids()) {
+            for (GridTimeoutObject object : getTimeoutObjects((IgniteEx)ignite)) {
+                if (object.getClass().getSimpleName().equals("LockTimeoutObject"))
+                    fail("Grids contain LockTimeoutObjects.");
+            }
+        }
+    }
+
+    /**
+     * Print the number of each timeout object type on each grid to the log.
+     */
+    private void logTimeoutObjectsFrequency() {
+        StringBuilder sb = new StringBuilder("Timeout objects frequency [");
+
+        for (Ignite ignite : G.allGrids()) {
+            IgniteEx igniteEx = (IgniteEx)ignite;
+
+            Map<String, Integer> objFreqMap = new HashMap<>();
+
+            Set<GridTimeoutObject> objs = getTimeoutObjects(igniteEx);
+
+            for (GridTimeoutObject obj : objs) {
+                String clsName = obj.getClass().getSimpleName();
+
+                Integer cnt = objFreqMap.get(clsName);
+
+                if (cnt == null)
+                    objFreqMap.put(clsName, 1);
+                else
+                    objFreqMap.put(clsName, cnt + 1);
+            }
+
+            sb.append("[")
+                .append(igniteEx.name()).append(": size=")
+                .append(objs.size()).append(", ");
+
+            for (Map.Entry<String, Integer> entry : objFreqMap.entrySet()) {
+                sb.append(entry.getKey()).append("=")
+                    .append(entry.getValue())
+                    .append(", ");
+            }
+
+            sb.delete(sb.length() - 2, sb.length())
+                .append("]; ");
+        }
+
+        sb.delete(sb.length() - 2, sb.length())
+            .append("]");
+
+        info(sb.toString()
+            .replaceAll("distributed.IgniteTxRollbackOnStopTest", "Grid"));
+    }
+
+    /**
+     * @param igniteEx IgniteEx.
+     * @return Set of timeout objects that process on current IgniteEx.
+     */
+    private Set<GridTimeoutObject> getTimeoutObjects(IgniteEx igniteEx) {
+        GridTimeoutProcessor timeout = igniteEx.context().timeout();
+
+        return GridTestUtils.getFieldValue(timeout, timeout.getClass(), "timeoutObjs");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b95c261f/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java
index eb248cf..606b102 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java
@@ -41,6 +41,11 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
     private GridTestKernalContext ctx;
 
     /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 60_000;
+    }
+
+    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         ctx = newContext();
 
@@ -84,7 +89,9 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
                 }
 
                 /** {@inheritDoc} */
-                @Override public long endTime() { return endTime; }
+                @Override public long endTime() {
+                    return endTime;
+                }
 
                 /** {@inheritDoc} */
                 @Override public void onTimeout() {
@@ -152,10 +159,14 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
                         private final long endTime = System.currentTimeMillis() + RAND.nextInt(1000) + 500;
 
                         /** {@inheritDoc} */
-                        @Override public IgniteUuid timeoutId() { return id; }
+                        @Override public IgniteUuid timeoutId() {
+                            return id;
+                        }
 
                         /** {@inheritDoc} */
-                        @Override public long endTime() { return endTime; }
+                        @Override public long endTime() {
+                            return endTime;
+                        }
 
                         /** {@inheritDoc} */
                         @Override public void onTimeout() {
@@ -307,9 +318,8 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
         assert timeObjs.size() == max;
 
         // Remove timeout objects so that they aren't able to times out (supposing the cycle takes less than 500 ms).
-        for (GridTimeoutObject obj : timeObjs) {
+        for (GridTimeoutObject obj : timeObjs)
             ctx.timeout().removeTimeoutObject(obj);
-        }
 
         Thread.sleep(1000);
 
@@ -350,7 +360,9 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
                         }
 
                         /** {@inheritDoc} */
-                        @Override public long endTime() { return endTime; }
+                        @Override public long endTime() {
+                            return endTime;
+                        }
 
                         /** {@inheritDoc} */
                         @Override public void onTimeout() {
@@ -370,9 +382,8 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
 
                 // Remove timeout objects so that they aren't able to times out
                 // (supposing the cycle takes less than 500 ms).
-                for (GridTimeoutObject obj : timeObjs) {
+                for (GridTimeoutObject obj : timeObjs)
                     ctx.timeout().removeTimeoutObject(obj);
-                }
             }
         }, threads, "timeout-test-worker");
 
@@ -381,6 +392,9 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
         assert callCnt.get() == 0;
     }
 
+    /**
+     * @throws Exception If test failed.
+     */
     public void testAddRemoveInterleaving() throws Exception {
         final AtomicInteger callCnt = new AtomicInteger(0);
 
@@ -430,9 +444,8 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
 
                 // Remove timeout objects so that they aren't able to times out
                 // (supposing the cycle takes less than 500 ms).
-                for (GridTimeoutObject obj : timeObjs) {
+                for (GridTimeoutObject obj : timeObjs)
                     ctx.timeout().removeTimeoutObject(obj);
-                }
             }
         }, 100, "timeout-test-worker");
 
@@ -516,10 +529,14 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
             private int cnt;
 
             /** {@inheritDoc} */
-            @Override public IgniteUuid timeoutId() { return id; }
+            @Override public IgniteUuid timeoutId() {
+                return id;
+            }
 
             /** {@inheritDoc} */
-            @Override public long endTime() { return endTime; }
+            @Override public long endTime() {
+                return endTime;
+            }
 
             /** {@inheritDoc} */
             @Override public void onTimeout() {
@@ -608,4 +625,31 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
 
         assert latch.await(3000, MILLISECONDS);
     }
+
+    /**
+     * Test that eaten {@link InterruptedException} will not hang on the closing of the grid.
+     *
+     * @throws Exception If test failed.
+     */
+    public void testCancelingWithClearedInterruptedFlag() throws Exception {
+        final CountDownLatch onTimeoutCalled = new CountDownLatch(1);
+
+        ctx.timeout().addTimeoutObject(new GridTimeoutObjectAdapter(10) {
+            /** {@inheritDoc} */
+            @Override public void onTimeout() {
+                try {
+                    onTimeoutCalled.countDown();
+
+                    // Wait for CacheProcessor has stopped and cause InterruptedException
+                    // which clears interrupted flag.
+                    Thread.sleep(Long.MAX_VALUE);
+                }
+                catch (InterruptedException ignore) {
+                    // No-op.
+                }
+            }
+        });
+
+        onTimeoutCalled.await();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b95c261f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
index 58e9dc3..a6be07e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
@@ -36,9 +36,9 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheGroupsTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheInterceptorSelfTestSuite;
 import org.apache.ignite.internal.processors.cache.IgniteCacheScanPredicateDeploymentSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheAsyncOperationsTest;
-import org.apache.ignite.internal.processors.cache.distributed.CacheGroupsPreloadTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheMixedModeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxGetAfterStopTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteTxRemoveTimeoutObjectsTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDaemonNodePartitionedSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedOnlyP2PDisabledByteArrayValuesSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedOnlyP2PEnabledByteArrayValuesSelfTest;
@@ -199,6 +199,8 @@ public class IgniteCacheTestSuite3 extends TestSuite {
 
         suite.addTestSuite(CacheAsyncOperationsTest.class);
 
+        suite.addTestSuite(IgniteTxRemoveTimeoutObjectsTest.class);
+
         return suite;
     }
 }


[22/27] ignite git commit: Added test in suite.

Posted by yz...@apache.org.
Added test in suite.


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

Branch: refs/heads/ignite-5658
Commit: 32f5cbc30228129c238dcb3a15394d9a438985c3
Parents: 644c9f3
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jul 18 12:15:46 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jul 18 12:15:46 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java   | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/32f5cbc3/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
index f09973b..5b562c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
@@ -18,11 +18,12 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.IgniteClusterActivateDeactivateTestWithPersistence;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsClientNearCachePutGetTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDynamicCacheTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsSingleNodePutGetPersistenceTest;
-import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsEvictionTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsCheckpointSimulationWithRealCpDisabledTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsEvictionTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.BPlusTreePageMemoryImplTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.BPlusTreeReuseListPageMemoryImplTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.MetadataStoragePageMemoryImplTest;
@@ -71,6 +72,8 @@ public class IgnitePdsTestSuite extends TestSuite {
         suite.addTestSuite(IgnitePdsDynamicCacheTest.class);
         suite.addTestSuite(IgnitePdsClientNearCachePutGetTest.class);
 
+        suite.addTestSuite(IgniteClusterActivateDeactivateTestWithPersistence.class);
+
         return suite;
     }
 }


[13/27] ignite git commit: ignite-5489 Fixed possible connection leaks when loadPreviousValue set to true

Posted by yz...@apache.org.
ignite-5489 Fixed possible connection leaks when loadPreviousValue set to true


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

Branch: refs/heads/ignite-5658
Commit: 1b2b26a82ea286472134a22619952c662b95033f
Parents: 7283edb
Author: Tikhonov Nikolay <ti...@gmail.com>
Authored: Wed Jun 21 17:55:05 2017 +0300
Committer: agura <ag...@gridgain.com>
Committed: Fri Jul 14 15:43:31 2017 +0300

----------------------------------------------------------------------
 .../store/GridCacheStoreManagerAdapter.java     |   7 +-
 .../cache/CacheConnectionLeakStoreTxTest.java   | 291 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   2 +
 3 files changed, 299 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2b26a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index c02e2c7..99541ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -327,7 +327,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheLoaderException(e));
             }
             finally {
-                sessionEnd0(tx, threwEx);
+                IgniteInternalTx tx0 = tx;
+
+                if (tx0 != null && (tx0.dht() && tx0.local()))
+                    tx0 = null;
+
+                sessionEnd0(tx0, threwEx);
             }
 
             if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2b26a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConnectionLeakStoreTxTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConnectionLeakStoreTxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConnectionLeakStoreTxTest.java
new file mode 100644
index 0000000..611f2cd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheConnectionLeakStoreTxTest.java
@@ -0,0 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.io.Serializable;
+import java.util.concurrent.ConcurrentHashMap;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.cache.store.CacheStoreSession;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.resources.CacheStoreSessionResource;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.cache.TestCacheSession;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ *
+ */
+public class CacheConnectionLeakStoreTxTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** */
+    private static final int CLIENT_NODE = 1;
+
+    /** */
+    private static boolean client;
+
+    /** */
+    private static volatile boolean isLoadFromStore;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid(0);
+
+        client = true;
+
+        startGrid(CLIENT_NODE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        isLoadFromStore = false;
+        TestStore.sessions.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupAtomic() throws Exception {
+        checkConnectionLeak(CacheAtomicityMode.ATOMIC, null, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupAtomicLoadFromStore() throws Exception {
+        isLoadFromStore = true;
+
+        checkConnectionLeak(CacheAtomicityMode.ATOMIC, null, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupOptimisticRepeatableRead() throws Exception {
+        checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, OPTIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupOptimisticRepeatableReadLoadFromStore() throws Exception {
+        isLoadFromStore = true;
+
+        checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, OPTIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupOptimisticReadCommitted() throws Exception {
+        checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, OPTIMISTIC, READ_COMMITTED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupOptimisticReadCommittedLoadFromStore() throws Exception {
+        isLoadFromStore = true;
+
+        checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, OPTIMISTIC, READ_COMMITTED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupPessimisticRepeatableRead() throws Exception {
+        checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, PESSIMISTIC, REPEATABLE_READ);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupPessimisticReadCommitted() throws Exception {
+        checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, PESSIMISTIC, READ_COMMITTED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConnectionLeakOneBackupPessimisticReadCommittedLoadFromStore() throws Exception {
+        isLoadFromStore = true;
+
+        checkConnectionLeak(CacheAtomicityMode.TRANSACTIONAL, PESSIMISTIC, READ_COMMITTED);
+    }
+
+    /**
+     * @param atomicityMode Atomicity mode.
+     * @param txConcurrency Transaction concurrency.
+     * @param txIsolation Transaction isolation.
+     *
+     * @throws Exception If failed.
+     */
+    private void checkConnectionLeak(
+            CacheAtomicityMode atomicityMode,
+            TransactionConcurrency txConcurrency,
+            TransactionIsolation txIsolation
+    ) throws Exception {
+        CacheConfiguration<Integer, Integer> cacheCfg = new CacheConfiguration<>();
+
+        cacheCfg.setName(CACHE_NAME);
+        cacheCfg.setCacheMode(CacheMode.PARTITIONED);
+        cacheCfg.setAtomicityMode(atomicityMode);
+        cacheCfg.setCacheStoreFactory(new TestStoreFactory());
+        cacheCfg.setReadThrough(true);
+        cacheCfg.setWriteThrough(false);
+        cacheCfg.setLoadPreviousValue(true);
+
+        Ignite ignite = ignite(CLIENT_NODE);
+        IgniteCache<Integer, Integer> cache = ignite.createCache(cacheCfg);
+
+        try {
+            assertEquals(0, cache.size());
+
+            if (atomicityMode == CacheAtomicityMode.TRANSACTIONAL) {
+                try (Transaction tx = ignite.transactions().txStart(txConcurrency, txIsolation)) {
+                    cacheOp(cache);
+
+                    tx.commit();
+                }
+            }
+            else {
+                cacheOp(cache);
+            }
+
+            assertTrue("Session was leak on nodes: " + TestStore.sessions, TestStore.sessions.isEmpty());
+        }
+        finally {
+            cache.destroy();
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     */
+    private void cacheOp(IgniteCache<Integer, Integer> cache) {
+        boolean b = cache.putIfAbsent(42, 42);
+
+        log.info("PutIfAbsent: " + b);
+
+        Integer val = cache.get(42);
+
+        log.info("Get: " + val);
+    }
+
+    /**
+     *
+     */
+    private static class TestStoreFactory implements Factory<CacheStoreAdapter<Integer, Integer>> {
+        /** {@inheritDoc} */
+        @Override public CacheStoreAdapter<Integer, Integer> create() {
+            return new TestStore();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestStore extends CacheStoreAdapter<Integer, Integer> implements Serializable {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /** */
+        private CacheStoreSession NULL = new TestCacheSession();
+
+        /** */
+        public static ConcurrentHashMap<CacheStoreSession, ClusterNode> sessions = new ConcurrentHashMap<>();
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            addSession();
+
+            return isLoadFromStore ? key : null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> e) throws CacheWriterException {
+            addSession();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            addSession();
+        }
+
+        /**  */
+        private void addSession() {
+            sessions.put(ses == null ? NULL : ses, ignite.cluster().localNode());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            sessions.remove(ses == null ? NULL : ses);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2b26a8/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 45f575e..e7f38be 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -21,6 +21,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListenerSelfTest;
 import org.apache.ignite.internal.processors.GridCacheTxLoadFromStoreOnLockSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheClientStoreSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheConnectionLeakStoreTxTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticReadCommittedSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticRepeatableReadSeltTest;
 import org.apache.ignite.internal.processors.cache.CacheGetEntryOptimisticSerializableSeltTest;
@@ -279,6 +280,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(CacheStoreUsageMultinodeStaticStartTxTest.class);
         suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartAtomicTest.class);
         suite.addTestSuite(CacheStoreUsageMultinodeDynamicStartTxTest.class);
+        suite.addTestSuite(CacheConnectionLeakStoreTxTest.class);
 
         suite.addTestSuite(GridCacheStoreManagerDeserializationTest.class);
         suite.addTestSuite(GridLocalCacheStoreManagerDeserializationTest.class);


[23/27] ignite git commit: Null cache shouldn't be mentioned in GridCacheCommandHandler error message when a cache is not found.

Posted by yz...@apache.org.
Null cache shouldn't be mentioned in GridCacheCommandHandler error message when a cache is not found.


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

Branch: refs/heads/ignite-5658
Commit: 2a0d9d691d49c3ae82962684b3d114bb0079867c
Parents: 32f5cbc
Author: shroman <rs...@yahoo.com>
Authored: Tue Jul 18 21:25:12 2017 +0900
Committer: shroman <rs...@yahoo.com>
Committed: Tue Jul 18 21:25:12 2017 +0900

----------------------------------------------------------------------
 .../processors/rest/handlers/cache/GridCacheCommandHandler.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2a0d9d69/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index c2ca587..0006f4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -338,7 +338,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         if (cache == null)
             throw new IgniteCheckedException(
-                "Failed to find cache for given cache name (null for default cache): " + cacheName);
+                "Failed to find cache for given cache name: " + cacheName);
 
         return cache;
     }
@@ -756,7 +756,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         if (cache == null)
             throw new IgniteCheckedException(
-                "Failed to find cache for given cache name (null for default cache): " + cacheName);
+                "Failed to find cache for given cache name: " + cacheName);
 
         return cache;
     }


[10/27] ignite git commit: IGNITE-5744 Ignore non user caches when automatically choosing a queryable cache inside JDBC driver

Posted by yz...@apache.org.
IGNITE-5744 Ignore non user caches when automatically choosing a queryable cache inside JDBC driver


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

Branch: refs/heads/ignite-5658
Commit: 3c1749da82e663500e45a34369eac48dbbc62bdc
Parents: 689b1b6
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu Jul 13 11:25:55 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jul 13 11:25:55 2017 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/3c1749da/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 85772d8..f7c6e11 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
@@ -1668,6 +1668,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     public IgniteCacheProxy<?, ?> getOrStartPublicCache(boolean start, boolean inclLoc) throws IgniteCheckedException {
         // Try to find started cache first.
         for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
+            if (!e.getValue().context().userCache())
+                continue;
+
             CacheConfiguration ccfg = e.getValue().configuration();
 
             String cacheName = ccfg.getName();
@@ -1680,6 +1683,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             for (Map.Entry<String, DynamicCacheDescriptor> e : cachesInfo.registeredCaches().entrySet()) {
                 DynamicCacheDescriptor desc = e.getValue();
 
+                if (!desc.cacheType().userCache())
+                    continue;
+
                 CacheConfiguration ccfg = desc.cacheConfiguration();
 
                 if (ccfg.getCacheMode() != LOCAL) {


[20/27] ignite git commit: Merge branch 'ignite-2.1'

Posted by yz...@apache.org.
Merge branch 'ignite-2.1'


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

Branch: refs/heads/ignite-5658
Commit: 624b451b2daad843c50fe359e61759c78ea951de
Parents: 621b0ed 1a2dde9
Author: devozerov <vo...@gridgain.com>
Authored: Mon Jul 17 15:13:05 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Jul 17 15:13:05 2017 +0300

----------------------------------------------------------------------
 examples/pom.xml              |  7 +++++++
 modules/geospatial/pom.xml    | 24 ++++++++++++++++++++++++
 modules/hibernate-4.2/pom.xml | 24 ++++++++++++++++++++++++
 modules/hibernate-5.1/pom.xml | 24 ++++++++++++++++++++++++
 modules/schedule/pom.xml      | 24 ++++++++++++++++++++++++
 5 files changed, 103 insertions(+)
----------------------------------------------------------------------



[04/27] ignite git commit: Removed CacheBinaryAutoStoreExample and relevant bean "h2-example-db" from example-default.xml because example duplicated existing CacheAutoStoreExample.

Posted by yz...@apache.org.
Removed CacheBinaryAutoStoreExample and relevant bean "h2-example-db" from example-default.xml because example duplicated existing CacheAutoStoreExample.


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

Branch: refs/heads/ignite-5658
Commit: 8afdc7baae73ecba67e0735baa97d03f2c4fc715
Parents: 5859b19
Author: devozerov <vo...@gridgain.com>
Authored: Wed Jul 12 13:51:43 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Jul 12 13:51:43 2017 +0300

----------------------------------------------------------------------
 examples/config/example-default.xml             |   7 -
 .../store/auto/CacheBinaryAutoStoreExample.java | 170 -------------------
 .../datagrid/store/auto/package-info.java       |  22 ---
 3 files changed, 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8afdc7ba/examples/config/example-default.xml
----------------------------------------------------------------------
diff --git a/examples/config/example-default.xml b/examples/config/example-default.xml
index 6bd6f16..e6c359d 100644
--- a/examples/config/example-default.xml
+++ b/examples/config/example-default.xml
@@ -28,13 +28,6 @@
         http://www.springframework.org/schema/beans/spring-beans.xsd
         http://www.springframework.org/schema/util
         http://www.springframework.org/schema/util/spring-util.xsd">
-
-    <!-- Datasource for sample in-memory H2 database. -->
-    <bean id="h2-example-db" class="org.h2.jdbcx.JdbcDataSource">
-        <property name="URL" value="jdbc:h2:tcp://localhost/mem:ExampleDb" />
-        <property name="user" value="sa" />
-    </bean>
-
     <bean abstract="true" id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <!-- Set to true to enable distributed class loading for examples, default is false. -->
         <property name="peerClassLoadingEnabled" value="true"/>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8afdc7ba/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
deleted file mode 100644
index c80c87b..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/CacheBinaryAutoStoreExample.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * 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.examples.binary.datagrid.store.auto;
-
-import java.sql.Types;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore;
-import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory;
-import org.apache.ignite.cache.store.jdbc.JdbcType;
-import org.apache.ignite.cache.store.jdbc.JdbcTypeField;
-import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.examples.ExampleNodeStartup;
-import org.apache.ignite.examples.model.Person;
-import org.apache.ignite.examples.util.DbH2ServerStartup;
-import org.apache.ignite.transactions.Transaction;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-
-/**
- * Demonstrates usage of cache with underlying persistent store configured.
- * <p>
- * This example uses {@link CacheJdbcPojoStore} as a persistent store.
- * <p>
- * To start the example, you should:
- * <ul>
- *     <li>Start H2 database TCP server using {@link DbH2ServerStartup}.</li>
- *     <li>Start a few nodes using {@link ExampleNodeStartup} or by starting remote nodes as specified below.</li>
- *     <li>Start example using {@link CacheBinaryAutoStoreExample}.</li>
- * </ul>
- * <p>
- * Remote nodes should always be started with special configuration file which
- * contains H2 data source bean descriptor: {@code 'ignite.{sh|bat} examples/config/example-ignite.xml'}.
- * <p>
- * Alternatively you can run {@link ExampleNodeStartup} in another JVM which will
- * start node with {@code examples/config/example-ignite.xml} configuration.
- */
-public class CacheBinaryAutoStoreExample {
-    /** Global person ID to use across entire example. */
-    private static final Long id = 25121642L;
-
-    /** Cache name. */
-    public static final String CACHE_NAME = CacheBinaryAutoStoreExample.class.getSimpleName();
-
-    /**
-     * Configure cache with store.
-     */
-    private static CacheConfiguration<Long, Person> cacheConfiguration() {
-        CacheJdbcPojoStoreFactory<Long, Person> storeFactory = new CacheJdbcPojoStoreFactory<>();
-
-        storeFactory.setDataSourceBean("h2-example-db");
-        storeFactory.setDialect(new H2Dialect());
-
-        JdbcType jdbcType = new JdbcType();
-
-        jdbcType.setCacheName(CACHE_NAME);
-        jdbcType.setDatabaseSchema("PUBLIC");
-        jdbcType.setDatabaseTable("PERSON");
-
-        jdbcType.setKeyType("java.lang.Long");
-        jdbcType.setKeyFields(new JdbcTypeField(Types.BIGINT, "ID", Long.class, "id"));
-
-        jdbcType.setValueType("org.apache.ignite.examples.model.Person");
-        jdbcType.setValueFields(
-            new JdbcTypeField(Types.BIGINT, "ID", Long.class, "id"),
-            new JdbcTypeField(Types.VARCHAR, "FIRST_NAME", String.class, "firstName"),
-            new JdbcTypeField(Types.VARCHAR, "LAST_NAME", String.class, "lastName")
-        );
-
-        storeFactory.setTypes(jdbcType);
-
-        CacheConfiguration<Long, Person> cfg = new CacheConfiguration<>(CACHE_NAME);
-
-        cfg.setCacheStoreFactory(storeFactory);
-
-        // Set atomicity as transaction, since we are showing transactions in the example.
-        cfg.setAtomicityMode(TRANSACTIONAL);
-
-        // This option will allow to start remote nodes without having user classes in classpath.
-        cfg.setStoreKeepBinary(true);
-
-        cfg.setReadThrough(true);
-        cfg.setWriteThrough(true);
-
-        return cfg;
-    }
-
-    /**
-     * Executes example.
-     *
-     * @param args Command line arguments, none required.
-     * @throws Exception If example execution failed.
-     */
-    public static void main(String[] args) throws Exception {
-        // To start ignite with desired configuration uncomment the appropriate line.
-        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
-            System.out.println();
-            System.out.println(">>> Populate database with data...");
-            DbH2ServerStartup.populateDatabase();
-
-            System.out.println();
-            System.out.println(">>> Cache auto store example started...");
-
-            try (IgniteCache<Long, Person> cache = ignite.getOrCreateCache(cacheConfiguration())) {
-                try (Transaction tx = ignite.transactions().txStart()) {
-                    Person val = cache.get(id);
-
-                    System.out.println(">>> Read value: " + val);
-
-                    val = cache.getAndPut(id, new Person(id, 1L, "Isaac", "Newton", 100.10, "English physicist and mathematician"));
-
-                    System.out.println(">>> Overwrote old value: " + val);
-
-                    val = cache.get(id);
-
-                    System.out.println(">>> Read value: " + val);
-
-                    System.out.println(">>> Update salary in transaction...");
-
-                    val.salary *= 2;
-
-                    cache.put(id, val);
-
-                    tx.commit();
-                }
-
-                System.out.println(">>> Read value after commit: " + cache.get(id));
-
-                cache.clear();
-
-                System.out.println(">>> ------------------------------------------");
-                System.out.println(">>> Load data to cache from DB with custom SQL...");
-
-                // Load cache on all data nodes with custom SQL statement.
-                cache.loadCache(null, "java.lang.Long", "select * from PERSON where id <= 3");
-
-                System.out.println(">>> Loaded cache entries: " + cache.size());
-
-                cache.clear();
-
-                // Load cache on all data nodes with default SQL statement.
-                System.out.println(">>> Load ALL data to cache from DB...");
-                cache.loadCache(null);
-
-                System.out.println(">>> Loaded cache entries: " + cache.size());
-            }
-            finally {
-                // Distributed cache could be removed from cluster only by #destroyCache() call.
-                ignite.destroyCache(CACHE_NAME);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8afdc7ba/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/package-info.java b/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/package-info.java
deleted file mode 100644
index 153f210..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/binary/datagrid/store/auto/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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 description. -->
- * Contains automatic JDBC store example.
- */
-package org.apache.ignite.examples.binary.datagrid.store.auto;