You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/10/04 08:44:44 UTC

[01/25] ignite git commit: IGNITE-6286: SQL: fixed BigDecimal argument handling. This closes #2750.

Repository: ignite
Updated Branches:
  refs/heads/ignite-5937 d4d2c3805 -> f8be46d80


IGNITE-6286: SQL: fixed BigDecimal argument handling. This closes #2750.


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

Branch: refs/heads/ignite-5937
Commit: 013d7dbf7811a1b8e207d6238aff67e43c28adad
Parents: 59ee8af
Author: Sergey Chernolyas <se...@gmail.com>
Authored: Mon Oct 2 10:52:21 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 2 10:52:21 2017 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |   3 +
 .../query/IgniteSqlParameterizedQueryTest.java  | 392 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +-
 3 files changed, 397 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/013d7dbf/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 9e6a1fa..fd7b9a8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -511,6 +512,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 stmt.setNull(idx, Types.VARCHAR);
             else if (obj instanceof BigInteger)
                 stmt.setObject(idx, obj, Types.JAVA_OBJECT);
+            else if (obj instanceof BigDecimal)
+                stmt.setObject(idx, obj, Types.DECIMAL);
             else
                 stmt.setObject(idx, obj);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/013d7dbf/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlParameterizedQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlParameterizedQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlParameterizedQueryTest.java
new file mode 100644
index 0000000..b5039cd
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlParameterizedQueryTest.java
@@ -0,0 +1,392 @@
+/*
+ * 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.query;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Date;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+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.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test sql queries with parameters for all types.
+ * The test is fix  for issue 'IGNITE-6286'
+ *
+ * @author Sergey Chernolyas &amp;sergey_chernolyas@gmail.com&amp;
+ * @see <a href="https://issues.apache.org/jira/browse/IGNITE-6286">IGNITE-6286</a>
+ */
+public class IgniteSqlParameterizedQueryTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE_BOOKMARK = "Bookmark";
+
+    /** */
+    private static final String NODE_CLIENT = "client";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        c.setDiscoverySpi(disco);
+
+        c.setCacheConfiguration(buildCacheConfiguration(CACHE_BOOKMARK));
+        if (gridName.equals(NODE_CLIENT))
+            c.setClientMode(true);
+
+        return c;
+    }
+
+    /**
+     * build cache configuration
+     * @param name cache name
+     * @return configuration
+     * @see CacheConfiguration
+     */
+    private CacheConfiguration buildCacheConfiguration(String name) {
+        CacheConfiguration ccfg = new CacheConfiguration(name);
+        ccfg.setIndexedTypes(String.class, Bookmark.class);
+        return ccfg;
+
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGrid(0);
+        startGrid(NODE_CLIENT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * method for create parametrized query and get first result
+     * @param field name of field
+     * @param val value
+     * @return fist searched object
+     * @see Bookmark
+     */
+    private Object columnValue(String field, Object val) {
+        IgniteCache<String, Bookmark> cache = grid(NODE_CLIENT).cache(CACHE_BOOKMARK);
+        SqlFieldsQuery qry = new SqlFieldsQuery("SELECT " + field + " from  Bookmark where " + field + " = ?");
+        qry.setArgs(val);
+
+        QueryCursor<List<?>> cursor = cache.query(qry);
+        List<List<?>> results = cursor.getAll();
+        assertEquals("Search by field '" + field +"' returns incorrect row count!",1, results.size());
+        List<?> row0 = results.get(0);
+        return row0.get(0);
+    }
+
+    /**
+     * testing parametrized query by field with supported type
+     * @throws Exception if any error occurs
+     */
+    public void testSupportedTypes() throws Exception {
+        IgniteCache<String, Bookmark> cache = grid(NODE_CLIENT).cache(CACHE_BOOKMARK);
+        Bookmark bookmark = new Bookmark();
+        bookmark.setId(UUID.randomUUID().toString());
+        bookmark.setStockCount(Integer.MAX_VALUE);
+        bookmark.setUrlPort(Short.MAX_VALUE);
+        bookmark.setUserId(Long.MAX_VALUE);
+        bookmark.setVisitRatio(Float.MAX_VALUE);
+        bookmark.setTaxPercentage(Double.MAX_VALUE);
+        bookmark.setFavourite(true);
+        bookmark.setDisplayMask(Byte.MAX_VALUE);
+        bookmark.setSerialNumber(UUID.randomUUID());
+        bookmark.setVisitCount(new BigInteger("1000000000000000"));
+        bookmark.setSiteWeight(new BigDecimal("1000000000000000.001"));
+        bookmark.setCreated(new Date());
+        cache.put(bookmark.id, bookmark);
+
+        assertEquals(bookmark.getId(), columnValue("id", bookmark.getId()));
+        assertEquals(bookmark.getStockCount(), columnValue("stockcount", bookmark.getStockCount()));
+        assertEquals(bookmark.getUrlPort(), columnValue("urlport", bookmark.getUrlPort()));
+        assertEquals(bookmark.getUserId(), columnValue("userid", bookmark.getUserId()));
+        assertEquals(bookmark.getVisitRatio(), columnValue("visitratio", bookmark.getVisitRatio()));
+        assertEquals(bookmark.getTaxPercentage(), columnValue("taxpercentage", bookmark.getTaxPercentage()));
+        assertEquals(bookmark.getFavourite(), columnValue("favourite", bookmark.getFavourite()));
+        assertEquals(bookmark.getDisplayMask(), columnValue("displaymask", bookmark.getDisplayMask()));
+        assertEquals(bookmark.getSerialNumber(), columnValue("serialnumber", bookmark.getSerialNumber()));
+        assertEquals(bookmark.getVisitCount(), columnValue("visitcount", bookmark.getVisitCount()));
+        assertEquals(bookmark.getSiteWeight(), columnValue("siteweight", bookmark.getSiteWeight()));
+        assertEquals(bookmark.getCreated(), columnValue("created", bookmark.getCreated()));
+    }
+
+    /**
+     * Object with all predefined SQL Data Types
+     * @see <a href="https://apacheignite.readme.io/docs/dml#section-advanced-configuration">SQL Data Types</a>
+     */
+    private static class Bookmark implements Serializable {
+        /** */
+        @QuerySqlField
+        private String id;
+
+        /** */
+        @QuerySqlField
+        private Integer stockCount;
+
+        /** */
+        @QuerySqlField
+        private Short urlPort;
+
+        /** */
+        @QuerySqlField
+        private Long userId;
+
+        /** */
+        @QuerySqlField
+        private Float visitRatio;
+
+        /** */
+        @QuerySqlField
+        private Double taxPercentage;
+
+        /** */
+        @QuerySqlField
+        private Boolean favourite;
+
+        /** */
+        @QuerySqlField
+        private Byte displayMask;
+
+        /** */
+        @QuerySqlField
+        private UUID serialNumber;
+
+        /** */
+        @QuerySqlField
+        private BigDecimal siteWeight;
+
+        /** */
+        @QuerySqlField
+        private BigInteger visitCount;
+
+        /** */
+        @QuerySqlField
+        private Date created;
+
+        /**
+         *
+         */
+        public String getId() {
+            return id;
+        }
+
+        /**
+         *
+         */
+        public void setId(String id) {
+            this.id = id;
+        }
+
+        /**
+         *
+         */
+        public Integer getStockCount() {
+            return stockCount;
+        }
+
+        /**
+         *
+         */
+        public void setStockCount(Integer stockCount) {
+            this.stockCount = stockCount;
+        }
+
+        /**
+         *
+         */
+        public Short getUrlPort() {
+            return urlPort;
+        }
+
+        /**
+         *
+         */
+        public void setUrlPort(Short urlPort) {
+            this.urlPort = urlPort;
+        }
+
+        /**
+         *
+         */
+        public Long getUserId() {
+            return userId;
+        }
+
+        /**
+         *
+         */
+        public void setUserId(Long userId) {
+            this.userId = userId;
+        }
+
+        /**
+         *
+         */
+        public Float getVisitRatio() {
+            return visitRatio;
+        }
+
+        /**
+         *
+         */
+        public void setVisitRatio(Float visitRatio) {
+            this.visitRatio = visitRatio;
+        }
+
+        /**
+         *
+         */
+        public Double getTaxPercentage() {
+            return taxPercentage;
+        }
+
+        /**
+         *
+         */
+        public void setTaxPercentage(Double taxPercentage) {
+            this.taxPercentage = taxPercentage;
+        }
+
+        /**
+         *
+         */
+        public Boolean getFavourite() {
+            return favourite;
+        }
+
+        /**
+         *
+         */
+        public void setFavourite(Boolean favourite) {
+            this.favourite = favourite;
+        }
+
+        /**
+         *
+         */
+        public Byte getDisplayMask() {
+            return displayMask;
+        }
+
+        /**
+         *
+         */
+        public void setDisplayMask(Byte displayMask) {
+            this.displayMask = displayMask;
+        }
+
+        /**
+         *
+         */
+        public UUID getSerialNumber() {
+            return serialNumber;
+        }
+
+        /**
+         *
+         */
+        public void setSerialNumber(UUID serialNumber) {
+            this.serialNumber = serialNumber;
+        }
+
+        /**
+         *
+         */
+        public BigDecimal getSiteWeight() {
+            return siteWeight;
+        }
+
+        /**
+         *
+         */
+        public void setSiteWeight(BigDecimal siteWeight) {
+            this.siteWeight = siteWeight;
+        }
+
+        /**
+         *
+         */
+        public BigInteger getVisitCount() {
+            return visitCount;
+        }
+
+        /**
+         *
+         */
+        public void setVisitCount(BigInteger visitCount) {
+            this.visitCount = visitCount;
+        }
+
+        /**
+         *
+         */
+        public Date getCreated() {
+            return created;
+        }
+
+        /**
+         *
+         */
+        public void setCreated(Date created) {
+            this.created = created;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+            Bookmark bookmark = (Bookmark)o;
+            return Objects.equals(id, bookmark.id);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return Objects.hash(id);
+        }
+    }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/013d7dbf/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index aaa8e57..c49649b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -123,6 +123,7 @@ import org.apache.ignite.internal.processors.cache.query.IgniteCacheQueryCacheDe
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest;
 import org.apache.ignite.internal.processors.client.ClientConnectorConfigurationValidationSelfTest;
+import org.apache.ignite.internal.processors.query.IgniteSqlParameterizedQueryTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest;
 import org.apache.ignite.internal.processors.query.IgniteQueryDedicatedPoolTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlEntryCacheModeAgnosticTest;
@@ -336,8 +337,8 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteSqlRoutingTest.class);
         suite.addTestSuite(IgniteSqlNotNullConstraintTest.class);
         suite.addTestSuite(LongIndexNameTest.class);
-
         suite.addTestSuite(GridCacheQuerySqlFieldInlineSizeSelfTest.class);
+        suite.addTestSuite(IgniteSqlParameterizedQueryTest.class);
 
         return suite;
     }


[10/25] ignite git commit: IGNITE-6350: SQL: forbid configurations with NOT NULL fields and read-through cache store. This closes #2656.

Posted by sb...@apache.org.
IGNITE-6350: SQL: forbid configurations with NOT NULL fields and read-through cache store. This closes #2656.


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

Branch: refs/heads/ignite-5937
Commit: 3256ee27e7cd929246f2e5813fcc35adda58cc3f
Parents: 16d2370
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Tue Oct 3 10:13:06 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Oct 3 10:13:06 2017 +0300

----------------------------------------------------------------------
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |  88 ++++++++-
 .../processors/cache/GridCacheUtils.java        |   6 +-
 .../processors/query/GridQueryProcessor.java    |   6 +-
 .../internal/processors/query/QueryUtils.java   |  17 ++
 .../query/h2/ddl/DdlStatementsProcessor.java    |  15 +-
 .../query/IgniteSqlNotNullConstraintTest.java   | 179 ++++++++++++++++++-
 .../Cache/CacheConfigurationTest.cs             |   2 +-
 7 files changed, 304 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
index 78020cf..952baa5 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.jdbc;
 
+import java.io.Serializable;
 import java.net.URL;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
@@ -29,8 +30,14 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Collections;
 import java.util.List;
+import javax.cache.Cache;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import org.apache.ignite.cache.CacheInterceptorAdapter;
 import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -39,13 +46,27 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
  * Test SQLSTATE codes propagation with (any) Ignite JDBC driver.
  */
 public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest {
+    /** */
+    protected static final String CACHE_STORE_TEMPLATE = "cache_store";
+
+    /** */
+    protected static final String CACHE_INTERCEPTOR_TEMPLATE = "cache_interceptor";
+
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        startGrid(getConfiguration(getTestIgniteInstanceName(0))
+        IgniteEx grid = startGrid(getConfiguration(getTestIgniteInstanceName(0))
             .setCacheConfiguration(new CacheConfiguration("test")
                 .setQueryEntities(Collections.singletonList(new QueryEntity(Integer.class, Integer.class)))));
+
+        // add cache template for cache with enabled read-through cache store
+        grid.addCacheConfiguration(new CacheConfiguration<>(CACHE_STORE_TEMPLATE)
+            .setCacheStoreFactory(singletonFactory(new TestCacheStore())).setReadThrough(true));
+
+        // add cache template for cache with enabled cache interceptor
+        grid.addCacheConfiguration(new CacheConfiguration<>(CACHE_INTERCEPTOR_TEMPLATE)
+            .setInterceptor(new TestCacheInterceptor()));
     }
 
     /** {@inheritDoc} */
@@ -505,6 +526,44 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
     }
 
     /**
+     * Check error code for the case not null field is configured for table belonging to cache
+     * with enabled read-through cache store.
+     *
+     * @throws SQLException if failed.
+     */
+    public void testNotNullRestrictionReadThroughCacheStore() throws SQLException {
+        checkErrorState(new ConnClosure() {
+            @Override public void run(Connection conn) throws Exception {
+                conn.setSchema("PUBLIC");
+
+                try (Statement stmt = conn.createStatement()) {
+                    stmt.execute("CREATE TABLE cache_store_nulltest(id INT PRIMARY KEY, age INT NOT NULL) " +
+                        "WITH \"template=" + CACHE_STORE_TEMPLATE + "\"");
+                }
+            }
+        }, "0A000");
+    }
+
+    /**
+     * Check error code for the case not null field is configured for table belonging to cache
+     * with configured cache interceptor.
+     *
+     * @throws SQLException if failed.
+     */
+    public void testNotNullRestrictionCacheInterceptor() throws SQLException {
+        checkErrorState(new ConnClosure() {
+            @Override public void run(Connection conn) throws Exception {
+                conn.setSchema("PUBLIC");
+
+                try (Statement stmt = conn.createStatement()) {
+                    stmt.execute("CREATE TABLE cache_interceptor_nulltest(id INT PRIMARY KEY, age INT NOT NULL) " +
+                        "WITH \"template=" + CACHE_INTERCEPTOR_TEMPLATE + "\"");
+                }
+            }
+        }, "0A000");
+    }
+
+    /**
      * @return Connection to execute statements on.
      * @throws SQLException if failed.
      */
@@ -570,4 +629,31 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
          */
         void run(Connection conn) throws Exception;
     }
+
+    /**
+     * Cache store stub.
+     */
+    protected class TestCacheStore extends CacheStoreAdapter<Object,Object> implements Serializable {
+        /** {@inheritDoc} */
+        @Override public Object load(Object key) throws CacheLoaderException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<?, ?> entry) throws CacheWriterException {
+            // No-op
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            // No-op
+        }
+    }
+
+    /**
+     * Cache interceptor stub.
+     */
+    private static class TestCacheInterceptor extends CacheInterceptorAdapter<Object, Object> implements Serializable {
+        // No-op
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 2018a64..4f76875 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1663,8 +1663,12 @@ public class GridCacheUtils {
         if (!F.isEmpty(entities)) {
             Collection<QueryEntity> normalEntities = new ArrayList<>(entities.size());
 
-            for (QueryEntity entity : entities)
+            for (QueryEntity entity : entities) {
+                if (!F.isEmpty(entity.getNotNullFields()))
+                    QueryUtils.checkNotNullAllowed(cfg);
+
                 normalEntities.add(QueryUtils.normalizeQueryEntity(entity, cfg.isSqlEscapeAll()));
+            }
 
             cfg.clearQueryEntities().setQueryEntities(normalEntities);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 58c3ce9..56e8a42 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -39,7 +39,6 @@ import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheKeyConfiguration;
@@ -1355,7 +1354,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param schemaName Schema name to create table in.
      * @param entity Entity to create table from.
      * @param templateName Template name.
-     * @param cacheName
+     * @param cacheName Cache name.
      * @param cacheGroup Cache group name.
      * @param affinityKey Affinity key column name.
      * @param atomicityMode Atomicity mode.
@@ -1389,6 +1388,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new SchemaOperationException("Template cache already contains query entities which it should not: " +
                 templateName);
 
+        if (!F.isEmpty(entity.getNotNullFields()))
+            QueryUtils.checkNotNullAllowed(ccfg);
+
         if (F.isEmpty(cacheName))
             cacheName = QueryUtils.createTableCacheName(schemaName, entity.getTableName());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index e76a6ca..1b61ce9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.property.QueryBinaryProperty;
 import org.apache.ignite.internal.processors.query.property.QueryClassProperty;
 import org.apache.ignite.internal.processors.query.property.QueryFieldAccessor;
@@ -1204,6 +1205,22 @@ public class QueryUtils {
     }
 
     /**
+     * Performs checks to forbid cache configurations that are not compatible with NOT NULL query fields.
+     * See {@link QueryEntity#setNotNullFields(Set)}.
+     *
+     * @param cfg Cache configuration.
+     */
+    public static void checkNotNullAllowed(CacheConfiguration cfg) {
+        if (cfg.isReadThrough())
+            throw new IgniteSQLException("NOT NULL constraint is not supported when CacheConfiguration.readThrough " +
+                "is enabled.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        if (cfg.getInterceptor() != null)
+            throw new IgniteSQLException("NOT NULL constraint is not supported when CacheConfiguration.interceptor " +
+                "is set.", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+    }
+
+    /**
      * Private constructor.
      */
     private QueryUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index 8a901dc..affd903 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -229,6 +229,8 @@ public class DdlStatementsProcessor {
                 else {
                     List<QueryField> cols = new ArrayList<>(cmd.columns().length);
 
+                    boolean allFieldsNullable = true;
+
                     for (GridSqlColumn col : cmd.columns()) {
                         if (tbl.doesColumnExist(col.columnName())) {
                             if ((!cmd.ifNotExists() || cmd.columns().length != 1)) {
@@ -242,14 +244,21 @@ public class DdlStatementsProcessor {
                             }
                         }
 
-                        cols.add(new QueryField(col.columnName(),
+                        QueryField field = new QueryField(col.columnName(),
                             DataType.getTypeClassName(col.column().getType()),
-                            col.column().isNullable()));
+                            col.column().isNullable());
+
+                        cols.add(field);
+
+                        allFieldsNullable &= field.isNullable();
                     }
 
                     if (cols != null) {
                         assert tbl.rowDescriptor() != null;
 
+                        if (!allFieldsNullable)
+                            QueryUtils.checkNotNullAllowed(tbl.cache().config());
+
                         fut = ctx.query().dynamicColumnAdd(tbl.cacheName(), cmd.schemaName(),
                             tbl.rowDescriptor().type().tableName(), cols, cmd.ifTableExists(), cmd.ifNotExists());
                     }
@@ -276,7 +285,7 @@ public class DdlStatementsProcessor {
             throw e;
         }
         catch (Exception e) {
-            throw new IgniteSQLException("Unexpected DLL operation failure: " + e.getMessage(), e);
+            throw new IgniteSQLException("Unexpected DDL operation failure: " + e.getMessage(), e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
index b724f02..b372eb3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlNotNullConstraintTest.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
+import javax.cache.Cache;
 import javax.cache.CacheException;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
@@ -32,6 +33,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.QueryEntity;
@@ -47,6 +49,7 @@ import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -76,9 +79,29 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
     /** Name of SQL table. */
     private static String TABLE_PERSON = "\"" + CACHE_PERSON +  "\".\"PERSON\"";
 
+    /** Template of cache with read-through setting. */
+    private static String CACHE_READ_THROUGH = "cacheReadThrough";
+
+    /** Template of cache with interceptor setting. */
+    private static String CACHE_INTERCEPTOR = "cacheInterceptor";
+
     /** Expected error message. */
     private static String ERR_MSG = "Null value is not allowed for field 'NAME'";
 
+    /** Expected error message for read-through restriction. */
+    private static String READ_THROUGH_ERR_MSG = "NOT NULL constraint is not supported when " +
+        "CacheConfiguration.readThrough is enabled.";
+
+    /** Expected error message for cache interceptor restriction. */
+    private static String INTERCEPTOR_ERR_MSG = "NOT NULL constraint is not supported when " +
+        "CacheConfiguration.interceptor is set.";
+
+    /** Name of the node which configuration includes restricted cache config. */
+    private static String READ_THROUGH_CFG_NODE_NAME = "nodeCacheReadThrough";
+
+    /** Name of the node which configuration includes restricted cache config. */
+    private static String INTERCEPTOR_CFG_NODE_NAME = "nodeCacheInterceptor";
+
     /** OK value. */
     private final Person okValue = new Person("Name", 18);
 
@@ -100,6 +123,18 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
 
         ccfgs.addAll(cacheConfigurations());
 
+        if (gridName.equals(READ_THROUGH_CFG_NODE_NAME)) {
+            ccfgs.add(buildCacheConfigurationRestricted("BadCfgTestCacheRT", true, false, true));
+
+            c.setClientMode(true);
+        }
+
+        if (gridName.equals(INTERCEPTOR_CFG_NODE_NAME)) {
+            ccfgs.add(buildCacheConfigurationRestricted("BadCfgTestCacheINT", false, true, true));
+
+            c.setClientMode(true);
+        }
+
         c.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
 
         if (gridName.equals(NODE_CLIENT)) {
@@ -161,6 +196,30 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /** */
+    private CacheConfiguration buildCacheConfigurationRestricted(String cacheName, boolean readThrough,
+        boolean interceptor, boolean hasQueryEntity) {
+        CacheConfiguration cfg = new CacheConfiguration<Integer, Person>()
+            .setName(cacheName)
+            .setCacheMode(CacheMode.PARTITIONED)
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+
+        if (readThrough) {
+            cfg.setCacheStoreFactory(singletonFactory(new TestStore()));
+            cfg.setReadThrough(true);
+        }
+
+        if (interceptor)
+            cfg.setInterceptor(new TestInterceptor());
+
+        if (hasQueryEntity) {
+            cfg.setQueryEntities(F.asList(new QueryEntity(Integer.class, Person.class)
+                .setNotNullFields(Collections.singleton("name"))));
+        }
+
+        return cfg;
+    }
+
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
@@ -169,6 +228,14 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
 
         startGrid(NODE_CLIENT);
 
+        // Add cache template with read-through cache store.
+        grid(NODE_CLIENT).addCacheConfiguration(
+            buildCacheConfigurationRestricted(CACHE_READ_THROUGH, true, false, false));
+
+        // Add cache template with cache interceptor.
+        grid(NODE_CLIENT).addCacheConfiguration(
+            buildCacheConfigurationRestricted(CACHE_INTERCEPTOR, false, true, false));
+
         awaitPartitionMapExchange();
     }
 
@@ -806,6 +873,86 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
         assertEquals("Bob", result.get(2).get(1));
     }
 
+    /** Check QueryEntity configuration fails with NOT NULL field and read-through. */
+    public void testReadThroughRestrictionQueryEntity() throws Exception {
+        // Node start-up failure (read-through cache store).
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return startGrid(READ_THROUGH_CFG_NODE_NAME);
+            }
+        }, IgniteCheckedException.class, READ_THROUGH_ERR_MSG);
+
+        // Dynamic cache start-up failure (read-through cache store)
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return grid(NODE_CLIENT).createCache(
+                    buildCacheConfigurationRestricted("dynBadCfgCacheRT", true, false, true));
+            }
+        }, IgniteCheckedException.class, READ_THROUGH_ERR_MSG);
+    }
+
+    /** Check QueryEntity configuration fails with NOT NULL field and cache interceptor. */
+    public void testInterceptorRestrictionQueryEntity() throws Exception {
+        // Node start-up failure (interceptor).
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return startGrid(INTERCEPTOR_CFG_NODE_NAME);
+            }
+        }, IgniteCheckedException.class, INTERCEPTOR_ERR_MSG);
+
+        // Dynamic cache start-up failure (interceptor)
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return grid(NODE_CLIENT).createCache(
+                    buildCacheConfigurationRestricted("dynBadCfgCacheINT", false, true, true));
+            }
+        }, IgniteCheckedException.class, INTERCEPTOR_ERR_MSG);
+    }
+
+    /** Check create table fails with NOT NULL field and read-through. */
+    public void testReadThroughRestrictionCreateTable() throws Exception {
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return executeSql("CREATE TABLE test(id INT PRIMARY KEY, name char NOT NULL) " +
+                    "WITH \"template=" + CACHE_READ_THROUGH+ "\"");
+            }
+        }, IgniteSQLException.class, READ_THROUGH_ERR_MSG);
+    }
+
+    /** Check create table fails with NOT NULL field and cache interceptor. */
+    public void testInterceptorRestrictionCreateTable() throws Exception {
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return executeSql("CREATE TABLE test(id INT PRIMARY KEY, name char NOT NULL) " +
+                    "WITH \"template=" + CACHE_INTERCEPTOR + "\"");
+            }
+        }, IgniteSQLException.class, INTERCEPTOR_ERR_MSG);
+    }
+
+    /** Check alter table fails with NOT NULL field and read-through. */
+    public void testReadThroughRestrictionAlterTable() throws Exception {
+        executeSql("CREATE TABLE test(id INT PRIMARY KEY, age INT) " +
+            "WITH \"template=" + CACHE_READ_THROUGH + "\"");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return executeSql("ALTER TABLE test ADD COLUMN name char NOT NULL");
+            }
+        }, IgniteSQLException.class, READ_THROUGH_ERR_MSG);
+    }
+
+    /** Check alter table fails with NOT NULL field and cache interceptor. */
+    public void testInterceptorRestrictionAlterTable() throws Exception {
+        executeSql("CREATE TABLE test(id INT PRIMARY KEY, age INT) " +
+            "WITH \"template=" + CACHE_INTERCEPTOR + "\"");
+
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return executeSql("ALTER TABLE test ADD COLUMN name char NOT NULL");
+            }
+        }, IgniteSQLException.class, INTERCEPTOR_ERR_MSG);
+    }
+
     /** */
     private void executeWithAllCaches(TestClosure clo) throws Exception {
         for (CacheConfiguration ccfg : cacheConfigurations())
@@ -1034,7 +1181,7 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Test store.
+     * Test cache store stub.
      */
     private static class TestStore extends CacheStoreAdapter<Integer, Person> {
         /** {@inheritDoc} */
@@ -1057,4 +1204,34 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
             // No-op
         }
     }
+
+    /**
+     * Test interceptor stub.
+     */
+    private static class TestInterceptor implements CacheInterceptor<Integer, Person> {
+        /** {@inheritDoc} */
+        @Nullable @Override public Person onGet(Integer key, @Nullable Person val) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Person onBeforePut(Cache.Entry<Integer, Person> entry, Person newVal) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAfterPut(Cache.Entry<Integer, Person> entry) {
+            // No-op
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public IgniteBiTuple<Boolean, Person> onBeforeRemove(Cache.Entry<Integer, Person> entry) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAfterRemove(Cache.Entry<Integer, Person> entry) {
+            // No-op
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3256ee27/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index abf8af0..ddf669d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -573,7 +573,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                 WriteBehindEnabled = false,
                 WriteSynchronizationMode = CacheWriteSynchronizationMode.PrimarySync,
                 CacheStoreFactory = new CacheStoreFactoryTest(),
-                ReadThrough = true,
+                ReadThrough = false,
                 WriteThrough = true,
                 WriteBehindCoalescing = false,
                 GroupName = "someGroup",


[14/25] ignite git commit: IGNITE-6382 .NET: Set up NDepend project

Posted by sb...@apache.org.
IGNITE-6382 .NET: Set up NDepend project

This closes #2786


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

Branch: refs/heads/ignite-5937
Commit: b21f750f74d0b6598693184d5925addfe828b7ca
Parents: f42f34b
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Oct 3 13:27:51 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Oct 3 13:27:51 2017 +0300

----------------------------------------------------------------------
 modules/platforms/.gitignore                    |     3 +-
 .../Impl/Binary/BinaryProcessorClient.cs        |     4 +-
 .../Client/Cache/Query/ClientQueryCursor.cs     |     1 -
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |     2 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |     2 +-
 modules/platforms/dotnet/Apache.Ignite.ndproj   | 11139 +++++++++++++++++
 parent/pom.xml                                  |     1 +
 7 files changed, 11146 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/.gitignore
----------------------------------------------------------------------
diff --git a/modules/platforms/.gitignore b/modules/platforms/.gitignore
index 6c05af0..4b415cb 100644
--- a/modules/platforms/.gitignore
+++ b/modules/platforms/.gitignore
@@ -28,4 +28,5 @@ ipch/
 [Rr]elease*/
 packages
 *.classname
-*.nupkg
\ No newline at end of file
+*.nupkg
+NDependOut
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs
index 26a8e9b..816e24a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryProcessorClient.cs
@@ -59,7 +59,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritdoc /> */
         public List<IBinaryType> GetBinaryTypes()
         {
-            throw new NotImplementedException();
+            throw new NotSupportedException();
         }
 
         /** <inheritdoc /> */
@@ -96,7 +96,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** <inheritdoc /> */
         public BinaryType RegisterEnum(string typeName, IEnumerable<KeyValuePair<string, int>> values)
         {
-            throw new NotImplementedException();
+            throw new NotSupportedException();
         }
 
         /** <inheritdoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
index 75aa5df..ff891db 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
@@ -18,7 +18,6 @@
 namespace Apache.Ignite.Core.Impl.Client.Cache.Query
 {
     using System.Collections.Generic;
-    using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using Apache.Ignite.Core.Cache;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
index 55f5be8..a81523a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
@@ -284,7 +284,7 @@ namespace Apache.Ignite.Core.Impl.Events
         /** <inheritDoc /> */
         public void RecordLocal(IEvent evt)
         {
-            throw new NotImplementedException("IGNITE-1410");
+            throw new NotSupportedException("IGNITE-1410");
         }
 
         /** <inheritDoc /> */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index 37a24b1..2cc3659 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -1060,7 +1060,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             return 0;
         }
 
-        private long MemoryReallocate(long memPtr, long cap, long unused, void* arg)
+        private static long MemoryReallocate(long memPtr, long cap, long unused, void* arg)
         {
             IgniteManager.Memory.Get(memPtr).Reallocate((int)cap);
 


[09/25] ignite git commit: IGNITE-6422 Visor CMD: Fixed cache statistics output.

Posted by sb...@apache.org.
IGNITE-6422 Visor CMD: Fixed cache statistics output.


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

Branch: refs/heads/ignite-5937
Commit: 16d23700d25961aae8d75e11c55349fab78f2a79
Parents: 5764960
Author: vsisko <vs...@gridgain.com>
Authored: Mon Oct 2 23:08:40 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Mon Oct 2 23:08:40 2017 +0700

----------------------------------------------------------------------
 .../cache/VisorCacheAggregatedMetrics.java      | 24 ++++++-------
 .../commands/cache/VisorCacheCommand.scala      | 38 ++++++++++----------
 2 files changed, 31 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/16d23700/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
index ccefa67..a0258cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
@@ -208,35 +208,35 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject {
 
     /**
      * @param metric Metrics to process.
-     * @return Off heap entries count.
+     * @return Off heap primary entries count.
      */
-    private long getOffHeapEntriesCount(VisorCacheMetrics metric) {
-        return metric.getOffHeapEntriesCount();
+    private long getOffHeapPrimaryEntriesCount(VisorCacheMetrics metric) {
+        return metric.getOffHeapPrimaryEntriesCount();
     }
 
     /**
-     * @return Minimum number of elements in off heap.
+     * @return Minimum number of primary elements in off heap.
      */
-    public long getMinimumOffHeapSize() {
+    public long getMinimumOffHeapPrimarySize() {
         if (minOffHeapSize == null) {
             minOffHeapSize = Long.MAX_VALUE;
 
             for (VisorCacheMetrics metric : metrics.values())
-                minOffHeapSize = Math.min(minOffHeapSize, getOffHeapEntriesCount(metric));
+                minOffHeapSize = Math.min(minOffHeapSize, getOffHeapPrimaryEntriesCount(metric));
         }
 
         return minOffHeapSize;
     }
 
     /**
-     * @return Average number of elements in off heap.
+     * @return Average number of primary elements in off heap.
      */
-    public double getAverageOffHeapSize() {
+    public double getAverageOffHeapPrimarySize() {
         if (avgOffHeapSize == null) {
             avgOffHeapSize = 0.0d;
 
             for (VisorCacheMetrics metric : metrics.values())
-                avgOffHeapSize += getOffHeapEntriesCount(metric);
+                avgOffHeapSize += getOffHeapPrimaryEntriesCount(metric);
 
             avgOffHeapSize /= metrics.size();
         }
@@ -245,14 +245,14 @@ public class VisorCacheAggregatedMetrics extends VisorDataTransferObject {
     }
 
     /**
-     * @return Maximum number of elements in off heap in the cache.
+     * @return Maximum number of primary elements in off heap.
      */
-    public long getMaximumOffHeapSize() {
+    public long getMaximumOffHeapPrimarySize() {
         if (maxOffHeapSize == null) {
             maxOffHeapSize = Long.MIN_VALUE;
 
             for (VisorCacheMetrics metric : metrics.values())
-                maxOffHeapSize = Math.max(maxOffHeapSize, getOffHeapEntriesCount(metric));
+                maxOffHeapSize = Math.max(maxOffHeapSize, getOffHeapPrimaryEntriesCount(metric));
         }
 
         return maxOffHeapSize;

http://git-wip-us.apache.org/repos/asf/ignite/blob/16d23700/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index f754275..dde3289 100755
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -328,12 +328,12 @@ class VisorCacheCommand {
                         ad.getMode,
                         ad.getNodes.size(),
                         (
-                            "min: " + (ad.getMinimumHeapSize + ad.getMinimumOffHeapSize) +
-                                " (" + ad.getMinimumHeapSize + " / " + ad.getMinimumOffHeapSize + ")",
-                            "avg: " + formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapSize) +
-                                " (" + formatDouble(ad.getAverageHeapSize) + " / " + formatDouble(ad.getAverageOffHeapSize) + ")",
-                            "max: " + (ad.getMaximumHeapSize + ad.getMaximumOffHeapSize) +
-                                " (" + ad.getMaximumHeapSize + " / " + ad.getMaximumOffHeapSize + ")"
+                            "min: " + (ad.getMinimumHeapSize + ad.getMinimumOffHeapPrimarySize) +
+                                " (" + ad.getMinimumHeapSize + " / " + ad.getMinimumOffHeapPrimarySize + ")",
+                            "avg: " + formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapPrimarySize) +
+                                " (" + formatDouble(ad.getAverageHeapSize) + " / " + formatDouble(ad.getAverageOffHeapPrimarySize) + ")",
+                            "max: " + (ad.getMaximumHeapSize + ad.getMaximumOffHeapPrimarySize) +
+                                " (" + ad.getMaximumHeapSize + " / " + ad.getMaximumOffHeapPrimarySize + ")"
                             ),
                         (
                             "min: " + ad.getMinimumHits,
@@ -385,13 +385,13 @@ class VisorCacheCommand {
 
                     csT += ("Name(@)", cacheNameVar)
                     csT += ("Nodes", m.size())
-                    csT += ("Total size Min/Avg/Max", (ad.getMinimumHeapSize + ad.getMinimumOffHeapSize) + " / " +
-                        formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapSize) + " / " +
-                        (ad.getMaximumHeapSize + ad.getMaximumOffHeapSize))
+                    csT += ("Total size Min/Avg/Max", (ad.getMinimumHeapSize + ad.getMinimumOffHeapPrimarySize) + " / " +
+                        formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapPrimarySize) + " / " +
+                        (ad.getMaximumHeapSize + ad.getMaximumOffHeapPrimarySize))
                     csT += ("  Heap size Min/Avg/Max", ad.getMinimumHeapSize + " / " +
                         formatDouble(ad.getAverageHeapSize) + " / " + ad.getMaximumHeapSize)
-                    csT += ("  Off-heap size Min/Avg/Max", ad.getMinimumOffHeapSize + " / " +
-                        formatDouble(ad.getAverageOffHeapSize) + " / " + ad.getMaximumOffHeapSize)
+                    csT += ("  Off-heap size Min/Avg/Max", ad.getMinimumOffHeapPrimarySize + " / " +
+                        formatDouble(ad.getAverageOffHeapPrimarySize) + " / " + ad.getMaximumOffHeapPrimarySize)
 
                     val ciT = VisorTextTable()
 
@@ -408,9 +408,9 @@ class VisorCacheCommand {
                             formatDouble(nm.getCurrentCpuLoad * 100d) + " %",
                             X.timeSpan2HMSM(nm.getUpTime),
                             (
-                                "Total: " + (cm.getHeapEntriesCount + cm.getOffHeapEntriesCount()),
+                                "Total: " + (cm.getHeapEntriesCount + cm.getOffHeapPrimaryEntriesCount),
                                 "  Heap: " + cm.getHeapEntriesCount,
-                                "  Off-Heap: " + cm.getOffHeapEntriesCount(),
+                                "  Off-Heap: " + cm.getOffHeapPrimaryEntriesCount,
                                 "  Off-Heap Memory: " + formatMemory(cm.getOffHeapAllocatedSize)
                             ),
                             (
@@ -644,12 +644,12 @@ class VisorCacheCommand {
                 mkCacheName(ad.getName),
                 ad.getMode,
                 (
-                    "min: " + (ad.getMinimumHeapSize + ad.getMinimumOffHeapSize) +
-                        " (" + ad.getMinimumHeapSize + " / " + ad.getMinimumOffHeapSize + ")",
-                    "avg: " + formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapSize) +
-                        " (" + formatDouble(ad.getAverageHeapSize) + " / " + formatDouble(ad.getAverageOffHeapSize) + ")",
-                    "max: " + (ad.getMaximumHeapSize + ad.getMaximumOffHeapSize) +
-                        " (" + ad.getMaximumHeapSize + " / " + ad.getMaximumOffHeapSize + ")"
+                    "min: " + (ad.getMinimumHeapSize + ad.getMinimumOffHeapPrimarySize) +
+                        " (" + ad.getMinimumHeapSize + " / " + ad.getMinimumOffHeapPrimarySize + ")",
+                    "avg: " + formatDouble(ad.getAverageHeapSize + ad.getAverageOffHeapPrimarySize) +
+                        " (" + formatDouble(ad.getAverageHeapSize) + " / " + formatDouble(ad.getAverageOffHeapPrimarySize) + ")",
+                    "max: " + (ad.getMaximumHeapSize + ad.getMaximumOffHeapPrimarySize) +
+                        " (" + ad.getMaximumHeapSize + " / " + ad.getMaximumOffHeapPrimarySize + ")"
                 ))
         })
 


[22/25] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-3478

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-3478

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
#	modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
#	modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java


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

Branch: refs/heads/ignite-5937
Commit: fc7b7e21c87689866eee83c6e73771333ab0492e
Parents: fdfe779 62f3c4c
Author: sboikov <sb...@gridgain.com>
Authored: Wed Oct 4 11:35:00 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Oct 4 11:35:00 2017 +0300

----------------------------------------------------------------------
 .../FoldersReuseCompatibilityTest.java          | 224 ++++++
 ...itePersistenceCompatibilityAbstractTest.java |   3 +
 .../IgniteCompatibilityBasicTestSuite.java      |   3 +
 .../apache/ignite/IgniteSystemProperties.java   |   7 +
 .../ignite/internal/GridKernalContext.java      |   6 +
 .../ignite/internal/GridKernalContextImpl.java  |  17 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../discovery/GridDiscoveryManager.java         |  55 +-
 .../cache/binary/BinaryMetadataFileStore.java   |   6 +-
 .../GridCacheDatabaseSharedManager.java         |  91 ++-
 .../IgniteCacheDatabaseSharedManager.java       |  13 +-
 .../persistence/file/FilePageStoreManager.java  |  29 +-
 .../filename/PdsConsistentIdProcessor.java      | 568 +++++++++++++++
 .../persistence/filename/PdsFolderSettings.java | 138 ++++
 .../filename/PdsFoldersResolver.java            |  33 +
 .../wal/FileWriteAheadLogManager.java           |  20 +-
 .../wal/reader/StandaloneGridKernalContext.java |  32 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   4 +-
 .../tcp/internal/TcpDiscoveryNode.java          |  15 +
 .../IgniteUidAsConsistentIdMigrationTest.java   | 712 +++++++++++++++++++
 .../db/wal/IgniteWalRecoveryTest.java           |  13 +-
 .../db/wal/reader/IgniteWalReaderTest.java      | 187 +++--
 .../db/wal/reader/MockWalIteratorFactory.java   |  25 +-
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   4 +
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |   2 +-
 25 files changed, 1996 insertions(+), 215 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fc7b7e21/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 88251aa,210b401..184b9a7
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@@ -34,7 -34,7 +34,8 @@@ import org.apache.ignite.internal.manag
  import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
  import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
  import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 +import org.apache.ignite.internal.processors.cache.mvcc.CacheCoordinatorsProcessor;
+ import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
  import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
  import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
  import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
@@@ -646,7 -646,7 +647,12 @@@ public interface GridKernalContext exte
      public PlatformProcessor platform();
  
      /**
 +     * @return Cache mvcc coordinator processor.
 +     */
 +    public CacheCoordinatorsProcessor coordinators();
++
++    /**
+      * @return PDS mode folder name resolver, also generates consistent ID in case new folder naming is used
+      */
+     public PdsFoldersResolver pdsFolderResolver();
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc7b7e21/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 86c0adc,1f0292c..50e1b27
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@@ -49,7 -49,7 +49,8 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager;
  import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
  import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 +import org.apache.ignite.internal.processors.cache.mvcc.CacheCoordinatorsProcessor;
+ import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
  import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
  import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
  import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
@@@ -581,11 -580,11 +585,13 @@@ public class GridKernalContextImpl impl
          else if (comp instanceof PlatformProcessor)
              platformProc = (PlatformProcessor)comp;
          else if (comp instanceof PoolProcessor)
-             poolProc = (PoolProcessor) comp;
+             poolProc = (PoolProcessor)comp;
          else if (comp instanceof GridMarshallerMappingProcessor)
              mappingProc = (GridMarshallerMappingProcessor)comp;
 +        else if (comp instanceof CacheCoordinatorsProcessor)
 +            coordProc = (CacheCoordinatorsProcessor)comp;
+         else if (comp instanceof PdsFoldersResolver)
+             pdsFolderRslvr = (PdsFoldersResolver)comp;
          else if (!(comp instanceof DiscoveryNodeValidationProcessor
                  || comp instanceof PlatformPluginProcessor))
              assert (comp instanceof GridPluginComponent) : "Unknown manager class: " + comp.getClass();

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc7b7e21/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 2dbbb7c,759bf64..58981d5
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@@ -114,8 -114,8 +114,9 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
  import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
  import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 +import org.apache.ignite.internal.processors.cache.mvcc.CacheCoordinatorsProcessor;
  import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicy;
+ import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
  import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
  import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
  import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
@@@ -938,9 -938,9 +939,10 @@@ public class IgniteKernal implements Ig
              // Start processors before discovery manager, so they will
              // be able to start receiving messages once discovery completes.
              try {
+                 startProcessor(new PdsConsistentIdProcessor(ctx));
 +                startProcessor(new CacheCoordinatorsProcessor(ctx));
                  startProcessor(createComponent(DiscoveryNodeValidationProcessor.class, ctx));
 -                startProcessor(new  GridAffinityProcessor(ctx));
 +                startProcessor(new GridAffinityProcessor(ctx));
                  startProcessor(createComponent(GridSegmentationProcessor.class, ctx));
                  startProcessor(createComponent(IgniteCacheObjectProcessor.class, ctx));
                  startProcessor(new GridClusterStateProcessor(ctx));

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc7b7e21/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/fc7b7e21/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index db575f9,e234766..f01a2b1
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@@ -47,7 -48,8 +48,9 @@@ import org.apache.ignite.internal.manag
  import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
  import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
  import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 +import org.apache.ignite.internal.processors.cache.mvcc.CacheCoordinatorsProcessor;
+ import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
+ import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
  import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
  import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
  import org.apache.ignite.internal.processors.cluster.ClusterProcessor;


[05/25] ignite git commit: IGNITE-5915 - Added warning for WAL NONE mode

Posted by sb...@apache.org.
IGNITE-5915 - Added warning for WAL NONE mode


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

Branch: refs/heads/ignite-5937
Commit: f57b6776775308753a1fa579c0e7599df241be61
Parents: 252eedc
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Oct 2 15:05:42 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Oct 2 15:06:12 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/configuration/WALMode.java | 14 ++++++++++----
 .../persistence/wal/FileWriteAheadLogManager.java     |  5 ++++-
 2 files changed, 14 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f57b6776/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java b/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java
index 9d4520e..bea0cba 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/WALMode.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.configuration;
 
+import org.apache.ignite.Ignite;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -24,22 +25,27 @@ import org.jetbrains.annotations.Nullable;
  */
 public enum WALMode {
     /**
-     * Default mode: full-sync disk writes. These writes survive power loss scenarios.
+     * Default mode: full-sync disk writes. These writes survive power loss scenarios. When a control is returned
+     * from the trnasaction commit operation, the changes are guaranteed to be forced on disk according to the
+     * transaction write synchronization mode.
      */
     DEFAULT,
 
     /**
-     * Log only mode: flushes application buffers. These writes survive process crash.
+     * Log only mode: flushes application buffers. These writes survive process crash. When a control is returned
+     * from the transaction commit operation, the changes are guaranteed to be forced to the OS buffer cache.
      */
     LOG_ONLY,
 
     /**
-     * Background mode. Does not force application buffer flush. Data may be lost in case of process crash.
+     * Background mode. Does not force application buffer flush. Last updates may be lost in case of a process crash.
      */
     BACKGROUND,
 
     /**
-     * WAL disabled.
+     * WAL is disabled. Data is guaranteed to be persisted on disk only in case of graceful cluster shutdown using
+     * {@link Ignite#active(boolean)} method. If an Ignite node is forefully terminated in NONE mode, it is likely
+     * that data stored on disk is corrupted and work directory will need to be cleared for a successfull node restart.
      */
     NONE;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f57b6776/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 76edcea..c8715aa 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
@@ -294,10 +294,13 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
 
             archiver = new FileArchiver(tup == null ? -1 : tup.get2());
 
-            if (mode != WALMode.DEFAULT) {
+            if (mode != WALMode.NONE) {
                 if (log.isInfoEnabled())
                     log.info("Started write-ahead log manager [mode=" + mode + ']');
             }
+            else
+                U.quietAndWarn(log, "Started write-ahead log manager in NONE mode, persisted data may be lost in " +
+                    "a case of unexpected node failure. Make sure to deactivate the cluster before shutdown.");
         }
     }
 


[03/25] ignite git commit: IGNITE-6485: Binary marshalling with writeReplace/readResolve fixed. This closes #2778.

Posted by sb...@apache.org.
IGNITE-6485: Binary marshalling with writeReplace/readResolve fixed. This closes #2778.


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

Branch: refs/heads/ignite-5937
Commit: d7fbbd546126252af45cb48a5d173a7a66ad9838
Parents: 9160d5e
Author: Andrey Gura <ag...@apache.org>
Authored: Mon Oct 2 12:38:34 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 2 12:38:34 2017 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryClassDescriptor.java  |   4 +-
 .../ignite/internal/util/IgniteUtils.java       |  43 ++++--
 .../binary/BinaryMarshallerSelfTest.java        | 136 +++++++++++++------
 3 files changed, 130 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d7fbbd54/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
index 4950a53..935211e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -344,9 +344,9 @@ public class BinaryClassDescriptor {
         Method writeReplaceMthd;
 
         if (mode == BinaryWriteMode.BINARY || mode == BinaryWriteMode.OBJECT) {
-            readResolveMtd = U.findNonPublicMethod(cls, "readResolve");
+            readResolveMtd = U.getNonPublicMethod(cls, "readResolve");
 
-            writeReplaceMthd = U.findNonPublicMethod(cls, "writeReplace");
+            writeReplaceMthd = U.getNonPublicMethod(cls, "writeReplace");
         }
         else {
             readResolveMtd = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7fbbd54/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 31b556d..bdcf87e 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
@@ -9407,22 +9407,14 @@ public abstract class IgniteUtils {
      * @param cls The class to search,
      * @param name Name of the method.
      * @param paramTypes Method parameters.
-     * @return Method or {@code null}
+     * @return Method or {@code null}.
      */
     @Nullable public static Method findNonPublicMethod(Class<?> cls, String name, Class<?>... paramTypes) {
         while (cls != null) {
-            try {
-                Method mtd = cls.getDeclaredMethod(name, paramTypes);
+            Method mtd = getNonPublicMethod(cls, name, paramTypes);
 
-                if (mtd.getReturnType() != void.class) {
-                    mtd.setAccessible(true);
-
-                    return mtd;
-                }
-            }
-            catch (NoSuchMethodException ignored) {
-                // No-op.
-            }
+            if (mtd != null)
+                return mtd;
 
             cls = cls.getSuperclass();
         }
@@ -9431,6 +9423,33 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Gets a method from the class.
+     *
+     * Method.getMethod() does not return non-public method.
+     *
+     * @param cls Target class.
+     * @param name Name of the method.
+     * @param paramTypes Method parameters.
+     * @return Method or {@code null}.
+     */
+    @Nullable public static Method getNonPublicMethod(Class<?> cls, String name, Class<?>... paramTypes) {
+        try {
+            Method mtd = cls.getDeclaredMethod(name, paramTypes);
+
+            if (mtd.getReturnType() != void.class) {
+                mtd.setAccessible(true);
+
+                return mtd;
+            }
+        }
+        catch (NoSuchMethodException ignored) {
+            // No-op.
+        }
+
+        return null;
+    }
+
+    /**
      * @param cls The class to search.
      * @param name Name of a field to get.
      * @return Field or {@code null}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d7fbbd54/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
index 926b3c0..ef68cd1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
@@ -938,6 +938,23 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testWriteReplace() throws Exception {
+        BinaryMarshaller marsh = binaryMarshaller(Collections.singleton(
+            new BinaryTypeConfiguration(TestObject.class.getName())
+        ));
+
+        TestObject obj = new TestObject();
+
+        BinaryObject po = marshal(obj, marsh);
+
+        assertEquals(obj, po.deserialize());
+
+        assertEquals(obj.val, ((BinaryObject)po.field("val")).deserialize());
+    }
+
+    /**
      *
      */
     private static class EnclosingObj implements Serializable {
@@ -2626,19 +2643,6 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception If ecxeption thrown.
-     */
-    public void testDeclareReadResolveInParent() throws Exception {
-        BinaryMarshaller marsh = binaryMarshaller(Arrays.asList(new BinaryTypeConfiguration(ChildBinary.class.getName())));
-
-        BinaryObjectImpl binaryObj = marshal(new ChildBinary(), marsh);
-
-        ChildBinary singleton = binaryObj.deserialize();
-
-        assertNotNull(singleton.s);
-    }
-
-    /**
      *
      */
     public void testDecimalFields() throws Exception {
@@ -5117,13 +5121,6 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
-    public static class ChildBinary extends ParentBinary {
-
-    }
-
-    /**
-     *
-     */
     public static class SimpleEnclosingObject {
         /** */
         private Object simpl;
@@ -5180,25 +5177,6 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
-     */
-    private static class ParentBinary {
-        /** */
-        public String s;
-
-        /**
-         * Package only visibility!!!!
-         *
-         * @return Object.
-         */
-        Object readResolve() {
-            s = "readResolve";
-
-            return this;
-        }
-    }
-
-    /**
      * Class B for duplicate fields test.
      */
     private static class DuplicateFieldsA {
@@ -5517,4 +5495,84 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
 
         abstract boolean isSupported();
     }
+
+    /** */
+    interface Intf {
+        /** */
+        long value();
+    }
+
+    /** */
+    static class TestObject {
+        /** Value. */
+        Intf val = new IntfImpl();
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestObject obj = (TestObject)o;
+
+            return val.equals(obj.val);
+        }
+    }
+
+    /** */
+    static class IntfImpl extends Cls implements Intf {
+        /** {@inheritDoc} */
+        @Override public long value() {
+            return longValue();
+        }
+    }
+
+    /** */
+    static class Cls implements Serializable {
+        /** Value. */
+        long val;
+
+        /** */
+        public long longValue() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            Cls cls = (Cls)o;
+
+            return val == cls.val;
+        }
+
+        /** */
+        private Object writeReplace() {
+            return new SerializationProxy(this);
+        }
+
+        /** */
+        private static class SerializationProxy implements Serializable {
+            /** Value. */
+            private final long val;
+
+            /** */
+            SerializationProxy(Cls a) {
+                val = a.longValue();
+            }
+
+            /** */
+            private Object readResolve() {
+                Cls a = new Cls();
+
+                a.val = val;
+
+                return a;
+            }
+        }
+    }
 }
\ No newline at end of file


[12/25] ignite git commit: IGNITE-6382 .NET: Set up NDepend project

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 4ee4310..b133b1e 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -882,6 +882,7 @@
                                         <exclude>**/*.FxCop</exclude>
                                         <exclude>**/*.ruleset</exclude>
                                         <exclude>**/*.csproj</exclude>
+                                        <exclude>**/*.ndproj</exclude>
                                         <exclude>**/*.csprojrel</exclude>
                                         <exclude>**/*.vcxproj</exclude>
                                         <exclude>**/*.vcxprojrel</exclude>


[24/25] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-3478' into ignite-5937

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-3478' into ignite-5937


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

Branch: refs/heads/ignite-5937
Commit: e6940bd5fe00cc77d1d27facfc56964c5b03c9fe
Parents: c4f98f3 a1d9ddd
Author: sboikov <sb...@gridgain.com>
Authored: Wed Oct 4 11:38:37 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Oct 4 11:38:37 2017 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     |    46 +
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |    88 +-
 .../FoldersReuseCompatibilityTest.java          |   224 +
 ...itePersistenceCompatibilityAbstractTest.java |     3 +
 .../IgniteCompatibilityBasicTestSuite.java      |     3 +
 .../apache/ignite/IgniteSystemProperties.java   |     7 +
 .../apache/ignite/configuration/WALMode.java    |    14 +-
 .../ignite/internal/GridKernalContext.java      |     8 +-
 .../ignite/internal/GridKernalContextImpl.java  |    17 +-
 .../apache/ignite/internal/IgniteKernal.java    |     6 +-
 .../internal/binary/BinaryClassDescriptor.java  |     4 +-
 .../discovery/GridDiscoveryManager.java         |    55 +-
 .../affinity/GridAffinityAssignmentCache.java   |     2 +-
 .../cache/GridCacheAffinityManager.java         |     2 +-
 .../processors/cache/GridCacheUtils.java        |     6 +-
 .../cache/binary/BinaryMetadataFileStore.java   |     6 +-
 .../dht/GridPartitionedGetFuture.java           |    21 +-
 .../dht/GridPartitionedSingleGetFuture.java     |     6 +-
 .../distributed/near/GridNearGetFuture.java     |     6 +-
 .../GridCacheDatabaseSharedManager.java         |    91 +-
 .../IgniteCacheDatabaseSharedManager.java       |    13 +-
 .../persistence/file/FilePageStoreManager.java  |    29 +-
 .../filename/PdsConsistentIdProcessor.java      |   568 +
 .../persistence/filename/PdsFolderSettings.java |   138 +
 .../filename/PdsFoldersResolver.java            |    33 +
 .../wal/FileWriteAheadLogManager.java           |    25 +-
 .../wal/reader/StandaloneGridKernalContext.java |    31 +-
 .../datastreamer/PlatformDataStreamer.java      |    14 +
 .../processors/query/GridQueryProcessor.java    |     6 +-
 .../internal/processors/query/QueryUtils.java   |    17 +
 .../processors/rest/GridRestCommand.java        |     2 +-
 .../processors/rest/GridRestProcessor.java      |     5 +-
 .../ignite/internal/util/IgniteUtils.java       |    43 +-
 .../cache/VisorCacheAggregatedMetrics.java      |    24 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |     4 +-
 .../tcp/internal/TcpDiscoveryNode.java          |    15 +
 .../binary/BinaryMarshallerSelfTest.java        |   136 +-
 .../IgniteUidAsConsistentIdMigrationTest.java   |   712 ++
 .../db/wal/IgniteWalRecoveryTest.java           |    13 +-
 .../db/wal/reader/IgniteWalReaderTest.java      |   187 +-
 .../db/wal/reader/MockWalIteratorFactory.java   |    25 +-
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |     4 +
 .../processors/query/h2/IgniteH2Indexing.java   |     3 +
 .../query/h2/ddl/DdlStatementsProcessor.java    |    15 +-
 .../query/IgniteSqlNotNullConstraintTest.java   |   179 +-
 .../query/IgniteSqlParameterizedQueryTest.java  |   392 +
 .../IgniteCacheQuerySelfTestSuite.java          |     3 +-
 modules/platforms/.gitignore                    |     3 +-
 .../Cache/CacheConfigurationTest.cs             |     2 +-
 .../Dataload/DataStreamerTest.cs                |    50 +-
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |     2 +-
 .../Apache.Ignite.Core.csproj                   |     1 +
 .../Datastream/DataStreamerDefaults.cs          |    46 +
 .../Datastream/IDataStreamer.cs                 |    21 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |     2 +
 .../Impl/Binary/BinaryProcessorClient.cs        |     4 +-
 .../Impl/Binary/BinaryReaderExtensions.cs       |    10 +-
 .../Impl/Binary/BinaryUtils.cs                  |    14 +
 .../Impl/Cache/Query/QueryCursorBase.cs         |     3 +
 .../Client/Cache/Query/ClientQueryCursor.cs     |     2 +
 .../Impl/Client/ClientSocket.cs                 |     2 +
 .../Impl/Datastream/DataStreamerImpl.cs         |    45 +-
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |     2 +-
 .../Impl/PlatformJniTarget.cs                   |     6 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |     3 +-
 modules/platforms/dotnet/Apache.Ignite.ndproj   | 11139 +++++++++++++++++
 .../http/jetty/GridJettyRestHandler.java        |    18 +
 .../rest/protocols/http/jetty/favicon.ico       |   Bin 1406 -> 1150 bytes
 .../commands/cache/VisorCacheCommand.scala      |    38 +-
 parent/pom.xml                                  |     1 +
 70 files changed, 14294 insertions(+), 371 deletions(-)
----------------------------------------------------------------------



[07/25] ignite git commit: .NET: Suppress some false VS2017 warnings

Posted by sb...@apache.org.
.NET: Suppress some false VS2017 warnings


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

Branch: refs/heads/ignite-5937
Commit: 5ca7909fdfb129eed1a58650f57d7dd968b1e287
Parents: 7aaacd8
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Oct 2 16:34:12 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Oct 2 16:34:12 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs        | 2 ++
 .../Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs     | 3 +++
 .../Impl/Client/Cache/Query/ClientQueryCursor.cs               | 3 +++
 .../dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs      | 2 ++
 .../dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs        | 6 ++++++
 .../Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs    | 1 +
 6 files changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
index 886dee9..4e5eae5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs
@@ -454,6 +454,8 @@ namespace Apache.Ignite.Core
         /// </summary>
         /// <param name="interopProc">Interop processor.</param>
         /// <param name="stream">Stream.</param>
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope",
+            Justification = "PlatformJniTarget is passed further")]
         internal static void OnStart(IUnmanagedTarget interopProc, IBinaryStream stream)
         {
             try

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs
index 8f00a2e..216d7ea 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Query/QueryCursorBase.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Impl.Cache.Query
     using System.Collections;
     using System.Collections.Generic;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
@@ -63,6 +64,8 @@ namespace Apache.Ignite.Core.Impl.Cache.Query
         /// <param name="marsh">Marshaller.</param>
         /// <param name="keepBinary">Keep binary flag.</param>
         /// <param name="initialBatchStream">Optional stream with initial batch.</param>
+        [SuppressMessage("Microsoft.Usage", "CA2214:DoNotCallOverridableMethodsInConstructors",
+            Justification = "ConvertGetBatch calls Read, which does not rely on constructor being run.")]
         protected QueryCursorBase(Marshaller marsh, bool keepBinary, IBinaryStream initialBatchStream = null)
         {
             Debug.Assert(marsh != null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
index 898bbbe..75aa5df 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/Query/ClientQueryCursor.cs
@@ -18,6 +18,8 @@
 namespace Apache.Ignite.Core.Impl.Client.Cache.Query
 {
     using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Impl.Binary;
@@ -65,6 +67,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cache.Query
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         protected override ICacheEntry<TK, TV> Read(BinaryReader reader)
         {
             return new CacheEntry<TK, TV>(reader.ReadObject<TK>(), reader.ReadObject<TV>());

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
index 4eb53dd..e565f31 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
@@ -201,6 +201,8 @@ namespace Apache.Ignite.Core.Impl.Client
         /// <summary>
         /// Connects the socket.
         /// </summary>
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope", 
+            Justification = "Socket is returned from this method.")]
         private static Socket Connect(IgniteClientConfiguration cfg)
         {
             List<Exception> errors = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs
index 725c112..c7d9ab4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformJniTarget.cs
@@ -174,6 +174,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")]
         public unsafe TR InObjectStreamOutObjectStream<TR>(int type, Action<IBinaryStream> writeAction, 
             Func<IBinaryStream, IPlatformTargetInternal, TR> readAction, IPlatformTargetInternal arg)
         {
@@ -244,6 +245,7 @@ namespace Apache.Ignite.Core.Impl
         /// <param name="keepBinary">Keep binary flag, only applicable to object futures. False by default.</param>
         /// <param name="convertFunc">The function to read future result from stream.</param>
         /// <returns>Created future.</returns>
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")]
         private Future<T> GetFuture<T>(Func<long, int, IUnmanagedTarget> listenAction, bool keepBinary = false,
             Func<BinaryReader, T> convertFunc = null)
         {
@@ -325,6 +327,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public long InStreamOutLong(int type, Action<IBinaryRawWriter> writeAction)
         {
             using (var stream = IgniteManager.Memory.Allocate().GetStream())
@@ -340,6 +343,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public T InStreamOutStream<T>(int type, Action<IBinaryRawWriter> writeAction, 
             Func<IBinaryRawReader, T> readAction)
         {
@@ -361,6 +365,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public IPlatformTarget InStreamOutObject(int type, Action<IBinaryRawWriter> writeAction)
         {
             using (var stream = IgniteManager.Memory.Allocate().GetStream())
@@ -429,6 +434,7 @@ namespace Apache.Ignite.Core.Impl
         }
 
         /** <inheritdoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public T OutStream<T>(int type, Func<IBinaryRawReader, T> readAction)
         {
             using (var stream = IgniteManager.Memory.Allocate().GetStream())

http://git-wip-us.apache.org/repos/asf/ignite/blob/5ca7909f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
index b291b3d..37a24b1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedCallbacks.cs
@@ -1210,6 +1210,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
 
         #region AffinityFunction
 
+        [SuppressMessage("Microsoft.Reliability", "CA2000:Dispose objects before losing scope")]
         private long AffinityFunctionInit(long memPtr, long unused, long unused1, void* baseFunc)
         {
             using (var stream = IgniteManager.Memory.Get(memPtr).GetStream())


[02/25] ignite git commit: IGNITE-6520: Using actual AffinityReadyFuture result

Posted by sb...@apache.org.
IGNITE-6520: Using actual AffinityReadyFuture result


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

Branch: refs/heads/ignite-5937
Commit: 9160d5eaf75d07ef16f6fe02b09aaa2ed76132b5
Parents: 013d7db
Author: Andrey Kuznetsov <st...@gmail.com>
Authored: Mon Oct 2 11:28:32 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 2 11:28:32 2017 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridPartitionedGetFuture.java        | 6 ++----
 .../cache/distributed/dht/GridPartitionedSingleGetFuture.java  | 6 ++----
 .../processors/cache/distributed/near/GridNearGetFuture.java   | 6 ++----
 3 files changed, 6 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9160d5ea/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 73c0ea5..015eb82 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -713,17 +713,15 @@ public class GridPartitionedGetFuture<K, V> extends CacheDistributedGetFutureAda
                 onDone(Collections.<K, V>emptyMap());
             }
             else {
-                final AffinityTopologyVersion updTopVer =
+                AffinityTopologyVersion updTopVer =
                     new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 
                 cctx.affinity().affinityReadyFuture(updTopVer).listen(
                     new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
                             try {
-                                fut.get();
-
                                 // Remap.
-                                map(keys.keySet(), F.t(node, keys), updTopVer);
+                                map(keys.keySet(), F.t(node, keys), fut.get());
 
                                 onDone(Collections.<K, V>emptyMap());
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9160d5ea/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
index 234ee91..61489e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedSingleGetFuture.java
@@ -703,16 +703,14 @@ public class GridPartitionedSingleGetFuture extends GridCacheFutureAdapter<Objec
             return false;
 
         if (canRemap) {
-            final AffinityTopologyVersion updTopVer = new AffinityTopologyVersion(
+            AffinityTopologyVersion updTopVer = new AffinityTopologyVersion(
                 Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 
             cctx.affinity().affinityReadyFuture(updTopVer).listen(
                 new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
                         try {
-                            fut.get();
-
-                            remap(updTopVer);
+                            remap(fut.get());
                         }
                         catch (IgniteCheckedException e) {
                             onDone(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/9160d5ea/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 807270d..2361507 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -917,17 +917,15 @@ public final class GridNearGetFuture<K, V> extends CacheDistributedGetFutureAdap
                 onDone(Collections.<K, V>emptyMap());
             }
             else {
-                final AffinityTopologyVersion updTopVer =
+                AffinityTopologyVersion updTopVer =
                     new AffinityTopologyVersion(Math.max(topVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
 
                 cctx.affinity().affinityReadyFuture(updTopVer).listen(
                     new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
                             try {
-                                fut.get();
-
                                 // Remap.
-                                map(keys.keySet(), F.t(node, keys), updTopVer);
+                                map(keys.keySet(), F.t(node, keys), fut.get());
 
                                 onDone(Collections.<K, V>emptyMap());
                             }


[13/25] ignite git commit: IGNITE-6382 .NET: Set up NDepend project

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b21f750f/modules/platforms/dotnet/Apache.Ignite.ndproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.ndproj b/modules/platforms/dotnet/Apache.Ignite.ndproj
new file mode 100644
index 0000000..9f5287b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.ndproj
@@ -0,0 +1,11139 @@
+<?xml version="1.0" encoding="utf-8" standalone="yes"?>
+<NDepend AppName="Apache.Ignite" Platform="DotNet" FileWrittenByProductVersion="2017.2.2.8962">
+  <OutputDir KeepXmlFiles="False">c:\w\incubator-ignite\modules\platforms\dotnet\NDependOut</OutputDir>
+  <Assemblies>
+    <Name>Apache.Ignite.Core</Name>
+  </Assemblies>
+  <FrameworkAssemblies>
+    <Name>mscorlib</Name>
+    <Name>System.Core</Name>
+    <Name>System.Xml</Name>
+    <Name>System</Name>
+    <Name>System.Configuration</Name>
+    <Name>System.Transactions</Name>
+  </FrameworkAssemblies>
+  <Dirs>
+    <Dir>C:\WINDOWS\Microsoft.NET\Framework\v4.0.30319</Dir>
+    <Dir>C:\WINDOWS\Microsoft.NET\Framework\v4.0.30319\WPF</Dir>
+    <Dir>$(NdProjectDir)\Apache.Ignite.Core\bin\Debug</Dir>
+  </Dirs>
+  <MergeCodeGeneratedByCompiler>True</MergeCodeGeneratedByCompiler>
+  <Report Kind="0" SectionsEnabled="110591" XslPath="" Flags="261120" />
+  <BuildComparisonSetting ProjectMode="CurrentProject" BuildMode="NDaysAgoAnalysisResult" ProjectFileToCompareWith="" BuildFileToCompareWith="" NDaysAgo="30" />
+  <BaselineInUISetting ProjectMode="CurrentProject" BuildMode="NDaysAgoAnalysisResult" ProjectFileToCompareWith="" BuildFileToCompareWith="" NDaysAgo="30" />
+  <CoverageFiles CoverageDir="" UncoverableAttribute="" />
+  <TrendMetrics UseCustomLog="False" LogRecurrence="3" LogLabel="2" UseCustomDir="False" CustomDir="">
+    <Chart Name="Size" ShowInReport="True">
+      <Serie MetricName="# Lines of Code" MetricUnit="Loc" Color="#FF00BFFF" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Lines of Code Covered" MetricUnit="Loc" Color="#FF32CD32" ChartType="Area" ScaleExp="0" />
+      <Serie MetricName="# Lines of Code (NotMyCode)" MetricUnit="Loc" Color="#FFA9A9A9" ChartType="Area" ScaleExp="0" />
+      <Serie MetricName="# Lines of Comments" MetricUnit="Lines" Color="#FF008000" ChartType="Line" ScaleExp="0" />
+    </Chart>
+    <Chart Name="% Coverage and % Debt" ShowInReport="True">
+      <Serie MetricName="Percentage Code Coverage" MetricUnit="%" Color="#FF32CD32" ChartType="Area" ScaleExp="0" />
+      <Serie MetricName="Percentage Debt (Metric)" MetricUnit="%" Color="#FFFF0000" ChartType="Line" ScaleExp="0" />
+    </Chart>
+    <Chart Name="Issues" ShowInReport="True">
+      <Serie MetricName="# New Issues since Baseline" MetricUnit="issues" Color="#FFFF0000" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Issues Fixed since Baseline" MetricUnit="issues" Color="#FF32CD32" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Blocker/Critical/Major Issues" MetricUnit="issues" Color="#FFFF8C00" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Issues" MetricUnit="issues" Color="#FFFFD700" ChartType="Line" ScaleExp="-2" />
+    </Chart>
+    <Chart Name="Rules" ShowInReport="True">
+      <Serie MetricName="# Rules" MetricUnit="Rules" Color="#FF66CDAA" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Rules Violated" MetricUnit="Rules" Color="#FFFF8C00" ChartType="Area" ScaleExp="0" />
+      <Serie MetricName="# Critical Rules Violated" MetricUnit="Rules" Color="#FFFF0000" ChartType="Area" ScaleExp="0" />
+    </Chart>
+    <Chart Name="Quality Gates" ShowInReport="True">
+      <Serie MetricName="# Quality Gates Fail" MetricUnit="quality gates" Color="#FFFF0000" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Quality Gates Warn" MetricUnit="quality gates" Color="#FFFF8C00" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="# Quality Gates" MetricUnit="quality gates" Color="#FF32CD32" ChartType="Line" ScaleExp="0" />
+    </Chart>
+    <Chart Name="Debt" ShowInReport="True">
+      <Serie MetricName="Debt (Metric)" MetricUnit="man-days" Color="#FFFF0000" ChartType="Line" ScaleExp="0" />
+      <Serie MetricName="Annual Interest (Metric)" MetricUnit="man-days" Color="#FFFF8C00" ChartType="Line" ScaleExp="0" />
+    </Chart>
+  </TrendMetrics>
+  <HistoricAnalysisResult PersistRecurrence="3" UseCustomDir="False" CustomDir="" />
+  <SourceFileRebasing FromPath="" ToPath="" />
+  <PathVariables />
+  <RuleFiles />
+  <ProjectRules AreActive="True" />
+  <ProjectDebtSettings DebtSettingsStorage="0" SettingsFilePath="">
+    <DebtSettings>
+      <DebtFactor>1</DebtFactor>
+      <AnnualInterestFactor>1</AnnualInterestFactor>
+      <DebtDefault>0</DebtDefault>
+      <AnnualInterestDefault>0</AnnualInterestDefault>
+      <DebtStringFormat>$ManDay$</DebtStringFormat>
+      <MoneyPerManHour>50</MoneyPerManHour>
+      <Currency>USD</Currency>
+      <CurrencyLocation>After</CurrencyLocation>
+      <EstimatedNumberOfManDayToDevelop1000LogicalLinesOfCode>18</EstimatedNumberOfManDayToDevelop1000LogicalLinesOfCode>
+      <NumberOfWorkDayPerYear>240</NumberOfWorkDayPerYear>
+      <NumberOfWorkHourPerDay>8</NumberOfWorkHourPerDay>
+      <A2B_RatingThreshold>5</A2B_RatingThreshold>
+      <B2C_RatingThreshold>10</B2C_RatingThreshold>
+      <C2D_RatingThreshold>20</C2D_RatingThreshold>
+      <D2E_RatingThreshold>50</D2E_RatingThreshold>
+      <Low2Medium_SeverityThreshold>1200000000</Low2Medium_SeverityThreshold>
+      <Medium2High_SeverityThreshold>12000000000</Medium2High_SeverityThreshold>
+      <High2Critical_SeverityThreshold>72000000000</High2Critical_SeverityThreshold>
+      <Critical2Blocker_SeverityThreshold>360000000000</Critical2Blocker_SeverityThreshold>
+    </DebtSettings>
+  </ProjectDebtSettings>
+  <Queries>
+    <Group Name="Quality Gates" Active="True" ShownInReport="True">
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Quality Gates Evolution</Name>
+from qg in QualityGates
+let qgBaseline = qg.OlderVersion()
+let relyOnDiff = qgBaseline == null
+let evolution = relyOnDiff ? (TrendIcon?)null : 
+                // When a quality gate relies on diff between now and baseline
+                // it is not executed against the baseline
+                qg.ValueDiff() == 0d ?
+                TrendIcon.Constant :
+                (qg.ValueDiff() > 0 ? 
+                  ( qg.MoreIsBad ?  TrendIcon.RedUp: TrendIcon.GreenUp) :
+                  (!qg.MoreIsBad ?  TrendIcon.RedDown: TrendIcon.GreenDown))
+select new { qg, 
+   Evolution      =  evolution,
+
+   BaselineStatus =  relyOnDiff? (QualityGateStatus?) null : qgBaseline.Status,
+   Status         =  qg.Status,
+
+   BaselineValue  =  relyOnDiff? (null) : qgBaseline.ValueString,
+   Value          =  qg.ValueString, 
+}
+ 
+// <Description>
+// Show quality gates evolution between baseline and now.
+//
+// When a quality gate relies on diff between now and baseline (like *New Debt since Baseline*)
+// it is not executed against the baseline and as a consequence its evolution is not available.
+//
+// Double-click a quality gate for editing.
+// </Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Percentage Coverage" Unit="%" />
+failif value < 70%
+warnif value < 80%
+codeBase.PercentageCoverage
+
+//<Description>
+// Code coverage is a measure used to describe the degree to which the source code of a program 
+// is tested by a particular test suite. A program with high code coverage, measured as a percentage, 
+// has had more of its source code executed during testing which suggests it has a lower chance of 
+// containing undetected software bugs compared to a program with low code coverage.
+//
+// Code coverage is certainly the most important quality code metric. But coverage is not enough
+// the team needs to ensure that results are checked at test-time. These checks can be done both 
+// in test code, and in application code through assertions. The important part is that a test
+// must fail explicitely when a check gets unvalidated during the test execution.
+//
+// This quality gate define a warn threshold (70%) and a fail threshold (80%). These are 
+// indicative thresholds and in practice the more the better. To achieve high coverage and 
+// low risk, make sure that new and refactored classes gets 100% covered by tests and that
+// the application and test code contains as many checks/assertions as possible.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Percentage Coverage on New Code" Unit="%" />
+failif value < 70%
+warnif value < 80%
+let newMethods = Application.Methods.Where(m => m.WasAdded() && m.NbLinesOfCode > 0)
+let locCovered = newMethods.Sum(m => m.NbLinesOfCodeCovered)
+let loc = newMethods.Sum(m => m.NbLinesOfCode)
+select 100d * locCovered / loc
+
+//<Description>
+// *New Code* is defined as methods added since the baseline.
+//
+// To achieve high code coverage it is essential that new code gets properly
+// tested and covered by tests. It is advised that all non-UI new classes gets
+// 100% covered.
+//
+// Typically 90% of a class is easy to cover by tests and 10% is hard to reach 
+// through tests. It means that this 10% remaining is not easily testable, which 
+// means it is not well designed, which often means that this code is especially 
+// **error-prone**. This is the reason why it is important to reach 100% coverage
+// for a class, to make sure that potentially *error-prone* code gets tested.
+//</Description>
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Percentage Coverage on Refactored Code" Unit="%" />
+failif value < 70%
+warnif value < 80%
+let newMethods = Application.Methods.Where(m => m.CodeWasChanged() && m.NbLinesOfCode > 0)
+let locCovered = newMethods.Sum(m => m.NbLinesOfCodeCovered)
+let loc = newMethods.Sum(m => m.NbLinesOfCode)
+select 100d * locCovered / loc
+
+//<Description>
+// *Refactored Code* is defined as methods where *code was changed* since the baseline.
+//
+// Comment changes and formatting changes are not considerd as refactoring.
+//
+// To achieve high code coverage it is essential that refactored code gets properly
+// tested and covered by tests. It is advised that when refactoring a class
+// or a method, it is important to also write tests to make sure it gets 100% covered.
+//
+// Typically 90% of a class is easy to cover by tests and 10% is hard to reach 
+// through tests. It means that this 10% remaining is not easily testable, which 
+// means it is not well designed, which often means that this code is especially 
+// **error-prone**. This is the reason why it is important to reach 100% coverage
+// for a class, to make sure that potentially *error-prone* code gets tested.
+//</Description>
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Blocker Issues" Unit="issues" />
+failif count > 0 issues
+from i in Issues
+where i.Severity == Severity.Blocker
+select new { i, i.Severity, i.Debt, i.AnnualInterest }
+
+//<Description>
+// An issue with the severity **Blocker** cannot move to production, it must be fixed.
+//
+// The severity of an issue is either defined explicitely in the rule source code,
+// either inferred from the issue *annual interest* and thresholds defined in the 
+// NDepend Project Properties > Issue and Debt.
+//</Description>
+
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Critical Issues" Unit="issues" />
+failif count > 10 issues
+warnif count > 0 issues
+
+from i in Issues
+where i.Severity == Severity.Critical
+select new { i, i.Severity, i.Debt, i.AnnualInterest }
+
+//<Description>
+// An issue with a severity level **Critical** shouldn't move to production. 
+// It still can for business imperative needs purposes, but at worst it must 
+// be fixed during the next iterations. 
+//
+// The severity of an issue is either defined explicitely in the rule source code,
+// either inferred from the issue *annual interest* and thresholds defined in the 
+// NDepend Project Properties > Issue and Debt.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="New Blocker / Critical / High Issues" Unit="issues" />
+failif count > 0 issues
+from i in Issues
+where i.Severity.EqualsAny(Severity.Blocker, Severity.Critical, Severity.High) &&  
+      // Count both the new issues and the issues that became at least Critical
+      (i.WasAdded() || i.OlderVersion().Severity < Severity.High)
+select new { i, i.Severity, i.Debt, i.AnnualInterest }
+
+
+//<Description>
+// An issue with the severity **Blocker** cannot move to production, it must be fixed.
+//
+// An issue with a severity level **Critical** shouldn't move to production. 
+// It still can for business imperative needs purposes, but at worth it must be fixed 
+// during the next iterations. 
+//
+// An issue with a severity level **High** should be fixed quickly, but can wait until 
+// the next scheduled interval.
+//
+// The severity of an issue is either defined explicitely in the rule source code,
+// either inferred from the issue *annual interest* and thresholds defined in the 
+// NDepend Project Properties > Issue and Debt.
+//</Description>
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Critical Rules Violated" Unit="rules" />
+failif count > 0 rules
+from r in Rules where r.IsCritical && r.IsViolated()
+select new { r, issues = r.Issues() }
+
+//<Description>
+// The concept of critical rule is useful to pinpoint certain rules that 
+// should not be violated.
+//
+// A rule can be made critical just by checking the *Critical button* in the
+// rule edition control and then saving the rule.
+//
+// This quality gate fails if any critical rule gets any violations.
+//
+// When no baseline is available, rules that rely on diff are not counted.
+// If you observe that this quality gate count slightly decreases with no apparent reason,
+// the reason is certainly that rules that rely on diff are not counted
+// because the baseline is not defined.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Percentage Debt" Unit="%" />
+failif value > 30%
+warnif value > 20%
+let timeToDev = codeBase.EffortToDevelop()
+let debt = Issues.Sum(i => i.Debt)
+select 100d * debt.ToManDay() / timeToDev.ToManDay()
+
+// <Description>
+// % Debt total is defined as a percentage on:
+//
+// • the estimated total effort to develop the code base
+//
+// • and the the estimated total time to fix all issues (the Debt)
+//
+// Estimated total effort to develop the code base is inferred from 
+// # lines of code of the code base and from the 
+// *Estimated number of man-day to develop 1000 logicial lines of code*
+// setting found in NDepend Project Properties > Issue and Debt.
+//
+// Debt documentation: http://www.ndepend.com/docs/technical-debt#Debt
+//
+// This quality gates fails if the estimated debt is more than 30%
+// of the estimated effort to develop the code base, and warns if the 
+// estimated debt is more than 20% of the estimated effort to develop 
+// the code base
+// </Description>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Debt" Unit="man-days" />
+failif value > 50 man-days
+warnif value > 30 man-days
+Issues.Sum(i => i.Debt).ToManDay()
+
+//<Description>
+// This Quality Gate is disabled per default because the fail and warn 
+// thresholds of unacceptable Debt in man-days can only depend on the 
+// project size, number of developers and overall context.
+//
+// However you can refer to the default Quality Gate **Percentage Debt**.
+//
+// The Debt is defined as the sum of estimated effort to fix all issues.
+// Debt documentation: http://www.ndepend.com/docs/technical-debt#Debt
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="New Debt since Baseline" Unit="man-days" />
+failif value > 2 man-days
+warnif value > 0 man-days
+let debt = Issues.Sum(i => i.Debt)
+let debtInBaseline = IssuesInBaseline.Sum(i => i.Debt)
+select (debt - debtInBaseline).ToManDay()
+
+
+//<Description>
+// This Quality Gate fails if the estimated effort to fix new or worsened
+// issues (what is called the *New Debt since Baseline*) is higher
+// than 2 man-days.
+//
+// This Quality Gate warns if this estimated effort is positive.
+//
+// Debt documentation: http://www.ndepend.com/docs/technical-debt#Debt
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Debt Rating per Namespace" Unit="namespaces" />
+failif count > 0 namespaces
+
+from n in Application.Namespaces
+where n.DebtRating() != null &&
+      n.DebtRating().Value.EqualsAny(DebtRating.E, DebtRating.D)
+select new { 
+   n, 
+   debtRating = n.DebtRating(),
+   debtRatio = n.DebtRatio(),  // % of debt from which DebtRating is inferred
+   devTimeInManDay = n.EffortToDevelop().ToDebt(), 
+   debtInManDay = n.AllDebt(),
+   issues = n.AllIssues() 
+}
+
+// <Description>
+// Forbid namespaces with a poor Debt Rating equals to **E** or **D**.
+//
+// The **Debt Rating** for a code element is estimated by the value of the **Debt Ratio**
+// and from the various rating thresholds defined in this project *Debt Settings*. 
+//
+// The **Debt Ratio** of a code element is a percentage of **Debt Amount** (in floating man-days) 
+// compared to the **estimated effort to develop the code element** (also in floating man-days).
+//
+// The **estimated effort to develop the code element** is inferred from the code elements
+// number of lines of code, and from the project *Debt Settings* parameters 
+// *estimated number of man-days to develop 1000* **logical lines of code**.
+//
+// The **logical lines of code** corresponds to the number of debug breakpoints in a method
+// and doesn't depend on code formatting nor comments.
+//
+// The Quality Gate can be modified to match assemblies, types or methods
+// with a poor Debt Rating, instead of matching namespaces.
+// </Description>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="Annual Interest" Unit="man-days" />
+failif value > 50 man-days
+warnif value > 30 man-days
+Issues.Sum(i => i.AnnualInterest).ToManDay()
+
+
+//<Description>
+// This Quality Gate is disabled per default because the fail and warn 
+// thresholds of unacceptable Annual-Interest in man-days can only depend
+// on the project size, number of developers and overall context.
+//
+// However you can refer to the default Quality Gate 
+// **New Annual Interest since Baseline**.
+//
+// The Annual-Interest is defined as the sum of estimated annual cost
+// in man-days, to leave all issues unfixed.
+//
+// Each rule can either provide a formula to compute the Annual-Interest 
+// per issue, or assign a **Severity** level for each issue. Some thresholds
+// defined in *Project Properties > Issue and Debt > Annual Interest* are
+// used to infer an Annual-Interest value from a Severity level.
+// Annual Interest documentation: http://www.ndepend.com/docs/technical-debt#AnnualInterest
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <QualityGate Name="New Annual Interest since Baseline" Unit="man-days" />
+failif value > 2 man-days
+warnif value > 0 man-days
+let ai = Issues.Sum(i => i.AnnualInterest)
+let aiInBaseline = IssuesInBaseline.Sum(i => i.AnnualInterest)
+select (ai - aiInBaseline).ToManDay()
+
+//<Description>
+// This Quality Gate fails if the estimated annual cost to leave all issues
+// unfixed, increased from more than 2 man-days since the baseline.
+//
+// This Quality Gate warns if this estimated annual cost is positive.
+//
+// This estimated annual cost is named the **Annual-Interest**.
+//
+// Each rule can either provide a formula to compute the Annual-Interest 
+// per issue, or assign a **Severity** level for each issue. Some thresholds
+// defined in *Project Properties > Issue and Debt > Annual Interest* are
+// used to infer an Annual-Interest value from a Severity level.
+// Annual Interest documentation: http://www.ndepend.com/docs/technical-debt#AnnualInterest
+//</Description>]]></Query>
+    </Group>
+    <Group Name="Hot Spots" Active="True" ShownInReport="True">
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Types Hot Spots</Name>
+from t in JustMyCode.Types
+where t.AllDebt() > Debt.Zero && 
+      t.AllAnnualInterest() > AnnualInterest.Zero
+orderby t.AllDebt().Value.TotalMinutes descending
+select new { t, 
+   Debt = t.AllDebt(),
+   Issues = t.AllIssues(), // AllIssues = {types issues} union {members issues}
+   AnnualInterest = t.AllAnnualInterest(),
+   BreakingPoint = t.AllBreakingPoint(),
+   t.NbLinesOfCode,
+   // t.PercentageCoverage,  to uncomment if coverage data is imported
+   DebtRating = t.DebtRating(), 
+   DebtRatio = t.DebtRatio() 
+}
+
+//<Description>
+// This query lists **types with most Debt**,
+// or in other words, types with issues that would need 
+// the largest effort to get fixed.
+//
+// Both issues on the type and its members are
+// taken account.
+//
+// Since untested code often generates a lot of 
+// Debt, the type size and percentage coverage is shown
+// (just uncomment *t.PercentageCoverage* in the query 
+// source code once you've imported the coverage data).
+//
+// The *Debt Rating* and *Debt Ratio* are also shown
+// for informational purpose.
+//
+// --
+//
+// The amount of *Debt* is not a measure to prioritize
+// the effort to fix issues, it is an estimation of how far 
+// the team is from clean code that abides by the rules set.
+//
+// For each issue the *Annual Interest* estimates the annual 
+// cost to leave the issues unfixed. The *Severity* of an issue
+// is estimated through thresholds from the *Annual Interest*.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issue unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issue. The **Breaking Point is the right metric
+// to prioritize issues fix**.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Types to Fix Priority</Name>
+from t in JustMyCode.Types
+where t.AllBreakingPoint() > TimeSpan.Zero &&
+      t.AllDebt().Value > 30.ToMinutes()
+orderby t.AllBreakingPoint().TotalMinutes ascending
+select new { t, 
+   BreakingPoint = t.AllBreakingPoint(),
+   Debt = t.AllDebt(),
+   AnnualInterest = t.AllAnnualInterest(),
+   Issues = t.AllIssues(),
+   t.NbLinesOfCode,
+   // t.PercentageCoverage,  to uncomment if coverage data is imported
+   DebtRating = t.DebtRating(), 
+   DebtRatio = t.DebtRatio() 
+}
+
+//<Description>
+// This query lists types per increasing
+// **Debt Breaking Point**.
+//
+// For each issue the *Debt* estimates the
+// effort to fix the issue, and the *Annual Interest*
+// estimates the annual cost to leave the issue unfixed.
+// The *Severity* of an issue is estimated through 
+// thresholds from the *Annual Interest* of the issue.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issue unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issues.
+//
+// Often new and refactored types since baseline will be 
+// listed first, because issues on these types get a 
+// higher *Annual Interest* because it is important to
+// focus first on new issues.
+// 
+//
+// --
+//
+// Both issues on the type and its members are
+// taken account.
+//
+// Only types with at least 30 minutes of Debt are listed
+// to avoid parasiting the list with the numerous
+// types with small *Debt*, on which the *Breaking Point*
+// value makes less sense. 
+//
+// The *Annual Interest* estimates the cost per year
+// in man-days to leave these issues unfixed.
+//
+// Since untested code often generates a lot of 
+// Debt, the type size and percentage coverage is shown
+// (just uncomment *t.PercentageCoverage* in the query 
+// source code once you've imported the coverage data).
+//
+// The *Debt Rating* and *Debt Ratio* are also shown
+// for informational purpose.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Issues to Fix Priority</Name>
+from i in Issues
+// Don't show first issues with BreakingPoint equals to zero.
+orderby i.BreakingPoint != TimeSpan.Zero ? i.BreakingPoint : TimeSpan.MaxValue
+select new { i, 
+   Debt = i.Debt,
+   AnnualInterest = i.AnnualInterest,
+   BreakingPoint = i.BreakingPoint,
+   CodeElement = i.CodeElement
+}
+
+//<Description>
+// This query lists issues per increasing
+// **Debt Breaking Point**.
+//
+// Double-click an issue to edit its rule and
+// select the issue in the rule result. This way
+// you can view all information concerning the issue.
+//
+// For each issue the *Debt* estimates the
+// effort to fix the issue, and the *Annual Interest*
+// estimates the annual cost to leave the issue unfixed.
+// The *Severity* of an issue is estimated through 
+// thresholds from the *Annual Interest* of the issue.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issue unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issue.
+//
+// Often issues on new and refactored code elements since 
+// baseline will be listed first, because such issues get a 
+// higher *Annual Interest* because it is important to
+// focus first on new issues on recent code.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Debt and Issues per Rule</Name>
+from r in Rules
+where r.IsViolated()
+orderby r.Debt().Value descending
+select new { 
+   r, 
+   Issues = r.Issues(), 
+   Debt = r.Debt(), 
+   AnnualInterest = r.AnnualInterest(), 
+   BreakingPoint = r.BreakingPoint(),
+   Category = r.Category
+}
+
+//<Description>
+// This query lists violated rules with most *Debt* first.
+//
+// A rule violated has issues. For each issue the *Debt* 
+// estimates the effort to fix the issue.
+//
+// --
+//
+// The amount of *Debt* is not a measure to prioritize
+// the effort to fix issues, it is an estimation of how far 
+// the team is from clean code that abides by the rules set.
+//
+// For each issue the *Annual Interest* estimates the annual 
+// cost to leave the issues unfixed. The *Severity* of an issue
+// is estimated through thresholds from the *Annual Interest*.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issue unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issue. The **Breaking Point is the right metric
+// to prioritize issues fix**.
+//
+// --
+//
+// Notice that rules can be grouped in *Rule Category*. This
+// way you'll see categories that generate most *Debt*.
+//
+// Typically the rules that generate most *Debt* are the 
+// ones related to *Code Coverage by Tests*, *Architecture*
+// and *Code Smells*.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>New Debt and Issues per Rule</Name>
+from r in Rules
+where r.IsViolated() && r.IssuesAdded().Count() > 0
+orderby r.DebtDiff().Value descending
+select new { 
+   r, 
+   IssuesAdded = r.IssuesAdded(),
+   IssuesFixed = r.IssuesFixed(),
+   Issues = r.Issues(), 
+   Debt = r.Debt(), 
+   DebtDiff = r.DebtDiff(),
+   Category = r.Category
+}
+
+//<Description>
+// This query lists violated rules that have new issues
+// since baseline, with most **new Debt** first.
+//
+// A rule violated has issues. For each issue the *Debt* 
+// estimates the effort to fix the issue.
+//
+// --
+//
+// New issues since the baseline are consequence of recent code 
+// refactoring sessions. They represent good opportunities
+// of fix because the code recently refactored is fresh in 
+// the developers mind, which means fixing now costs less
+// than fixing later.
+//
+// Fixing issues on recently touched code is also a good way 
+// to foster practices that will lead to higher code quality 
+// and maintainability, including writing unit-tests
+// and avoiding unnecessary complex code.
+//
+// --
+//
+// Notice that rules can be grouped in *Rule Category*. This
+// way you'll see categories that generate most *Debt*.
+//
+// Typically the rules that generate most *Debt* are the 
+// ones related to *Code Coverage by Tests*, *Architecture*
+// and *Code Smells*.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Debt and Issues per Code Element</Name>
+from elem in CodeElements
+where elem.HasIssue()
+orderby elem.Debt().Value descending
+select new { 
+   elem, 
+   Issues = elem.Issues(), 
+   Debt = elem.Debt(), 
+   AnnualInterest = elem.AnnualInterest(), 
+   BreakingPoint = elem.BreakingPoint()
+}
+
+//<Description>
+// This query lists code elements that have issues, 
+// with most *Debt* first.
+//
+// For each code element the *Debt* estimates 
+// the effort to fix the element issues.
+//
+// The amount of *Debt* is not a measure to prioritize
+// the effort to fix issues, it is an estimation of how far 
+// the team is from clean code that abides by the rules set.
+//
+// For each element the *Annual Interest* estimates the annual 
+// cost to leave the elements issues unfixed. The *Severity* of an 
+// issue is estimated through thresholds from the *Annual Interest*
+// of the issue.
+//
+// The **Debt Breaking Point** represents the duration
+// from now when the estimated cost to leave the issues unfixed
+// costs as much as the estimated effort to fix it.
+//
+// Hence the shorter the **Debt Breaking Point** 
+// the largest the **Return on Investment** for fixing 
+// the issue. The **Breaking Point is the right metric
+// to prioritize issues fix**.
+//</Description>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>New Debt and Issues per Code Element</Name>
+from elem in CodeElements
+where elem.HasIssue() && elem.IssuesAdded().Count() > 0
+orderby elem.DebtDiff().Value descending
+select new { 
+   elem, 
+   IssuesAdded = elem.IssuesAdded(),
+   IssuesFixed = elem.IssuesFixed(),
+   Issues = elem.Issues(), 
+   Debt = elem.Debt(), 
+   DebtDiff = elem.DebtDiff()
+}
+    //<Description>
+// This query lists code elements that have new issues
+// since baseline, with most **new Debt** first.
+//
+// For each code element the *Debt* estimates 
+// the effort to fix the element issues.
+//
+// New issues since the baseline are consequence of recent code 
+// refactoring sessions. They represent good opportunities
+// of fix because the code recently refactored is fresh in 
+// the developers mind, which means fixing now costs less
+// than fixing later.
+//
+// Fixing issues on recently touched code is also a good way 
+// to foster practices that will lead to higher code quality 
+// and maintainability, including writing unit-tests
+// and avoiding unnecessary complex code.
+//</Description>
+]]></Query>
+    </Group>
+    <Group Name="Code Smells" Active="True" ShownInReport="False">
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="True"><![CDATA[// <Name>Avoid types too big</Name>
+warnif count > 0 from t in JustMyCode.Types where 
+
+   // First filter on type to optimize 
+   t.NbLinesOfCode > 200 
+   // # IL Instructions is commented, because with LINQ syntax, a few lines of code can compile to hundreds of IL instructions.
+   // || t.NbILInstructions > 3000
+
+   // What matters is the # lines of code in JustMyCode
+   let locJustMyCode = t.MethodsAndContructors.Where(m => JustMyCode.Contains(m)).Sum(m => m.NbLinesOfCode)
+   where locJustMyCode > 200
+
+   let isStaticWithNoMutableState = (t.IsStatic && t.Fields.Any(f => !f.IsImmutable))
+   let staticFactor = (isStaticWithNoMutableState ? 0.2 : 1)
+
+   orderby locJustMyCode descending
+select new { 
+   t, 
+   locJustMyCode, 
+   t.NbILInstructions,
+   t.Methods, 
+   t.Fields,
+   
+   Debt = (staticFactor*locJustMyCode.Linear(200, 1, 2000, 10)).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity major for 300 loc
+   // to interest for severity critical for 2000 loc
+   AnnualInterest = staticFactor*(locJustMyCode.Linear(
+                          200,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                          2000, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule matches types with more than 200 lines of code.
+// **Only lines of code in JustMyCode methods are taken account.**
+//
+// Types where *NbLinesOfCode > 200* are extremely complex 
+// to develop and maintain.
+// See the definition of the NbLinesOfCode metric here 
+// http://www.ndepend.com/docs/code-metrics#NbLinesOfCode
+//
+// Maybe you are facing the **God Class** phenomenon:
+// A **God Class** is a class that controls way too many other classes 
+// in the system and has grown beyond all logic to become 
+// *The Class That Does Everything*.
+//</Description>
+
+//<HowToFix>
+// Types with many lines of code
+// should be split in a group of smaller types.
+// 
+// To refactor a *God Class* you'll need patience, 
+// and you might even need to recreate everything from scratch.
+// Here are a few refactoring advices:
+//
+// • The logic in the *God Class* must be splitted in smaller classes.
+// These smaller classes can eventually become private classes nested
+// in the original *God Class*, whose instances objects become 
+// composed of instances of smaller nested classes.
+//
+// • Smaller classes partitioning should be driven by the multiple
+// responsibilities handled by the *God Class*. To identify these 
+// responsibilities it often helps to look for subsets of methods
+// strongly coupled with subsets of fields.
+//
+// • If the *God Class* contains way more logic than states, a good 
+// option can be to define one or several static classes that 
+// contains no static field but only pure static methods. A pure static 
+// method is a function that computes a result only from inputs
+// parameters, it doesn't read nor assign any static or instance field.
+// The main advantage of pure static methods is that they are easily
+// testable.
+// 
+// • Try to maintain the interface of the *God Class* at first 
+// and delegate calls to the new extracted classes. 
+// In the end the *God Class* should be a pure facade without its own logic.
+// Then you can keep it for convenience or throw it away and 
+// start to use the new classes only.
+//
+// • Unit Tests can help: write tests for each method before extracting it 
+// to ensure you don't break functionality.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 1 hour for a 200 lines of code type,
+// up to 10 hours for a type with 2.000 or more lines of code.
+//
+// In Debt and Interest computation, this rule takes account of the fact 
+// that static types with no mutable fields are just a collection of 
+// static methods that can be easily splitted and moved from one type 
+// to another.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid types with too many methods</Name>
+warnif count > 0 from t in JustMyCode.Types 
+
+  // Optimization: Fast discard of non-relevant types 
+  where t.Methods.Count() > 20
+
+  // Don't match these methods
+  let methods = t.Methods.Where(
+       m => !(m.IsGeneratedByCompiler ||
+              m.IsConstructor || m.IsClassConstructor ||
+              m.IsPropertyGetter || m.IsPropertySetter ||
+              m.IsEventAdder || m.IsEventRemover))
+
+  where methods.Count() > 20 
+  orderby methods.Count() descending
+
+  let isStaticWithNoMutableState = (t.IsStatic && t.Fields.Any(f => !f.IsImmutable))
+  let staticFactor = (isStaticWithNoMutableState ? 0.2 : 1)
+
+select new { 
+   t, 
+   nbMethods = methods.Count(),
+   instanceMethods = methods.Where(m => !m.IsStatic), 
+   staticMethods = methods.Where(m => m.IsStatic),
+
+   t.NbLinesOfCode,
+
+   Debt = (staticFactor*methods.Count().Linear(20, 1, 200, 10)).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity major for 30 methods
+   // to interest for severity critical for 200 methods
+   AnnualInterest = (staticFactor*methods.Count().Linear(
+                              20,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                              200, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule matches types with more than 20 methods. 
+// Such type might be hard to understand and maintain.
+//
+// Notice that methods like constructors or property 
+// and event accessors are not taken account.
+//
+// Having many methods for a type might be a symptom
+// of too many responsibilities implemented.
+//
+// Maybe you are facing the **God Class** phenomenon:
+// A **God Class** is a class that controls way too many other classes 
+// in the system and has grown beyond all logic to become 
+// *The Class That Does Everything*.
+//</Description>
+
+//<HowToFix>
+// To refactor properly a *God Class* please read *HowToFix advices* 
+// from the default rule **Types to Big**.
+////
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 1 hour for a type with 20 methods,
+// up to 10 hours for a type with 200 or more methods.
+//
+// In Debt and Interest computation, this rule takes account of the fact 
+// that static types with no mutable fields are just a collection of 
+// static methods that can be easily splitted and moved from one type 
+// to another.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid types with too many fields</Name>
+warnif count > 0 from t in JustMyCode.Types 
+
+  // Optimization: Fast discard of non-relevant types 
+  where !t.IsEnumeration &&
+         t.Fields.Count() > 15
+        
+  // Count instance fields and non-constant static fields
+  let fields = t.Fields.Where(f =>
+          !f.IsGeneratedByCompiler &&
+          !f.IsLiteral &&
+          !(f.IsStatic && f.IsInitOnly) &&
+           JustMyCode.Contains(f) )
+
+  where fields.Count() > 15
+ 
+  let methodsAssigningFields = fields.SelectMany(f => f.MethodsAssigningMe)
+
+  orderby fields.Count() descending
+select new { 
+   t, 
+   instanceFields = fields.Where(f => !f.IsStatic),
+   staticFields = fields.Where(f => f.IsStatic),
+methodsAssigningFields ,   
+
+   // See definition of Size of Instances metric here:
+   // http://www.ndepend.com/docs/code-metrics#SizeOfInst
+   t.SizeOfInst,
+
+   Debt = fields.Count().Linear(15, 1, 200, 10).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity major for 30 methods
+   // to interest for severity critical for 200 methods
+   AnnualInterest = fields.Count().Linear(15,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                          200, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule matches types with more than 15 fields. 
+// Such type might be hard to understand and maintain.
+//
+// Notice that constant fields and static-readonly fields are not counted.
+// Enumerations types are not counted also.
+//
+// Having many fields for a type might be a symptom
+// of too many responsibilities implemented.
+//</Description>
+
+//<HowToFix>
+// To refactor such type and increase code quality and maintainability,
+// certainly you'll have to group subsets of fields into smaller types
+// and dispatch the logic implemented into the methods 
+// into these smaller types.
+//
+// More refactoring advices can be found in the default rule 
+// **Types to Big**, *HowToFix* section.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 1 hour for a type with 15 fields,
+// to up to 10 hours for a type with 200 or more fields. 
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="True"><![CDATA[// <Name>Avoid methods too big, too complex</Name>
+warnif count > 0 from m in JustMyCode.Methods where 
+   m.ILNestingDepth > 2 &&
+  (m.NbLinesOfCode > 35 ||
+   m.CyclomaticComplexity > 20 ||
+   m.ILCyclomaticComplexity > 60)
+
+  let complexityScore = m.NbLinesOfCode/2 + m.CyclomaticComplexity + m.ILCyclomaticComplexity/3 + 3*m.ILNestingDepth
+
+  orderby complexityScore descending,
+          m.CyclomaticComplexity descending,
+          m.ILCyclomaticComplexity descending,
+          m.ILNestingDepth descending
+select new { 
+   m, 
+   m.NbLinesOfCode,
+   m.CyclomaticComplexity, 
+   m.ILCyclomaticComplexity,
+   m.ILNestingDepth,
+   complexityScore,
+
+   Debt = complexityScore.Linear(30, 40,    400, 8*60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity minor 
+   // to interest for severity major
+   AnnualInterest = complexityScore .Linear(30,     Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                            200, 2*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+   
+}
+
+//<Description>
+// This rule matches methods where *ILNestingDepth* > 2
+// and (*NbLinesOfCode* > 35 
+// or *CyclomaticComplexity* > 20
+// or *ILCyclomaticComplexity* > 60)
+// Such method is typically hard to understand and maintain.
+//
+// Maybe you are facing the **God Method** phenomenon.
+// A "God Method" is a method that does way too many processes in the system 
+// and has grown beyond all logic to become *The Method That Does Everything*.
+// When need for new processes increases suddenly some programmers realize: 
+// why should I create a new method for each processe if I can only add an *if*.
+//
+// See the definition of the *CyclomaticComplexity* metric here:
+// http://www.ndepend.com/docs/code-metrics#CC
+//
+// See the definition of the *ILCyclomaticComplexity* metric here:
+// http://www.ndepend.com/docs/code-metrics#ILCC
+//
+// See the definition of the *ILNestingDepth* metric here:
+// http://www.ndepend.com/docs/code-metrics#ILNestingDepth
+//</Description>
+
+//<HowToFix>
+// A large and complex method should be split in smaller methods, 
+// or even one or several classes can be created for that.
+//
+// During this process it is important to question the scope of each
+// variable local to the method. This can be an indication if
+// such local variable will become an instance field of the newly created class(es).
+//
+// Large *switch…case* structures might be refactored through the help
+// of a set of types that implement a common interface, the interface polymorphism
+// playing the role of the *switch cases tests*.
+//
+// Unit Tests can help: write tests for each method before extracting it 
+// to ensure you don't break functionality.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies from 40 minutes to 8 hours, linearly from a weighted complexity score.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="True"><![CDATA[// <Name>Avoid methods with too many parameters</Name>
+warnif count > 0 from m in JustMyCode.Methods where 
+  m.NbParameters >= 7
+  orderby m.NbParameters descending
+select new { 
+   m, 
+   m.NbParameters,
+
+   Debt = m.NbParameters.Linear(7, 1,  40, 6).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for 7 parameters
+   // to interest for severity Critical for 40 parameters
+   AnnualInterest = m.NbParameters.Linear(7,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                          40, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule matches methods with more than 8 parameters.
+// Such method is painful to call and might degrade performance.
+// See the definition of the *NbParameters* metric here: 
+// http://www.ndepend.com/docs/code-metrics#NbParameters
+//</Description>
+
+//<HowToFix>
+// More properties/fields can be added to the declaring type to 
+// handle numerous states. An alternative is to provide 
+// a class or a structure dedicated to handle arguments passing.
+// For example see the class *System.Diagnostics.ProcessStartInfo* 
+// and the method *System.Diagnostics.Process.Start(ProcessStartInfo)*.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 1 hour for a method with 7 parameters,
+// up to 6 hours for a methods with 40 or more parameters.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid methods with too many local variables</Name>
+warnif count > 0 from m in JustMyCode.Methods where 
+  m.NbVariables > 15 
+  orderby m.NbVariables descending
+select new { 
+   m, 
+   m.NbVariables,
+
+   Debt = m.NbVariables.Linear(15, 1,   80, 6).ToHours().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for 15 variables
+   // to interest for severity Critical for 80 variables
+   AnnualInterest = m.NbVariables.Linear(15,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                         80,  Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()
+
+}
+
+//<Description>
+// This rule matches methods with more than 15 variables.
+//
+// Methods where *NbVariables > 8* are hard to understand and maintain.
+// Methods where *NbVariables > 15* are extremely complex and must be refactored. 
+//
+// See the definition of the *Nbvariables* metric here: 
+// http://www.ndepend.com/docs/code-metrics#Nbvariables
+//</Description>
+
+//<HowToFix>
+// To refactor such method and increase code quality and maintainability,
+// certainly you'll have to split the method into several smaller methods
+// or even create one or several classes to implement the logic.
+//
+// During this process it is important to question the scope of each
+// variable local to the method. This can be an indication if
+// such local variable will become an instance field of the newly created class(es).
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 10 minutes for a method with 15 variables,
+// up to 2 hours for a methods with 80 or more variables.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid methods with too many overloads</Name>
+warnif count > 0 
+
+let lookup = JustMyCode.Methods.Where(m =>
+   m.NbOverloads >= 6 && 
+  !m.IsOperator && // Don't report operator overload
+
+   // Don't match overloads due tu the visitor pattern, based on a naming convention.
+  !m.SimpleName.ToLower().StartsWithAny("visit", "dispatch")
+).ToLookup(m => m.ParentType.FullName + "."+ m.SimpleName)
+
+from @group in lookup
+let overloads = @group.ToArray()
+orderby overloads.Length descending
+
+select new { 
+   m = @group.First(), 
+   overloads,
+   Debt = (3*overloads.Length).ToMinutes().ToDebt(),
+   Severity = Severity.Medium
+}
+
+//<Description>
+// Method overloading is the ability to create multiple methods of the same name 
+// with different implementations, and various set of parameters.
+//
+// This rule matches sets of methods with 6 overloads or more.
+//
+// Such method set might be a problem to maintain 
+// and provokes coupling higher than necessary.
+//
+// See the definition of the *NbOverloads* metric here 
+// http://www.ndepend.com/docs/code-metrics#NbOverloads
+//</Description>
+
+//<HowToFix>
+// Typically the *too many overloads* phenomenon appears when an algorithm
+// takes a various set of in-parameters. Each overload is presented as 
+// a facility to provide a various set of in-parameters.
+// In such situation, the C# and VB.NET language feature named 
+// *Named and Optional arguments* should be used.
+//
+// The *too many overloads* phenomenon can also be a consequence of the usage
+// of the **visitor design pattern** http://en.wikipedia.org/wiki/Visitor_pattern 
+// since a method named *Visit()* must be provided for each sub type.
+// For this reason, the default version of this rule doesn't match overloads whose name
+// start with "visit" or "dispatch" (case-unsensitive) to avoid match 
+// overload visitors, and you can adapt this rule to your own naming convention.
+//
+// Sometime *too many overloads* phenomenon is not the symptom of a problem,
+// for example when a *numeric to something conversion* method applies to 
+// all numeric and nullable numeric types.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is of 3 minutes per method overload.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid methods potentially poorly commented</Name>
+warnif count > 0 from m in JustMyCode.Methods where 
+  m.PercentageComment < 10 && 
+  m.NbLinesOfCode > 20  
+
+  let nbLinesOfCodeNotCommented = m.NbLinesOfCode - m.NbLinesOfComment
+
+  orderby nbLinesOfCodeNotCommented descending
+
+select new { 
+   m, 
+   m.PercentageComment, 
+   m.NbLinesOfCode, 
+   m.NbLinesOfComment,
+   nbLinesOfCodeNotCommented,
+
+   Debt = nbLinesOfCodeNotCommented .Linear(20, 2,  200, 20).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity major for 300 loc
+   // to interest for severity critical for 2000 loc
+   AnnualInterest = m.PercentageComment.Linear(
+                         0,  8 *(Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes), 
+                         20,     Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()  
+}
+
+//<Description>
+// This rule matches methods with less than 10% of comment lines and that have 
+// at least 20 lines of code. Such method might need to be more commented.
+//
+// See the definitions of the *Comments metric* here:
+// http://www.ndepend.com/docs/code-metrics#PercentageComment
+// http://www.ndepend.com/docs/code-metrics#NbLinesOfComment
+// 
+// Notice that only comments about the method implementation
+// (comments in method body) are taken account.
+//</Description>
+
+//<HowToFix>
+// Typically add more comment. But code commenting is subject to controversy.
+// While poorly written and designed code would needs a lot of comment 
+// to be understood, clean code doesn't need that much comment, especially
+// if variables and methods are properly named and convey enough information.
+// Unit-Test code can also play the role of code commenting.
+//
+// However, even when writing clean and well-tested code, one will have
+// to write **hacks** at a point, usually to circumvent some API limitations or bugs.
+// A hack is a non-trivial piece of code, that doesn't make sense at first glance,
+// and that took time and web research to be found.
+// In such situation comments must absolutely be used to express the intention, 
+// the need for the hacks and the source where the solution has been found.
+//
+// The estimated Debt, which means the effort to comment such method,
+// varies linearly from 2 minutes for 10 lines of code not commented,
+// up to 20 minutes for 200 or more, lines of code not commented.
+//</HowToFix>]]></Query>
+      <Query Active="False" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid types with poor cohesion</Name>
+warnif count > 0 from t in JustMyCode.Types where 
+  t.LCOM > 0.8  && 
+  t.NbFields > 10 && 
+  t.NbMethods >10 
+
+  let poorCohesionScore = 1/(1.01 - t.LCOM)
+  orderby poorCohesionScore descending
+
+  select new { 
+   t, 
+   t.LCOM, 
+   t.NbMethods, 
+   t.NbFields,
+   poorCohesionScore,
+
+   Debt = poorCohesionScore.Linear(5, 5, 50, 4*60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for low poorCohesionScore
+   // to 4 times interest for severity High for high poorCohesionScore
+   AnnualInterest = poorCohesionScore.Linear(5,     Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+                                             50, 4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+   
+}
+
+//<Description>
+// This rule is based on the *LCOM code metric*,
+// LCOM stands for **Lack Of Cohesion of Methods**.
+// See the definition of the LCOM metric here 
+// http://www.ndepend.com/docs/code-metrics#LCOM
+//
+// The LCOM metric measures the fact that most methods are using most fields.
+// A class is considered utterly cohesive (which is good)
+// if all its methods use all its instance fields.
+//
+// Only types with enough methods and fields are taken account to avoid bias.
+// The LCOM takes its values in the range [0-1].
+//
+// This rule matches types with LCOM higher than 0.8.
+// Such value generally pinpoints a **poorly cohesive class**.
+//</Description>
+
+//<HowToFix>
+// To refactor a poorly cohesive type and increase code quality and maintainability,
+// certainly you'll have to split the type into several smaller and more cohesive types
+// that together, implement the same logic.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 5 minutes for a type with a low poorCohesionScore,
+// up to 4 hours for a type with high poorCohesionScore.
+//</HowToFix>]]></Query>
+    </Group>
+    <Group Name="Code Smells Regression" Active="True" ShownInReport="False">
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>From now, all types added should respect basic quality principles</Name>
+warnif count > 0 from t in JustMyCode.Types where
+
+// Only match types added since Baseline.
+// Uncomment this line to match also refactored types since Baseline.
+// (t.WasAdded() || t.CodeWasChanged()) &&
+   t.WasAdded() &&
+
+// Eliminate interfaces, enumerations or types only with constant fields
+// by making sure we are matching type with code.
+t.NbLinesOfCode > 10 &&
+
+// Optimization: Fast discard of non-relevant types 
+(t.Fields.Count() > 20 || t.Methods.Count() > 20)
+      
+// Count instance fields and non-constant static fields
+let fields = t.Fields.Where(f => 
+      !f.IsLiteral &&
+      !(f.IsStatic && f.IsInitOnly))
+
+// Don't match these methods
+let methods = t.Methods.Where(
+   m => !(m.IsConstructor || m.IsClassConstructor ||
+          m.IsGeneratedByCompiler ||
+          m.IsPropertyGetter || m.IsPropertySetter ||
+          m.IsEventAdder || m.IsEventRemover))
+  
+where 
+
+// Low Quality types     Metrics' definitions are available here:
+//     http://www.ndepend.com/docs/code-metrics#MetricsOnTypes
+(  // Types with too many methods
+   fields.Count() > 20 ||
+
+   methods.Count() > 20 ||
+               
+   // Complex Types that use more than 50 other types
+   t.NbTypesUsed > 50
+)
+select new { 
+   t, 
+   t.NbLinesOfCode, 
+
+   instanceMethods = methods.Where(m => !m.IsStatic), 
+   staticMethods = methods.Where(m => m.IsStatic),
+  
+   instanceFields = fields.Where(f => !f.IsStatic),
+   staticFields = fields.Where(f => f.IsStatic),
+  
+   t.TypesUsed,
+
+   // Constant Debt estimation, since for such type rules in category "Code Smells" 
+   // accurately estimate the Debt.
+   Debt     = 10.ToMinutes().ToDebt(),
+
+   // The Severity is higher for new types than for refactored types 
+   AnnualInterest= (t.WasAdded() ? 3 : 1) *
+                    Severity.High.AnnualInterestThreshold()
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+// This rule operates only on types added since baseline.
+//
+// This rule can be easily modified to also match types refactored since baseline,
+// that don't satisfy all quality criterions.
+//
+// Types matched by this rule not only have been recently added or refactored,
+// but also somehow violate one or several basic quality principles,
+// whether it has too many methods,
+// it has too many fields,
+// or is using too many types.
+// Any of these criterions is often a symptom of a type with too many responsibilities.
+//
+// Notice that to count methods and fields, methods like constructors 
+// or property and event accessors are not taken account.
+// Notice that constants fields and static-readonly fields are not counted.
+// Enumerations types are not counted also.
+//</Description>
+
+//<HowToFix>
+// To refactor such type and increase code quality and maintainability,
+// certainly you'll have to split the type into several smaller types
+// that together, implement the same logic.
+//
+// Issues of this rule have a constant 10 minutes Debt, because the Debt,
+// which means the effort to fix such issue, is already estimated for issues
+// of rules in the category **Code Smells**.
+//
+// However issues of this rule have a **High** severity, with even more 
+// interests for issues on new types since baseline, because the proper time 
+// to increase the quality of these types is **now**, before they get commited 
+// in the next production release.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>From now, all types added should be 100% covered by tests</Name>
+warnif count > 0 from t in JustMyCode.Types where
+
+// Only match types added since Baseline.
+// Uncomment this line to match also refactored types since Baseline.
+// (t.WasAdded() || t.CodeWasChanged()) &&
+   t.WasAdded() &&
+
+  // …that are not 100% covered by tests
+  t.PercentageCoverage < 100
+
+  let methodsCulprit = t.Methods.Where(m => m.PercentageCoverage < 100)
+
+select new { 
+   t, 
+   t.PercentageCoverage, 
+   methodsCulprit,
+   t.NbLinesOfCode,
+
+   // Constant Debt estimation, since for such type rules in category "Coverage" 
+   // accurately estimate the untested code Debt.
+   Debt     = 10.ToMinutes().ToDebt(),
+
+   // The Severity is higher for new types than for refactored types 
+   AnnualInterest= (t.WasAdded() ? 3 : 1) *
+                    Severity.High.AnnualInterestThreshold()
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+// This rule operates only on types added since baseline.
+//
+// This rule can be easily modified to also match types refactored since baseline,
+// that are not 100% covered by tests.
+//
+// This rule is executed only if some code coverage data is imported
+// from some code coverage files.
+//
+// Often covering 10% of remaining uncovered code of a class, 
+// requires as much work as covering the first 90%.
+// For this reason, typically teams estimate that 90% coverage is enough.
+// However *untestable code* usually means *poorly written code* 
+// which usually leads to *error prone code*.
+// So it might be worth refactoring and making sure to cover the 10% remaining code
+// because **most tricky bugs might come from this small portion of hard-to-test code**.
+//
+// Not all classes should be 100% covered by tests (like UI code can be hard to test)
+// but you should make sure that most of the logic of your application
+// is defined in some *easy-to-test classes*, 100% covered by tests.
+//
+// In this context, this rule warns when a type added or refactored since the baseline,
+// is not fully covered by tests.
+//</Description>
+
+//<HowToFix>
+// Write more unit-tests dedicated to cover code not covered yet.
+// If you find some *hard-to-test code*, it is certainly a sign that this code
+// is not *well designed* and hence, needs refactoring.
+//
+// You'll find code impossible to cover by unit-tests, like calls to *MessageBox.Show()*.
+// An infrastructure must be defined to be able to *mock* such code at test-time.
+//
+// Issues of this rule have a constant 10 minutes Debt, because the Debt,
+// which means the effort to write tests for the culprit type, is already 
+// estimated for issues in the category **Code Coverage**. 
+//
+// However issues of this rule have a **High** severity, with even more 
+// interests for issues on new types since baseline, because the proper time 
+// to write tests for these types is **now**, before they get commited 
+// in the next production release.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>From now, all methods added should respect basic quality principles</Name>
+warnif count > 0 from m in JustMyCode.Methods where
+
+// Only match methods added since Baseline.
+// Uncomment this line to match also refactored methods since Baseline.
+// (m.WasAdded() || m.CodeWasChanged()) &&
+   m.WasAdded() &&
+ 
+// Low Quality methods// Metrics' definitions
+(  m.NbLinesOfCode > 30 ||          // http://www.ndepend.com/docs/code-metrics#NbLinesOfCode
+   m.NbILInstructions > 200 ||      // http://www.ndepend.com/docs/code-metrics#NbILInstructions
+   m.CyclomaticComplexity > 20 ||   // http://www.ndepend.com/docs/code-metrics#CC
+   m.ILCyclomaticComplexity > 50 || // http://www.ndepend.com/docs/code-metrics#ILCC
+   m.ILNestingDepth > 4 ||          // http://www.ndepend.com/docs/code-metrics#ILNestingDepth
+   m.NbParameters > 5 ||            // http://www.ndepend.com/docs/code-metrics#NbParameters
+   m.NbVariables > 8 ||             // http://www.ndepend.com/docs/code-metrics#NbVariables
+   m.NbOverloads > 6 )
+select new { 
+   m, 
+   m.NbLinesOfCode,
+   m.NbILInstructions,
+   m.CyclomaticComplexity, 
+   m.ILCyclomaticComplexity,
+   m.ILNestingDepth, 
+   m.NbParameters, 
+   m.NbVariables, 
+   m.NbOverloads, // http://www.ndepend.com/docs/code-metrics#NbOverloads
+
+   // Constant Debt estimation, since for such method rules in category "Code Smells" 
+   // accurately estimate the Debt.
+   Debt     = 5.ToMinutes().ToDebt(),
+
+   // The Severity is higher for new methods than for refactored methods
+   AnnualInterest= (m.WasAdded() ? 3 : 1) *
+                   Severity.High.AnnualInterestThreshold()
+}  
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+// This rule operates only on methods added or refactored since the baseline.
+//
+// This rule can be easily modified to also match methods refactored since baseline,
+// that don't satisfy all quality criterions.
+//
+// Methods matched by this rule not only have been recently added or refactored,
+// but also somehow violate one or several basic quality principles,
+// whether it is too large (too many *lines of code*), 
+// too complex (too many *if*, *switch case*, loops…)
+// has too many variables, too many parameters
+// or has too many overloads.
+//</Description>
+
+//<HowToFix>
+// To refactor such method and increase code quality and maintainability,
+// certainly you'll have to split the method into several smaller methods
+// or even create one or several classes to implement the logic.
+//
+// During this process it is important to question the scope of each
+// variable local to the method. This can be an indication if
+// such local variable will become an instance field of the newly created class(es).
+//
+// Large *switch…case* structures might be refactored through the help
+// of a set of types that implement a common interface, the interface polymorphism
+// playing the role of the *switch cases tests*.
+//
+// Unit Tests can help: write tests for each method before extracting it 
+// to ensure you don't break functionality.
+//
+// Issues of this rule have a constant 5 minutes Debt, because the Debt,
+// which means the effort to fix such issue, is already estimated for issues
+// of rules in the category **Code Smells**.
+// 
+// However issues of this rule have a **High** severity, with even more 
+// interests for issues on new methods since baseline, because the proper time 
+// to increase the quality of these methods is **now**, before they get commited 
+// in the next production release.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid decreasing code coverage by tests of types</Name>
+warnif count > 0 
+from t in JustMyCode.Types where
+  t.IsPresentInBothBuilds() && t.CoverageDataAvailable && t.OlderVersion().CoverageDataAvailable
+let locDiff = (int)t.NbLinesOfCode.Value - (int)t.OlderVersion().NbLinesOfCode.Value
+where locDiff >= 0
+let uncoveredLoc  = (int)t.NbLinesOfCodeNotCovered.Value - ((int)t.OlderVersion().NbLinesOfCodeNotCovered.Value + locDiff)
+where uncoveredLoc > 0
+
+orderby uncoveredLoc descending
+
+select new { 
+   t,
+   OldCoveragePercent = t.OlderVersion().PercentageCoverage,
+   NewCoveragePercent = t.PercentageCoverage,
+   OldLoc = t.OlderVersion().NbLinesOfCode,
+   NewLoc = t.NbLinesOfCode,
+   uncoveredLoc,
+
+   Debt = uncoveredLoc.Linear(1, 15,  100, 3*60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity High for one line of code that is not covered by tests anymore
+   // to interest for severity Critical for 50 lines of code that are not covered by tests anymore
+   AnnualInterest = uncoveredLoc.Linear(1,    Severity.High.AnnualInterestThreshold().Value.TotalMinutes, 
+                                        50, 2*Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes).ToMinutes().ToAnnualInterest()
+   
+
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// This rule is executed only if some code coverage data is imported
+// from some code coverage files.
+//
+// This rule warns when the number of lines of a type covered by tests
+// decreased since the baseline. In case the type faced some refactoring
+// since the baseline, this loss in coverage is estimated only for types 
+// with more lines of code, where # lines of code covered now is lower
+// than # lines of code covered in baseline + the extra number of
+// lines of code.
+//
+// Such situation can mean that some tests have been removed
+// but more often, this means that the type has been modified,
+// and that changes haven't been covered properly by tests.
+//
+// To visualize changes in code, right-click a matched type and select:
+//
+// • Compare older and newer versions of source file
+//
+// • or Compare older and newer versions disassembled with Reflector
+//</Description>
+
+//<HowToFix>
+// Write more unit-tests dedicated to cover changes in matched types
+// not covered yet.
+// If you find some *hard-to-test code*, it is certainly a sign that this code
+// is not *well designed* and hence, needs refactoring.
+//
+// The estimated Debt, which means the effort to cover by test 
+// code that used to be covered, varies linearly 15 minutes to 3 hours,
+// depending on the number of lines of code that are not covered by tests anymore.
+//
+// Severity of issues of this rule varies from **High** to **Critical** 
+// depending on the number of lines of code that are not covered by tests anymore.
+// Because the loss in code coverage happened since the baseline,
+// the severity is high because it is important to focus on these issues 
+// **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid making complex methods even more complex</Name>
+warnif count > 0 
+
+let complexityScoreProc = new Func<IMethod, double>(m => 
+    (m.CyclomaticComplexity + m.ILCyclomaticComplexity/3 + 5*m.ILNestingDepth).Value)
+
+from m in JustMyCode.Methods where
+ !m.IsAbstract &&
+  m.IsPresentInBothBuilds() &&
+  m.CodeWasChanged() &&
+  m.OlderVersion().CyclomaticComplexity > 6 
+
+let complexityScore = complexityScoreProc(m)
+let oldComplexityScore = complexityScoreProc(m.OlderVersion())
+where complexityScore > oldComplexityScore 
+
+let complexityScoreDiff = complexityScoreProc(m) - complexityScoreProc(m.OlderVersion())
+orderby complexityScoreDiff descending
+
+select new { 
+   m,
+   oldComplexityScore ,
+   complexityScore ,
+   diff= complexityScoreDiff,
+
+   Debt = complexityScoreDiff.Linear(1, 15,   50, 60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for a tiny complexity increment
+   // to interest for severity critical for 2000 loc
+   AnnualInterest = complexityScoreDiff.Linear(1,     Severity.High.AnnualInterestThreshold().Value.TotalMinutes, 
+                                               50, 4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+   
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// The method complexity is measured through the code metric
+// *Cyclomatic Complexity* defined here:
+// http://www.ndepend.com/docs/code-metrics#CC
+//
+// This rule warns when a method already complex
+// (i.e with *Cyclomatic Complexity* higher than 6)
+// become even more complex since the baseline.
+//
+// This rule needs assemblies PDB files and source code 
+// to be available at analysis time, because the *Cyclomatic Complexity*
+// is inferred from the source code and source code location
+// is inferred from PDB files. See:
+// http://www.ndepend.com/docs/ndepend-analysis-inputs-explanation
+//
+// To visualize changes in code, right-click a matched method and select:
+//
+// • Compare older and newer versions of source file
+//
+// • or Compare older and newer versions disassembled with Reflector
+//</Description>
+
+//<HowToFix>
+// A large and complex method should be split in smaller methods, 
+// or even one or several classes can be created for that.
+//
+// During this process it is important to question the scope of each
+// variable local to the method. This can be an indication if
+// such local variable will become an instance field of the newly created class(es).
+//
+// Large *switch…case* structures might be refactored through the help
+// of a set of types that implement a common interface, the interface polymorphism
+// playing the role of the *switch cases tests*.
+//
+// Unit Tests can help: write tests for each method before extracting it 
+// to ensure you don't break functionality.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 15 to 60 minutes depending on the extra complexity added.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid making large methods even larger</Name>
+
+warnif count > 0 
+from m in JustMyCode.Methods where
+ !m.IsAbstract &&
+
+ // Eliminate constructors from match, since they get larger
+ // as soons as some fields initialization are added.
+ !m.IsConstructor &&
+ !m.IsClassConstructor &&
+
+  // Filter just here for optimization
+  m.NbLinesOfCode > 15 &&
+
+  m.IsPresentInBothBuilds() &&
+  m.CodeWasChanged() 
+
+let oldLoc = m.OlderVersion().NbLinesOfCode
+where oldLoc > 15 && m.NbLinesOfCode > oldLoc
+
+let diff = m.NbLinesOfCode - oldLoc
+where diff > 0
+orderby diff descending 
+
+select new { 
+   m,
+   oldLoc,
+   newLoc = m.NbLinesOfCode,
+   diff,
+
+   Debt = diff.Linear(1, 10,   100, 60).ToMinutes().ToDebt(),
+
+   // The annual interest varies linearly from interest for severity Medium for a tiny complexity increment
+   // to interest for severity critical for 2000 loc
+   AnnualInterest = diff .Linear(1,      Severity.High.AnnualInterestThreshold().Value.TotalMinutes, 
+                                 100, 4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+   
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// This rule warns when a method already large
+// (i.e with more than 15 lines of code)
+// become even larger since the baseline.
+//
+// The method size is measured through the code metric
+// *# Lines of Code* defined here:
+// http://www.ndepend.com/docs/code-metrics#NbLinesOfCode
+//
+// This rule needs assemblies PDB files 
+// to be available at analysis time, because the *# Lines of Code*
+// is inferred from PDB files. See:
+// http://www.ndepend.com/docs/ndepend-analysis-inputs-explanation
+//
+// To visualize changes in code, right-click a matched method and select:
+//
+// • Compare older and newer versions of source file
+//
+// • or Compare older and newer versions disassembled with Reflector
+//</Description>
+
+//<HowToFix>
+// Usually too big methods should be split in smaller methods.
+//
+// But long methods with no branch conditions, that typically initialize some data,
+// are not necessarily a problem to maintain, and might not need refactoring.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 5 to 20 minutes depending 
+// on the number of lines of code added.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 10 to 60 minutes depending on the extra complexity added.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid adding methods to a type that already had many methods</Name>
+
+warnif count > 0 
+
+// Don't count constructors and methods generated by the compiler!
+let getMethodsProc = new Func<IType, IList<IMethod>>(
+   t => t.Methods.Where(m =>
+      !m.IsConstructor && !m.IsClassConstructor && 
+      !m.IsGeneratedByCompiler).ToArray()) 
+
+
+from t in JustMyCode.Types where
+  
+  t.NbMethods > 30 &&   // Just here for optimization
+
+  t.IsPresentInBothBuilds()
+
+  // Optimization: fast discard of non-relevant types
+  where t.OlderVersion().NbMethods > 30
+
+  let oldMethods = getMethodsProc(t.OlderVersion())
+  where oldMethods.Count > 30
+
+  let newMethods = getMethodsProc(t)
+  where newMethods.Count > oldMethods.Count
+
+  let addedMethods = newMethods.Where(m => m.WasAdded())
+  let removedMethods = oldMethods.Where(m => m.WasRemoved())
+
+  orderby addedMethods.Count() descending
+
+select new { 
+   t,
+   nbOldMethods = oldMethods.Count,
+   nbNewMethods = newMethods.Count,
+   addedMethods,
+   removedMethods,
+
+   Debt           = (10*addedMethods.Count()).ToMinutes().ToDebt(),
+   AnnualInterest =     addedMethods.Count().Linear(
+          1,       Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes, 
+          100,  4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// Types where number of methods is greater than 15 
+// might be hard to understand and maintain.
+//
+// This rule lists types that already had more than 15 methods
+// at the baseline time, and for which new methods have been added.
+//
+// Having many methods for a type might be a symptom
+// of too many responsibilities implemented.
+//
+// Notice that constructors and methods generated by the compiler 
+// are not taken account.
+//</Description>
+
+//<HowToFix>
+// To refactor such type and increase code quality and maintainability,
+// certainly you'll have to split the type into several smaller types
+// that together, implement the same logic.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is equal to 10 minutes per method added.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid adding instance fields to a type that already had many instance fields</Name>
+
+warnif count > 0 
+
+let getFieldsProc = new Func<IType, IList<IField>>(
+   t => t.Fields.Where(f => 
+          !f.IsLiteral &&
+          !f.IsGeneratedByCompiler &&
+          !f.IsStatic).ToArray()) 
+
+
+from t in JustMyCode.Types where
+  
+ !t.IsEnumeration &&
+  t.IsPresentInBothBuilds()
+
+  // Optimization: fast discard of non-relevant types
+  where t.OlderVersion().NbFields > 15
+
+  let oldFields = getFieldsProc(t.OlderVersion())
+  where oldFields.Count > 15
+
+  let newFields = getFieldsProc(t)
+  where newFields.Count > oldFields.Count
+
+  let addedFields = newFields.Where(f => f.WasAdded())
+  let removedFields = oldFields.Where(f => f.WasRemoved())
+
+  orderby addedFields.Count() descending
+
+select new { 
+   t,
+   nbOldFields = oldFields.Count,
+   nbNewFields = newFields.Count,
+   addedFields,
+   removedFields,
+
+   Debt           = (10*addedFields.Count()).ToMinutes().ToDebt(),
+   AnnualInterest =     addedFields.Count().Linear(
+          1,       Severity.High.AnnualInterestThreshold().Value.TotalMinutes, 
+          100,  4*(Severity.High.AnnualInterestThreshold().Value.TotalMinutes)).ToMinutes().ToAnnualInterest()
+
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// Types where number of fields is greater than 15 
+// might be hard to understand and maintain.
+//
+// This rule lists types that already had more than 15 fields
+// at the baseline time, and for which new fields have been added.
+//
+// Having many fields for a type might be a symptom
+// of too many responsibilities implemented.
+//
+// Notice that *constants* fields and *static-readonly* fields are not taken account.
+// Enumerations types are not taken account also.
+//</Description>
+
+//<HowToFix>
+// To refactor such type and increase code quality and maintainability,
+// certainly you'll have to group subsets of fields into smaller types
+// and dispatch the logic implemented into the methods 
+// into these smaller types.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is equal to 10 minutes per field added.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[//<Name>Avoid transforming an immutable type into a mutable one</Name>
+
+warnif count > 0
+from t in Application.Types where
+   t.CodeWasChanged() &&
+   t.OlderVersion().IsImmutable &&
+  !t.IsImmutable && 
+  // Don't take account of immutable types transformed into static types (not deemed as immutable)
+  !t.IsStatic
+
+let culpritFields = t.InstanceFields.Where(f => f.IsImmutable)
+select new {
+   t, 
+   culpritFields,
+   Debt = (10 + 10*culpritFields.Count()).ToMinutes().ToDebt(),
+   Severity = Severity.High
+}
+
+//<Description>
+// This rule is executed only if a *baseline for comparison* is defined (*diff mode*).
+//
+// A type is considered as *immutable* if its instance fields
+// cannot be modified once an instance has been built by a constructor.
+//
+// Being immutable has several fortunate consequences for a type.
+// For example its instance objects can be used concurrently 
+// from several threads without the need to synchronize accesses.
+//
+// Hence users of such type often rely on the fact that the type is immutable.
+// If an immutable type becomes mutable, there are chances that this will break 
+// users code.
+//
+// This is why this rule warns about such immutable type that become mutable.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is equal to 2 minutes per instance field that became mutable.
+//</Description>
+
+//<HowToFix>
+// If being immutable is an important property for a matched type,
+// then the code must be refactored to preserve immutability.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// is equal to 10 minutes plus 10 minutes per instance fields of
+// the matched type that is now mutable.
+//
+// Issues of this rule have a **High** severity, because it is important to focus 
+// on these issues **now**, before such code gets released in production.
+//</HowToFix>]]></Query>
+    </Group>
+    <Group Name="Object Oriented Design" Active="True" ShownInReport="True">
+      <Query Active="False" DisplayList="True" DisplayStat="True" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Avoid interfaces too big</Name>
+warnif count > 0 
+
+from i in JustMyCode.Types
+where i.IsInterface && i.NbMethods >= 10 // Optimization  First threshold
+
+// A get;set; property count as one method
+let properties = i.Methods.Where(m => m.SimpleName.Length > 4 && (m.IsPropertyGetter || m.IsPropertySetter))
+  .Distinct(m => m.SimpleName.Substring(4, m.SimpleName.Length -4))
+
+// An event count as one method
+let events = i.Methods.Where(m => (m.IsEventAdder|| m.IsEventRemover))
+  .Distinct(m => m.SimpleName.Replace("add_","").Replace("remove_",""))
+
+let methods = i.Methods.Where(m => !m.IsPropertyGetter && !m.IsPropertySetter && !m.IsEventAdder && !m.IsEventRemover)
+let methodsCount = methods.Count() + properties.Count() + events.Count()
+where methodsCount >= 10
+let publicFactor = i.IsPubliclyVisible ? 1 : 0.5
+orderby methodsCount descending
+select new {  
+   i, 
+   Methods= methods,
+   Properties = properties,
+   Events = events,
+   Debt = (publicFactor*methodsCount.Linear(10, 20,   100, 7*60)).ToMinutes().ToDebt(),
+   // The annual interest varies linearly from interest for severity Medium for an interface with 10 methods
+   // to interest for severity Critical for an interface with 100 methods and more
+   AnnualInterest = (publicFactor*methodsCount.Linear(
+                       10,  Severity.Medium.AnnualInterestThreshold().Value.TotalMinutes,
+                       100, Severity.Critical.AnnualInterestThreshold().Value.TotalMinutes))
+                     .ToMinutes().ToAnnualInterest()
+}
+
+
+//<Description>
+// This rule matches interfaces with more than 10 methods.
+// Interfaces are abstractions and are meant to simplify the code structure.
+// An interface should represent a single responsibility.
+// Making an interface too large, too complex, necessarily means
+// that the interface has too many responsibilities.
+//
+// A property with getter or setter or both count as one method.
+// An event count as one method.
+//</Description>
+
+//<HowToFix>
+// Typically to fix such issue, the interface must be refactored
+// in a grape of smaller *single-responsibility* interfaces.
+//
+// A classic example is a *ISession* large interface, responsible
+// for holding states, run commands and offer various accesses
+// and facilities.
+//
+// The classic problem for a large public interface is that it has
+// many clients that consume it. As a consequence splitting it in 
+// smaller interfaces has an important impact and it is not always
+// feasible.
+//
+// The estimated Debt, which means the effort to fix such issue,
+// varies linearly from 20 minutes for an interface with 10 methods,
+// up to 7 hours for an interface with 100 or more methods.
+// The Debt is divided by two if the interface is not publicly
+// visible, because in such situation only the current project is impacted
+// by the refactoring.
+//</HowToFix>
+]]></Query>
+      <Query Active="True" DisplayList="True" DisplayStat="False" DisplaySelectionView="False" IsCriticalRule="False"><![CDATA[// <Name>Base class should not use derivatives</Name>
+warnif count > 0 
+let excludedTypes = new[] {"TcpDiscoveryIpFinderBase", "EvictionPolicyBase", "PlatformTargetAdapter"}
+from baseClass in JustMyCode.Types
+where baseClass.IsClass  && !excludedTypes.Contains(baseClass.Name)
+      && baseClass.NbChildren > 0 // <-- for optimization!
+let derivedClassesUsed = baseClass.DerivedTypes.UsedBy(baseClass)
+   // Don't warn when a base class is using nested private derived class
+   .Where(derivedClass => 
+            !(derivedClass.IsNested && 
+              derivedClass.Visibility == Visibility.Private && 
+              derivedClass.ParentType == baseClass
+              ))
+where derivedClassesUsed.Count() > 0
+
+let derivedClassesMemberUsed = derivedClassesUsed.SelectMany(c => c.Members).UsedBy(baseClass)
+orderby derivedClassesMemberUsed.Count() descending
+
+select new { 
+   baseClass, 
+   derivedClassesUsed,
+   derivedClassesMemberUsed,
+
+   Debt = 3*(derivedClassesUsed.Count()+derivedClassesMemberUsed.Count()).ToMinutes().ToDebt(),
+   Severity = Severity.High
+}
+
+//<Description>
+// In *Object-Oriented Programming*, the **open/closed principle** states:
+// *software entities (components, classes, methods, etc.) should be open 
+// for extension, but closed for modification*. 
+// http://en.wikipedia.org/wiki/Open/closed_principle
+//
+// Hence a base class should be designed properly to make it easy to derive from,
+// this is *extension*. But creating a new derived class, or modifying an
+// existing one, shouldn't provoke any *modification* in the base class.
+// And if a base class is using some derivative classes somehow, there
+// are good chances that such *modification* will be needed.
+//
+// Extending the base class is not anymore a simple operation,
+// this is not good design.
+//
+// Note that this rule doesn't warn when a base class is using a derived class 
+// that is nested in t

<TRUNCATED>

[11/25] ignite git commit: IGNITE-6546 Fixed faveicon.

Posted by sb...@apache.org.
IGNITE-6546 Fixed faveicon.


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

Branch: refs/heads/ignite-5937
Commit: f42f34b8dc293e2077ea99a49136d3e2d439e832
Parents: 3256ee2
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Oct 3 14:33:10 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Oct 3 14:33:10 2017 +0700

----------------------------------------------------------------------
 .../rest/protocols/http/jetty/favicon.ico        | Bin 1406 -> 1150 bytes
 1 file changed, 0 insertions(+), 0 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f42f34b8/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico
index 6f6ed44..b36f8d7 100644
Binary files a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico and b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/favicon.ico differ


[18/25] ignite git commit: .NET: Fix TestRecordLocal

Posted by sb...@apache.org.
.NET: Fix TestRecordLocal


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

Branch: refs/heads/ignite-5937
Commit: 855fe4b51982933742106678ccd01bdcba66d210
Parents: db69e90
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Oct 3 18:33:57 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Oct 3 18:33:57 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/855fe4b5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
index 672ff9e..a7c0534 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
@@ -314,7 +314,7 @@ namespace Apache.Ignite.Core.Tests
         [Test]
         public void TestRecordLocal()
         {
-            Assert.Throws<NotImplementedException>(() => _grid1.GetEvents().RecordLocal(new MyEvent()));
+            Assert.Throws<NotSupportedException>(() => _grid1.GetEvents().RecordLocal(new MyEvent()));
         }
 
         /// <summary>


[04/25] ignite git commit: ignite-5733 Add ability to activate/deactivate/checkstate through HTTP REST API

Posted by sb...@apache.org.
ignite-5733 Add ability to activate/deactivate/checkstate through HTTP REST API

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-5937
Commit: 252eedcccf33c81f1a4007627eb1a29b48b72f7f
Parents: d7fbbd5
Author: Alexander Belyak <al...@xored.com>
Authored: Mon Oct 2 12:58:18 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Mon Oct 2 12:58:18 2017 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     | 46 ++++++++++++++++++++
 .../processors/rest/GridRestCommand.java        |  2 +-
 .../processors/rest/GridRestProcessor.java      |  5 ++-
 .../http/jetty/GridJettyRestHandler.java        | 18 ++++++++
 4 files changed, 68 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/252eedcc/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 5f83f6f..b2725b8 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
@@ -710,6 +710,18 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
     /**
      * @throws Exception If failed.
      */
+    public void testDeactivateActivate() throws Exception {
+
+        assertClusterState(true);
+
+        changeClusterState(false);
+
+        changeClusterState(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testPut() throws Exception {
         String ret = content(F.asMap("cacheName", DEFAULT_CACHE_NAME, "cmd", GridRestCommand.CACHE_PUT.key(),
             "key", "putKey", "val", "putVal"));
@@ -2375,4 +2387,38 @@ public abstract class JettyRestProcessorAbstractSelfTest extends AbstractRestPro
 
         return cfg;
     }
+
+    /**
+     * Test if current cluster state equals expected.
+     *
+     * @param exp Expected state.
+     * @throws Exception If failed.
+     */
+    private void assertClusterState(boolean exp) throws Exception {
+        String ret = content(F.asMap("cmd", GridRestCommand.CLUSTER_CURRENT_STATE.key()));
+
+        info("Cluster state: " + ret);
+        JsonNode res = jsonResponse(ret);
+
+        assertEquals(exp, res.asBoolean());
+        assertEquals(exp, grid(0).active());
+    }
+
+    /**
+     * Change cluster state and test new state.
+     *
+     * @param state Desired state.
+     * @throws Exception If failed.
+     */
+    private void changeClusterState(boolean state) throws Exception {
+        String cmd = (state ? GridRestCommand.CLUSTER_ACTIVE : GridRestCommand.CLUSTER_INACTIVE).key();
+
+        String ret = content(F.asMap("cmd", cmd));
+
+        JsonNode res = jsonResponse(ret);
+
+        assertTrue(res.isNull());
+
+        assertClusterState(state);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/252eedcc/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
index 24b4bda..2b53b54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestCommand.java
@@ -166,7 +166,7 @@ public enum GridRestCommand {
     CLUSTER_INACTIVE("inactive"),
 
     /** */
-    CLUSTER_CURRENT_STATE("currentState");
+    CLUSTER_CURRENT_STATE("currentstate");
 
     /** Enum values. */
     private static final GridRestCommand[] VALS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/252eedcc/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 9842883..64ab173 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
@@ -67,8 +67,6 @@ import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.internal.util.worker.GridWorkerFuture;
-import org.apache.ignite.internal.visor.compute.VisorGatewayTask;
-import org.apache.ignite.internal.visor.misc.VisorChangeGridActiveStateTask;
 import org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteInClosure;
@@ -811,6 +809,9 @@ public class GridRestProcessor extends GridProcessorAdapter {
             case ATOMIC_DECREMENT:
             case NAME:
             case LOG:
+            case CLUSTER_CURRENT_STATE:
+            case CLUSTER_ACTIVE:
+            case CLUSTER_INACTIVE:
                 break;
 
             default:

http://git-wip-us.apache.org/repos/asf/ignite/blob/252eedcc/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
----------------------------------------------------------------------
diff --git a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
index 327c13a..b8a42f3 100644
--- a/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
+++ b/modules/rest-http/src/main/java/org/apache/ignite/internal/processors/rest/protocols/http/jetty/GridJettyRestHandler.java
@@ -45,6 +45,7 @@ import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
 import org.apache.ignite.internal.processors.rest.GridRestResponse;
 import org.apache.ignite.internal.processors.rest.request.DataStructuresRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestCacheRequest;
+import org.apache.ignite.internal.processors.rest.request.GridRestChangeStateRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestLogRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestTaskRequest;
@@ -63,6 +64,8 @@ import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_G
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_PUT_ALL;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.CACHE_REMOVE_ALL;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.EXECUTE_SQL_QUERY;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_ACTIVE;
+import static org.apache.ignite.internal.processors.rest.GridRestCommand.CLUSTER_CURRENT_STATE;
 import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_FAILED;
 
 /**
@@ -540,6 +543,21 @@ public class GridJettyRestHandler extends AbstractHandler {
                 break;
             }
 
+            case CLUSTER_ACTIVE:
+            case CLUSTER_INACTIVE:
+            case CLUSTER_CURRENT_STATE: {
+                GridRestChangeStateRequest restReq0 = new GridRestChangeStateRequest();
+
+                if (cmd == CLUSTER_CURRENT_STATE)
+                    restReq0.reqCurrentState();
+                else
+                    restReq0.active(cmd == CLUSTER_ACTIVE);
+
+                restReq = restReq0;
+
+                break;
+            }
+
             case EXECUTE_SQL_QUERY:
             case EXECUTE_SQL_FIELDS_QUERY: {
                 RestQueryRequest restReq0 = new RestQueryRequest();


[15/25] ignite git commit: Fixed affinityReadyFuture to use actual for result.

Posted by sb...@apache.org.
Fixed affinityReadyFuture to use actual for result.


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

Branch: refs/heads/ignite-5937
Commit: db69e90279d05ea980cf33ab4230c2c9d4dd2c01
Parents: b21f750
Author: sboikov <sb...@gridgain.com>
Authored: Tue Oct 3 16:42:01 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Oct 3 16:42:01 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/affinity/GridAffinityAssignmentCache.java  | 2 +-
 .../ignite/internal/processors/cache/GridCacheAffinityManager.java | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/db69e902/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index f921251..cbc4eae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -392,7 +392,7 @@ public class GridAffinityAssignmentCache {
                 log.debug("Completing topology ready future right away [head=" + aff.topologyVersion() +
                     ", topVer=" + topVer + ']');
 
-            fut.onDone(topVer);
+            fut.onDone(aff.topologyVersion());
         }
         else if (stopErr != null)
             fut.onDone(stopErr);

http://git-wip-us.apache.org/repos/asf/ignite/blob/db69e902/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index 702b848..fbe1a95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -99,7 +99,7 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
 
         IgniteInternalFuture<AffinityTopologyVersion> fut = aff.readyFuture(topVer);
 
-        return fut != null ? fut : new GridFinishedFuture<>(topVer);
+        return fut != null ? fut : cctx.shared().exchange().lastFinishedFuture();
     }
 
     /**


[17/25] ignite git commit: ignite-3478

Posted by sb...@apache.org.
ignite-3478


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

Branch: refs/heads/ignite-5937
Commit: fdfe779f063d015dd0fcb7f2880a1b227a3e190f
Parents: 07a5602
Author: sboikov <sb...@gridgain.com>
Authored: Tue Oct 3 17:34:30 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Oct 3 17:34:30 2017 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridPartitionedGetFuture.java  | 11 -----------
 1 file changed, 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fdfe779f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 67d918f..7993d05 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -675,17 +675,6 @@ public class GridPartitionedGetFuture<K, V> extends CacheDistributedGetFutureAda
         return Collections.emptyMap();
     }
 
-    /**
-     * @param curTopVer Current topology version.
-     * @return Future to wait for before remapping.
-     */
-    private IgniteInternalFuture<AffinityTopologyVersion> waitRemapFuture(AffinityTopologyVersion curTopVer) {
-        AffinityTopologyVersion updTopVer =
-            new AffinityTopologyVersion(Math.max(curTopVer.topologyVersion() + 1, cctx.discovery().topologyVersion()));
-
-        return cctx.affinity().affinityReadyFuture(updTopVer);
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         Collection<String> futs = F.viewReadOnly(futures(), new C1<IgniteInternalFuture<?>, String>() {


[25/25] ignite git commit: ignite-5937

Posted by sb...@apache.org.
ignite-5937


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

Branch: refs/heads/ignite-5937
Commit: f8be46d80875890f95970af34e55a418c7892434
Parents: e6940bd
Author: sboikov <sb...@gridgain.com>
Authored: Wed Oct 4 11:44:31 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Oct 4 11:44:31 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/tree/BPlusTree.java       | 62 ++++++++++++--------
 1 file changed, 36 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f8be46d8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index 05adb41..b6c5c96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -973,6 +973,35 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
     }
 
+    /**
+     * @param lower Lower bound inclusive.
+     * @param upper Upper bound inclusive.
+     * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row.
+     * @return First found item which meets bounds and pass predicate.
+     * @throws IgniteCheckedException If failed.
+     */
+    public T findOneBounded(L lower, L upper, RowPredicate<L, T> p, Object x) throws IgniteCheckedException {
+        checkDestroyed();
+
+        try {
+            FindOneCursor cursor = new FindOneCursor(lower, upper, p, x);
+
+            return cursor.findOne();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteCheckedException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
+        }
+        catch (RuntimeException e) {
+            throw new IgniteException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
+        }
+        catch (AssertionError e) {
+            throw new AssertionError("Assertion error on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
+        }
+        finally {
+            checkDestroyed();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public T findFirst() throws IgniteCheckedException {
         checkDestroyed();
@@ -4385,32 +4414,6 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     protected abstract T getRow(BPlusIO<L> io, long pageAddr, int idx, Object x) throws IgniteCheckedException;
 
-    public interface RowPredicate<L, T extends L> {
-        public boolean apply(BPlusTree<L, T> tree, BPlusIO<L> io, long pageAddr, int idx) throws IgniteCheckedException;
-    }
-
-    public T findOneBounded(L lower, L upper, RowPredicate<L, T> p, Object x) throws IgniteCheckedException {
-        checkDestroyed();
-
-        try {
-            FindOneCursor cursor = new FindOneCursor(lower, upper, p, x);
-
-            return cursor.findOne();
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteCheckedException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
-        }
-        catch (RuntimeException e) {
-            throw new IgniteException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
-        }
-        catch (AssertionError e) {
-            throw new AssertionError("Assertion error on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
-        }
-        finally {
-            checkDestroyed();
-        }
-    }
-
     /**
      *
      */
@@ -4996,4 +4999,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         DONE
     }
+
+    /**
+     *
+     */
+    public interface RowPredicate<L, T extends L> {
+        public boolean apply(BPlusTree<L, T> tree, BPlusIO<L> io, long pageAddr, int idx) throws IgniteCheckedException;
+    }
 }


[06/25] ignite git commit: IGNITE-6231 .NET: Do not start a new thread explicitly in Flusher.RunThread

Posted by sb...@apache.org.
IGNITE-6231 .NET: Do not start a new thread explicitly in Flusher.RunThread


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

Branch: refs/heads/ignite-5937
Commit: 7aaacd8f92645558430b5343a1cb8bf7e908529a
Parents: f57b677
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Oct 2 16:09:04 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Oct 2 16:09:04 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7aaacd8f/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
index fb2df01..96b24ab 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
@@ -858,7 +858,7 @@ namespace Apache.Ignite.Core.Impl.Datastream
             /// </summary>
             public void RunThread()
             {
-                new Thread(Run).Start();
+                Task.Factory.StartNew(Run);
             }
         }
 


[08/25] ignite git commit: IGNITE-6517 .NET: DataStreamer DefaultPerNodeBufferSize, DefaultParallelOpsMultiplier, Timeout

Posted by sb...@apache.org.
IGNITE-6517 .NET: DataStreamer DefaultPerNodeBufferSize, DefaultParallelOpsMultiplier, Timeout

This closes #2785


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

Branch: refs/heads/ignite-5937
Commit: 5764960e802e91b87956f4515e289eaf0003a2de
Parents: 5ca7909
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Oct 2 16:48:23 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Oct 2 16:48:23 2017 +0300

----------------------------------------------------------------------
 .../datastreamer/PlatformDataStreamer.java      | 14 ++++++
 .../Dataload/DataStreamerTest.cs                | 50 +++++++++++++++++---
 .../Apache.Ignite.Core.csproj                   |  1 +
 .../Datastream/DataStreamerDefaults.cs          | 46 ++++++++++++++++++
 .../Datastream/IDataStreamer.cs                 | 21 +++++++-
 .../Impl/Binary/BinaryReaderExtensions.cs       | 10 +---
 .../Impl/Binary/BinaryUtils.cs                  | 14 ++++++
 .../Impl/Datastream/DataStreamerImpl.cs         | 43 ++++++++++++++++-
 8 files changed, 179 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
index fba0a4c..8cd14c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -86,6 +86,12 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
     /** */
     private static final int OP_LISTEN_TOPOLOGY = 11;
 
+    /** */
+    private static final int OP_GET_TIMEOUT = 12;
+
+    /** */
+    private static final int OP_SET_TIMEOUT = 13;
+
     /** Cache name. */
     private final String cacheName;
 
@@ -230,6 +236,14 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
 
             case OP_PER_NODE_PARALLEL_OPS:
                 return ldr.perNodeParallelOperations();
+
+            case OP_GET_TIMEOUT:
+                return ldr.timeout();
+
+            case OP_SET_TIMEOUT:
+                ldr.timeout(val);
+
+                return TRUE;
         }
 
         return super.processInLongOutLong(type, val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
index fe5955f..60a1067 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTest.cs
@@ -95,25 +95,40 @@ namespace Apache.Ignite.Core.Tests.Dataload
         {
             using (IDataStreamer<int, int> ldr = _grid.GetDataStreamer<int, int>(CacheName))
             {
+                Assert.AreEqual(CacheName, ldr.CacheName);
+                Assert.AreEqual(0, ldr.AutoFlushFrequency);
+
+                Assert.IsFalse(ldr.AllowOverwrite);
                 ldr.AllowOverwrite = true;
                 Assert.IsTrue(ldr.AllowOverwrite);
                 ldr.AllowOverwrite = false;
                 Assert.IsFalse(ldr.AllowOverwrite);
 
+                Assert.IsFalse(ldr.SkipStore);
                 ldr.SkipStore = true;
                 Assert.IsTrue(ldr.SkipStore);
                 ldr.SkipStore = false;
                 Assert.IsFalse(ldr.SkipStore);
 
+                Assert.AreEqual(DataStreamerDefaults.DefaultPerNodeBufferSize, ldr.PerNodeBufferSize);
                 ldr.PerNodeBufferSize = 1;
                 Assert.AreEqual(1, ldr.PerNodeBufferSize);
                 ldr.PerNodeBufferSize = 2;
                 Assert.AreEqual(2, ldr.PerNodeBufferSize);
 
-                ldr.PerNodeParallelOperations = 1;
-                Assert.AreEqual(1, ldr.PerNodeParallelOperations);
+                Assert.AreEqual(0, ldr.PerNodeParallelOperations);
+                var ops = DataStreamerDefaults.DefaultParallelOperationsMultiplier *
+                          IgniteConfiguration.DefaultThreadPoolSize;
+                ldr.PerNodeParallelOperations = ops;
+                Assert.AreEqual(ops, ldr.PerNodeParallelOperations);
                 ldr.PerNodeParallelOperations = 2;
                 Assert.AreEqual(2, ldr.PerNodeParallelOperations);
+
+                Assert.AreEqual(DataStreamerDefaults.DefaultTimeout, ldr.Timeout);
+                ldr.Timeout = TimeSpan.MaxValue;
+                Assert.AreEqual(TimeSpan.MaxValue, ldr.Timeout);
+                ldr.Timeout = TimeSpan.FromSeconds(1.5);
+                Assert.AreEqual(1.5, ldr.Timeout.TotalSeconds);
             }
         }
 
@@ -123,28 +138,37 @@ namespace Apache.Ignite.Core.Tests.Dataload
         [Test]        
         public void TestAddRemove()
         {
-            using (IDataStreamer<int, int> ldr = _grid.GetDataStreamer<int, int>(CacheName))
+            IDataStreamer<int, int> ldr;
+
+            using (ldr = _grid.GetDataStreamer<int, int>(CacheName))
             {
+                Assert.IsFalse(ldr.Task.IsCompleted);
+
                 ldr.AllowOverwrite = true;
 
                 // Additions.
-                ldr.AddData(1, 1);
+                var task = ldr.AddData(1, 1);
                 ldr.Flush();                
                 Assert.AreEqual(1, _cache.Get(1));
+                Assert.IsTrue(task.IsCompleted);
+                Assert.IsFalse(ldr.Task.IsCompleted);
 
-                ldr.AddData(new KeyValuePair<int, int>(2, 2));
+                task = ldr.AddData(new KeyValuePair<int, int>(2, 2));
                 ldr.Flush();
                 Assert.AreEqual(2, _cache.Get(2));
+                Assert.IsTrue(task.IsCompleted);
 
-                ldr.AddData(new List<KeyValuePair<int, int>> { new KeyValuePair<int, int>(3, 3), new KeyValuePair<int, int>(4, 4) });
+                task = ldr.AddData(new [] { new KeyValuePair<int, int>(3, 3), new KeyValuePair<int, int>(4, 4) });
                 ldr.Flush();
                 Assert.AreEqual(3, _cache.Get(3));
                 Assert.AreEqual(4, _cache.Get(4));
+                Assert.IsTrue(task.IsCompleted);
 
                 // Removal.
-                ldr.RemoveData(1);
+                task = ldr.RemoveData(1);
                 ldr.Flush();
                 Assert.IsFalse(_cache.ContainsKey(1));
+                Assert.IsTrue(task.IsCompleted);
 
                 // Mixed.
                 ldr.AddData(5, 5);                
@@ -165,6 +189,8 @@ namespace Apache.Ignite.Core.Tests.Dataload
                 for (int i = 5; i < 13; i++)
                     Assert.AreEqual(i, _cache.Get(i));
             }
+
+            Assert.IsTrue(ldr.Task.IsCompleted);
         }
 
         /// <summary>
@@ -517,6 +543,16 @@ namespace Apache.Ignite.Core.Tests.Dataload
 
                 for (var i = 0; i < 100; i++)
                     Assert.AreEqual(i + 1, cache.Get(i).Val);
+
+                // Repeating WithKeepBinary call: valid args.
+                Assert.AreSame(ldr, ldr.WithKeepBinary<int, IBinaryObject>());
+
+                // Invalid type args.
+                var ex = Assert.Throws<InvalidOperationException>(() => ldr.WithKeepBinary<string, IBinaryObject>());
+
+                Assert.AreEqual(
+                    "Can't change type of binary streamer. WithKeepBinary has been called on an instance of " +
+                    "binary streamer with incompatible generic arguments.", ex.Message);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 67c540c..58abd26 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -104,6 +104,7 @@
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Configuration\Package-Info.cs" />
     <Compile Include="Configuration\ClientConnectorConfiguration.cs" />
+    <Compile Include="Datastream\DataStreamerDefaults.cs" />
     <Compile Include="Impl\Binary\BinaryTypeId.cs" />
     <Compile Include="Impl\Client\Cache\CacheFlags.cs" />
     <Compile Include="Impl\Client\Cache\Query\ClientQueryCursor.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/DataStreamerDefaults.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/DataStreamerDefaults.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/DataStreamerDefaults.cs
new file mode 100644
index 0000000..315ae7f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/DataStreamerDefaults.cs
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Datastream
+{
+    using System;
+
+    /// <summary>
+    /// Data streamer configuration defaults.
+    /// </summary>
+    public static class DataStreamerDefaults
+    {
+        /// <summary>
+        /// The default per node buffer size, see <see cref="IDataStreamer{TK,TV}.PerNodeBufferSize"/>.
+        /// </summary>
+        public const int DefaultPerNodeBufferSize = 512;
+
+        /// <summary>
+        /// Default multiplier for parallel operations per node:
+        /// <see cref="IDataStreamer{TK,TV}.PerNodeParallelOperations"/> = 
+        /// <see cref="IgniteConfiguration.DataStreamerThreadPoolSize"/> * 
+        /// <see cref="DefaultParallelOperationsMultiplier"/>.
+        /// </summary>
+        public const int DefaultParallelOperationsMultiplier = 8;
+
+        /// <summary>
+        /// The default timeout (see <see cref="IDataStreamer{TK,TV}.Timeout"/>).
+        /// Negative value means no timeout.
+        /// </summary>
+        public static readonly TimeSpan DefaultTimeout = TimeSpan.FromMilliseconds(-1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
index 222f6c3..277130c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Datastream/IDataStreamer.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Datastream
 {
     using System;
     using System.Collections.Generic;
+    using System.ComponentModel;
     using System.Threading.Tasks;
     using Apache.Ignite.Core.Cache.Store;
 
@@ -110,8 +111,9 @@ namespace Apache.Ignite.Core.Datastream
         /// <para />
         /// Setter must be called before any add/remove operation.
         /// <para />
-        /// Default is <c>1024</c>.
+        /// Default is <see cref="DataStreamerDefaults.DefaultPerNodeBufferSize"/>.
         /// </summary>
+        [DefaultValue(DataStreamerDefaults.DefaultPerNodeBufferSize)]
         int PerNodeBufferSize { get; set; }
 
         /// <summary>
@@ -119,7 +121,9 @@ namespace Apache.Ignite.Core.Datastream
         /// <para />
         /// Setter must be called before any add/remove operation.
         /// <para />
-        /// Default is <c>16</c>.
+        /// Default is 0, which means Ignite calculates this automatically as 
+        /// <see cref="IgniteConfiguration.DataStreamerThreadPoolSize"/> * 
+        /// <see cref="DataStreamerDefaults.DefaultParallelOperationsMultiplier"/>.
         /// </summary>
         int PerNodeParallelOperations { get; set; }
 
@@ -208,5 +212,18 @@ namespace Apache.Ignite.Core.Datastream
         /// <typeparam name="TV1">Value type in binary mode.</typeparam>
         /// <returns>Streamer instance with binary mode enabled.</returns>
         IDataStreamer<TK1, TV1> WithKeepBinary<TK1, TV1>();
+
+        /// <summary>
+        /// Gets or sets the timeout. Negative values mean no timeout.
+        /// Default is <see cref="DataStreamerDefaults.DefaultTimeout"/>.
+        /// <para />
+        /// Timeout is used in the following cases:
+        /// <li>Any data addition method can be blocked when all per node parallel operations are exhausted.
+        /// The timeout defines the max time you will be blocked waiting for a permit to add a chunk of data
+        /// into the streamer;</li> 
+        /// <li>Total timeout time for <see cref="Flush"/> operation;</li>
+        /// <li>Total timeout time for <see cref="Close"/> operation.</li>
+        /// </summary>
+        TimeSpan Timeout { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs
index 7556c41..da87d21 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs
@@ -58,15 +58,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <returns>TimeSpan.</returns>
         public static TimeSpan ReadLongAsTimespan(this IBinaryRawReader reader)
         {
-            long ms = reader.ReadLong();
-
-            if (ms >= TimeSpan.MaxValue.TotalMilliseconds)
-                return TimeSpan.MaxValue;
-
-            if (ms <= TimeSpan.MinValue.TotalMilliseconds)
-                return TimeSpan.MinValue;
-
-            return TimeSpan.FromMilliseconds(ms);
+            return BinaryUtils.LongToTimeSpan(reader.ReadLong());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
index 46e6752..139783d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryUtils.cs
@@ -1664,6 +1664,20 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
+        /// Converts long to timespan.
+        /// </summary>
+        public static TimeSpan LongToTimeSpan(long ms)
+        {
+            if (ms >= TimeSpan.MaxValue.TotalMilliseconds)
+                return TimeSpan.MaxValue;
+
+            if (ms <= TimeSpan.MinValue.TotalMilliseconds)
+                return TimeSpan.MinValue;
+
+            return TimeSpan.FromMilliseconds(ms);
+        }
+
+        /// <summary>
         /// Creates and instance from the type name in reader.
         /// </summary>
         private static T CreateInstance<T>(BinaryReader reader)

http://git-wip-us.apache.org/repos/asf/ignite/blob/5764960e/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
index 96b24ab..555c6e6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
@@ -93,6 +93,12 @@ namespace Apache.Ignite.Core.Impl.Datastream
         /** */
         private const int OpListenTopology = 11;
 
+        /** */
+        private const int OpGetTimeout = 12;
+
+        /** */
+        private const int OpSetTimeout = 13;
+
         /** Cache name. */
         private readonly string _cacheName;
 
@@ -356,8 +362,6 @@ namespace Apache.Ignite.Core.Impl.Datastream
         {
             get
             {
-                ThrowIfDisposed();
-
                 return _closeFut.Task;
             }
         }
@@ -549,6 +553,41 @@ namespace Apache.Ignite.Core.Impl.Datastream
         }
 
         /** <inheritDoc /> */
+        public TimeSpan Timeout
+        {
+            get
+            {
+                _rwLock.EnterReadLock();
+
+                try
+                {
+                    ThrowIfDisposed();
+
+                    return BinaryUtils.LongToTimeSpan(DoOutInOp(OpGetTimeout));
+                }
+                finally
+                {
+                    _rwLock.ExitReadLock();
+                }
+            }
+            set
+            {
+                _rwLock.EnterWriteLock();
+
+                try
+                {
+                    ThrowIfDisposed();
+
+                    DoOutInOp(OpSetTimeout, (long) value.TotalMilliseconds);
+                }
+                finally
+                {
+                    _rwLock.ExitWriteLock();
+                }
+            }
+        }
+
+        /** <inheritDoc /> */
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]
         protected override void Dispose(bool disposing)
         {


[23/25] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-3478

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-3478

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
#	modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
#	modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java


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

Branch: refs/heads/ignite-5937
Commit: a1d9ddd10dfa935b81703c4ba1e2670fba282389
Parents: fc7b7e2
Author: sboikov <sb...@gridgain.com>
Authored: Wed Oct 4 11:35:53 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Oct 4 11:35:53 2017 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/internal/GridKernalContext.java   | 2 +-
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java    | 2 +-
 .../cache/persistence/wal/reader/StandaloneGridKernalContext.java | 3 +--
 3 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d9ddd1/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 184b9a7..efe63e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -43,8 +43,8 @@ import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
-import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelper;
+import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
 import org.apache.ignite.internal.processors.igfs.IgfsHelper;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d9ddd1/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 58981d5..b287ae4 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
@@ -232,10 +232,10 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_US
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MEMORY_CONFIG;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_OFFHEAP_SIZE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PREFIX;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_OFFHEAP_SIZE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_RESTART_ENABLED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_PORT_RANGE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SPI_CLASS;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1d9ddd1/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index f01a2b1..3db9692 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.persistence.wal.reader;
 
 import java.io.File;
-import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.util.Iterator;
 import java.util.List;
@@ -49,8 +48,8 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.mvcc.CacheCoordinatorsProcessor;
-import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
 import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;


[19/25] ignite git commit: IGNITE-6285 Enhance persistent store paths handling - Fixes #2775.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
new file mode 100644
index 0000000..fe7e4df
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/filename/IgniteUidAsConsistentIdMigrationTest.java
@@ -0,0 +1,712 @@
+/*
+ * 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.db.filename;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.regex.Pattern;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridStringLogger;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.parseSubFolderName;
+
+/**
+ * Test for new and old style persistent storage folders generation
+ */
+public class IgniteUidAsConsistentIdMigrationTest extends GridCommonAbstractTest {
+    /** Cache name for test. */
+    public static final String CACHE_NAME = "dummy";
+
+    /** Clear DB folder after each test. May be set to false for local debug */
+    private static final boolean deleteAfter = true;
+
+    /** Clear DB folder before each test. */
+    private static final boolean deleteBefore = true;
+
+    /** Fail test if delete of DB folder was not completed. */
+    private static final boolean failIfDeleteNotCompleted = true;
+
+    /** Configured consistent id. */
+    private String configuredConsistentId;
+
+    /** Logger to accumulate messages, null will cause logger won't be customized */
+    private GridStringLogger strLog;
+
+    /** Clear properties after this test run. Flag protects from failed test */
+    private boolean clearPropsAfterTest = false;
+
+    /** Place storage in temp folder for current test run. */
+    private boolean placeStorageInTemp;
+
+    /** A path to persistent store custom path for current test run. */
+    private File pstStoreCustomPath;
+
+    /** A path to persistent store WAL work custom path. */
+    private File pstWalStoreCustomPath;
+
+    /** A path to persistent store WAL archive custom path. */
+    private File pstWalArchCustomPath;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+
+        if (deleteBefore)
+            deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        if (deleteAfter)
+            deleteWorkFiles();
+
+        if (clearPropsAfterTest) {
+            System.clearProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID);
+            System.clearProperty(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT);
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        boolean ok = true;
+
+        if (pstStoreCustomPath != null)
+            ok &= deleteRecursively(pstStoreCustomPath);
+        else
+            ok &= deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
+
+        if (pstWalArchCustomPath != null)
+            ok &= deleteRecursively(pstWalArchCustomPath);
+
+        if (pstWalStoreCustomPath != null)
+            ok &= deleteRecursively(pstWalStoreCustomPath);
+
+        ok &= deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "binary_meta", false));
+
+        if (failIfDeleteNotCompleted)
+            assertTrue(ok);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (configuredConsistentId != null)
+            cfg.setConsistentId(configuredConsistentId);
+
+        final PersistentStoreConfiguration psCfg = new PersistentStoreConfiguration();
+
+        if (placeStorageInTemp) {
+            final File tempDir = new File(System.getProperty("java.io.tmpdir"));
+
+            pstStoreCustomPath = new File(tempDir, "Store");
+            pstWalStoreCustomPath = new File(tempDir, "WalStore");
+            pstWalArchCustomPath = new File(tempDir, "WalArchive");
+
+            psCfg.setPersistentStorePath(pstStoreCustomPath.getAbsolutePath());
+            psCfg.setWalStorePath(pstWalStoreCustomPath.getAbsolutePath());
+            psCfg.setWalArchivePath(pstWalArchCustomPath.getAbsolutePath());
+        }
+
+        cfg.setPersistentStoreConfiguration(psCfg);
+
+        final MemoryConfiguration memCfg = new MemoryConfiguration();
+        final MemoryPolicyConfiguration memPolCfg = new MemoryPolicyConfiguration();
+
+        memPolCfg.setMaxSize(32 * 1024 * 1024); // we don't need much memory for this test
+        memCfg.setMemoryPolicies(memPolCfg);
+        cfg.setMemoryConfiguration(memCfg);
+
+        if (strLog != null)
+            cfg.setGridLogger(strLog);
+
+        return cfg;
+    }
+
+    /**
+     * Checks start on empty PDS folder, in that case node 0 should start with random UUID.
+     *
+     * @throws Exception if failed.
+     */
+    public void testNewStyleIdIsGenerated() throws Exception {
+        final Ignite ignite = startActivateFillDataGrid(0);
+
+        //test UUID is parsable from consistent ID test
+        UUID.fromString(ignite.cluster().localNode().consistentId().toString());
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite));
+        stopGrid(0);
+    }
+
+    /**
+     * Checks start on empty PDS folder, in that case node 0 should start with random UUID.
+     *
+     * @throws Exception if failed.
+     */
+    public void testNewStyleIdIsGeneratedInCustomStorePath() throws Exception {
+        placeStorageInTemp = true;
+        final Ignite ignite = startActivateFillDataGrid(0);
+
+        //test UUID is parsable from consistent ID test
+        UUID.fromString(ignite.cluster().localNode().consistentId().toString());
+        final String subfolderName = genNewStyleSubfolderName(0, ignite);
+
+        assertDirectoryExist("binary_meta", subfolderName);
+
+        assertDirectoryExist(pstWalArchCustomPath, subfolderName);
+        assertDirectoryExist(pstWalArchCustomPath, subfolderName);
+        assertDirectoryExist(pstStoreCustomPath, subfolderName);
+
+        stopGrid(0);
+    }
+
+    /**
+     * Checks start on empty PDS folder using configured ConsistentId. We should start using this ID in compatible mode.
+     *
+     * @throws Exception if failed.
+     */
+    public void testPreconfiguredConsitentIdIsApplied() throws Exception {
+        this.configuredConsistentId = "someConfiguredConsistentId";
+        Ignite ignite = startActivateFillDataGrid(0);
+
+        assertPdsDirsDefaultExist(configuredConsistentId);
+        stopGrid(0);
+    }
+
+    /**
+     * Checks start on configured ConsistentId with same value as default, this emulate old style folder is already
+     * available. We should restart using this folder.
+     *
+     * @throws Exception if failed
+     */
+    public void testRestartOnExistingOldStyleId() throws Exception {
+        final String expDfltConsistentId = "127.0.0.1:47500";
+
+        this.configuredConsistentId = expDfltConsistentId; //this is for create old node folder
+
+        final Ignite igniteEx = startActivateGrid(0);
+
+        final String expVal = "there is compatible mode with old style folders!";
+
+        igniteEx.getOrCreateCache(CACHE_NAME).put("hi", expVal);
+
+        assertPdsDirsDefaultExist(U.maskForFileName(configuredConsistentId));
+        stopGrid(0);
+
+        this.configuredConsistentId = null; //now set up grid on existing folder
+
+        final Ignite igniteRestart = startActivateGrid(0);
+
+        assertEquals(expDfltConsistentId, igniteRestart.cluster().localNode().consistentId());
+        final IgniteCache<Object, Object> cache = igniteRestart.cache(CACHE_NAME);
+
+        assertNotNull("Expected to have cache [" + CACHE_NAME + "] using [" + expDfltConsistentId + "] as PDS folder", cache);
+        final Object valFromCache = cache.get("hi");
+
+        assertNotNull("Expected to load data from cache using [" + expDfltConsistentId + "] as PDS folder", valFromCache);
+        assertTrue(expVal.equals(valFromCache));
+        stopGrid(0);
+    }
+
+    /**
+     * Start stop grid without activation should cause lock to be released and restarted node should have index 0
+     *
+     * @throws Exception if failed
+     */
+    public void testStartWithoutActivate() throws Exception {
+        //start stop grid without activate
+        startGrid(0);
+        stopGrid(0);
+
+        Ignite igniteRestart = startActivateFillDataGrid(0);
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, igniteRestart));
+        stopGrid(0);
+    }
+
+    /**
+     * Checks start on empty PDS folder, in that case node 0 should start with random UUID
+     *
+     * @throws Exception if failed
+     */
+    public void testRestartOnSameFolderWillCauseSameUuidGeneration() throws Exception {
+        final UUID uuid;
+        {
+            final Ignite ignite = startActivateFillDataGrid(0);
+
+            assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite));
+
+            uuid = (UUID)ignite.cluster().localNode().consistentId();
+            stopGrid(0);
+        }
+
+        {
+            final Ignite igniteRestart = startActivateGrid(0);
+
+            assertTrue("there!".equals(igniteRestart.cache(CACHE_NAME).get("hi")));
+
+            final Object consIdRestart = igniteRestart.cluster().localNode().consistentId();
+
+            assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, igniteRestart));
+            stopGrid(0);
+
+            assertEquals(uuid, consIdRestart);
+        }
+    }
+
+    /**
+     * This test starts node, activates, deactivates node, and then start second node.
+     * Expected behaviour is following: second node will join topology with separate node folder
+     *
+     * @throws Exception if failed
+     */
+    public void testStartNodeAfterDeactivate() throws Exception {
+        final UUID uuid;
+        {
+            final Ignite ignite = startActivateFillDataGrid(0);
+
+            assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite));
+
+            uuid = (UUID)ignite.cluster().localNode().consistentId();
+            ignite.active(false);
+        }
+        {
+            final Ignite igniteRestart = startActivateGrid(1);
+
+            grid(0).active(true);
+            final Object consIdRestart = igniteRestart.cluster().localNode().consistentId();
+
+            assertPdsDirsDefaultExist(genNewStyleSubfolderName(1, igniteRestart));
+
+            stopGrid(1);
+            assertFalse(consIdRestart.equals(uuid));
+        }
+        stopGrid(0);
+        assertNodeIndexesInFolder(0, 1);
+    }
+
+    /**
+     * @param idx Index of the grid to start.
+     * @return Started and activated grid.
+     * @throws Exception If failed.
+     */
+    @NotNull private Ignite startActivateFillDataGrid(int idx) throws Exception {
+        final Ignite ignite = startActivateGrid(idx);
+
+        ignite.getOrCreateCache(CACHE_NAME).put("hi", "there!");
+
+        return ignite;
+    }
+
+    /**
+     * Starts and activates new grid with given index.
+     *
+     * @param idx Index of the grid to start.
+     * @return Started and activated grid.
+     * @throws Exception If anything failed.
+     */
+    @NotNull private Ignite startActivateGrid(int idx) throws Exception {
+        final Ignite ignite = startGrid(idx);
+
+        ignite.active(true);
+
+        return ignite;
+    }
+
+    /**
+     * Generates folder name in new style using constant prefix and UUID
+     *
+     * @param nodeIdx expected node index to check
+     * @param ignite ignite instance
+     * @return name of storage related subfolders
+     */
+    @NotNull private String genNewStyleSubfolderName(final int nodeIdx, final Ignite ignite) {
+        final Object consistentId = ignite.cluster().localNode().consistentId();
+
+        assertTrue("For new style folders consistent ID should be UUID," +
+                " but actual class is " + (consistentId == null ? null : consistentId.getClass()),
+            consistentId instanceof UUID);
+
+        return PdsConsistentIdProcessor.genNewStyleSubfolderName(nodeIdx, (UUID)consistentId);
+    }
+
+    /**
+     * test two nodes started at the same db root folder, second node should get index 1
+     *
+     * @throws Exception if failed
+     */
+    public void testNodeIndexIncremented() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+        final Ignite ignite1 = startGrid(1);
+
+        ignite0.active(true);
+
+        ignite0.getOrCreateCache(CACHE_NAME).put("hi", "there!");
+        ignite1.getOrCreateCache(CACHE_NAME).put("hi1", "there!");
+
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite0));
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(1, ignite1));
+
+        stopGrid(0);
+        stopGrid(1);
+        assertNodeIndexesInFolder(0, 1);
+    }
+
+    /**
+     * Test verified that new style folder is taken always with lowest index
+     *
+     * @throws Exception if failed
+     */
+    public void testNewStyleAlwaysSmallestNodeIndexIsCreated() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+        final Ignite ignite1 = startGrid(1);
+        final Ignite ignite2 = startGrid(2);
+        final Ignite ignite3 = startGrid(3);
+
+        ignite0.active(true);
+
+        ignite0.getOrCreateCache(CACHE_NAME).put("hi", "there!");
+        ignite3.getOrCreateCache(CACHE_NAME).put("hi1", "there!");
+
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite0));
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(1, ignite1));
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(2, ignite2));
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(3, ignite3));
+
+        assertNodeIndexesInFolder(0, 1, 2, 3);
+        stopAllGrids();
+
+        //this grid should take folder with index 0 as unlocked
+        final Ignite ignite4Restart = startActivateGrid(3);
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite4Restart));
+
+        assertNodeIndexesInFolder(0, 1, 2, 3);
+        stopAllGrids();
+    }
+
+    /**
+     * Test verified that new style folder is taken always with lowest index
+     *
+     * @throws Exception if failed
+     */
+    public void testNewStyleAlwaysSmallestNodeIndexIsCreatedMultithreaded() throws Exception {
+        final Ignite ignite0 = startGridsMultiThreaded(11);
+
+        ignite0.active(true);
+
+        ignite0.getOrCreateCache(CACHE_NAME).put("hi", "there!");
+        ignite0.getOrCreateCache(CACHE_NAME).put("hi1", "there!");
+
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite0));
+
+        assertNodeIndexesInFolder(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+        stopAllGrids();
+
+        //this grid should take folder with index 0 as unlocked
+        final Ignite ignite4Restart = startActivateGrid(4);
+        assertPdsDirsDefaultExist(genNewStyleSubfolderName(0, ignite4Restart));
+        stopAllGrids();
+
+        assertNodeIndexesInFolder(0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+    }
+
+    /**
+     * Test start two nodes with predefined conistent ID (emulate old fashion node). Then restart two nodes. Expected
+     * both nodes will get its own old folders
+     *
+     * @throws Exception if failed.
+     */
+    public void testStartTwoOldStyleNodes() throws Exception {
+        final String expDfltConsistentId1 = "127.0.0.1:47500";
+
+        this.configuredConsistentId = expDfltConsistentId1; //this is for create old node folder
+        final Ignite ignite = startGrid(0);
+
+        final String expDfltConsistentId2 = "127.0.0.1:47501";
+
+        this.configuredConsistentId = expDfltConsistentId2; //this is for create old node folder
+        final Ignite ignite2 = startGrid(1);
+
+        ignite.active(true);
+
+        final String expVal = "there is compatible mode with old style folders!";
+
+        ignite2.getOrCreateCache(CACHE_NAME).put("hi", expVal);
+
+        assertPdsDirsDefaultExist(U.maskForFileName(expDfltConsistentId1));
+        assertPdsDirsDefaultExist(U.maskForFileName(expDfltConsistentId2));
+        stopAllGrids();
+
+        this.configuredConsistentId = null; //now set up grid on existing folder
+
+        final Ignite igniteRestart = startGrid(0);
+        final Ignite igniteRestart2 = startGrid(1);
+
+        igniteRestart2.active(true);
+
+        assertEquals(expDfltConsistentId1, igniteRestart.cluster().localNode().consistentId());
+        assertEquals(expDfltConsistentId2, igniteRestart2.cluster().localNode().consistentId());
+
+        final IgniteCache<Object, Object> cache = igniteRestart.cache(CACHE_NAME);
+
+        assertNotNull("Expected to have cache [" + CACHE_NAME + "] using [" + expDfltConsistentId1 + "] as PDS folder", cache);
+        final Object valFromCache = cache.get("hi");
+
+        assertNotNull("Expected to load data from cache using [" + expDfltConsistentId1 + "] as PDS folder", valFromCache);
+        assertTrue(expVal.equals(valFromCache));
+
+        assertNodeIndexesInFolder(); //no new style nodes should be found
+        stopGrid(0);
+    }
+
+    /**
+     * Tests compatible mode enabled by this test to start.
+     * Expected to be 2 folders and no new style folders in this case.
+     *
+     * @throws Exception if failed.
+     */
+    public void testStartOldStyleNodesByCompatibleProperty() throws Exception {
+        clearPropsAfterTest = true;
+        System.setProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, "true");
+
+        final Ignite ignite1 = startGrid(0);
+        final Ignite ignite2 = startGrid(1);
+
+        ignite1.active(true);
+
+        final String expVal = "there is compatible mode with old style folders!";
+
+        ignite2.getOrCreateCache(CACHE_NAME).put("hi", expVal);
+
+        assertNodeIndexesInFolder(); // expected to have no new style folders
+
+        final Object consistentId1 = ignite1.cluster().localNode().consistentId();
+
+        assertPdsDirsDefaultExist(U.maskForFileName(consistentId1.toString()));
+        final Object consistentId2 = ignite2.cluster().localNode().consistentId();
+
+        assertPdsDirsDefaultExist(U.maskForFileName(consistentId2.toString()));
+        stopAllGrids();
+
+        System.clearProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID);
+        final Ignite igniteRestart = startGrid(0);
+        final Ignite igniteRestart2 = startGrid(1);
+
+        igniteRestart2.active(true);
+
+        assertEquals(consistentId1, igniteRestart.cluster().localNode().consistentId());
+        assertEquals(consistentId2, igniteRestart2.cluster().localNode().consistentId());
+
+        assertNodeIndexesInFolder(); //new style nodes should not be found
+        stopGrid(0);
+    }
+
+    /**
+     * Tests compatible mode enabled by this test to start, also no port is enabled.
+     * Expected to be 1 folder and no new style folders in this case.
+     *
+     * @throws Exception if failed.
+     */
+    public void testStartOldStyleNoPortsNodesByCompatibleProperty() throws Exception {
+        clearPropsAfterTest = true;
+        System.setProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, "true");
+        System.setProperty(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT, "true");
+
+        final Ignite ignite1 = startGrid(0);
+
+        ignite1.active(true);
+
+        final String expVal = "there is compatible mode with old style folders!";
+
+        ignite1.getOrCreateCache(CACHE_NAME).put("hi", expVal);
+
+        assertNodeIndexesInFolder(); // expected to have no new style folders
+
+        final Object consistentId1 = ignite1.cluster().localNode().consistentId();
+
+        assertPdsDirsDefaultExist(U.maskForFileName(consistentId1.toString()));
+        stopAllGrids();
+
+        System.clearProperty(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID);
+
+        final Ignite igniteRestart = startGrid(0);
+
+        igniteRestart.active(true);
+
+        assertEquals(consistentId1, igniteRestart.cluster().localNode().consistentId());
+
+        assertNodeIndexesInFolder(); //new style nodes should not be found
+        stopGrid(0);
+        System.clearProperty(IGNITE_CONSISTENT_ID_BY_HOST_WITHOUT_PORT);
+    }
+
+    /**
+     * Test case If there are no matching folders,
+     * but the directory contains old-style consistent IDs.
+     * Ignite should print out a warning.
+     *
+     * @throws Exception if failed.
+     */
+    public void testOldStyleNodeWithUnexpectedPort() throws Exception {
+        this.configuredConsistentId = "127.0.0.1:49999"; //emulated old-style node with not appropriate consistent ID
+        final Ignite ignite = startActivateFillDataGrid(0);
+        final IgniteCache<Object, Object> second = ignite.getOrCreateCache("second");
+
+        final int entries = 100;
+
+        for (int i = 0; i < entries; i++)
+            second.put((int)(Math.random() * entries), getClass().getName());
+
+        final String prevVerFolder = U.maskForFileName(ignite.cluster().localNode().consistentId().toString());
+        final String path = new File(new File(U.defaultWorkDirectory(), "db"), prevVerFolder).getCanonicalPath();
+
+        assertPdsDirsDefaultExist(prevVerFolder);
+        stopAllGrids();
+
+        this.configuredConsistentId = null;
+        this.strLog = new GridStringLogger();
+        startActivateGrid(0);
+        assertNodeIndexesInFolder(0); //one 0 index folder is created
+
+        final String wholeNodeLog = strLog.toString();
+        stopAllGrids();
+
+        String foundWarning = null;
+        for (String line : wholeNodeLog.split("\n")) {
+            if (line.contains("There is other non-empty storage folder under storage base directory")) {
+                foundWarning = line;
+                break;
+            }
+        }
+
+        if (foundWarning != null)
+            log.info("\nWARNING generated successfully [\n" + foundWarning + "\n]");
+
+        assertTrue("Expected to warn user on existence of old style path",
+            foundWarning != null);
+
+        assertTrue("Expected to warn user on existence of old style path [" + path + "]",
+            foundWarning.contains(path));
+
+        assertTrue("Expected to print some size for [" + path + "]",
+            Pattern.compile(" [0-9]* bytes").matcher(foundWarning).find());
+
+        strLog = null;
+        startActivateGrid(0);
+        assertNodeIndexesInFolder(0); //one 0 index folder is created
+        stopAllGrids();
+    }
+
+    /**
+     * @param indexes expected new style node indexes in folders
+     * @throws IgniteCheckedException if failed
+     */
+    private void assertNodeIndexesInFolder(Integer... indexes) throws IgniteCheckedException {
+        assertEquals(new TreeSet<>(Arrays.asList(indexes)), getAllNodeIndexesInFolder());
+    }
+
+    /**
+     * @return set of all indexes of nodes found in work folder
+     * @throws IgniteCheckedException if failed.
+     */
+    @NotNull private Set<Integer> getAllNodeIndexesInFolder() throws IgniteCheckedException {
+        final File curFolder = new File(U.defaultWorkDirectory(), PdsConsistentIdProcessor.DB_DEFAULT_FOLDER);
+        final Set<Integer> indexes = new TreeSet<>();
+        final File[] files = curFolder.listFiles(PdsConsistentIdProcessor.DB_SUBFOLDERS_NEW_STYLE_FILTER);
+
+        for (File file : files) {
+            final PdsConsistentIdProcessor.FolderCandidate uid = parseSubFolderName(file, log);
+
+            if (uid != null)
+                indexes.add(uid.nodeIndex());
+        }
+
+        return indexes;
+    }
+
+    /**
+     * Checks existence of all storage-related directories
+     *
+     * @param subDirName sub directories name expected
+     * @throws IgniteCheckedException if IO error occur
+     */
+    private void assertPdsDirsDefaultExist(String subDirName) throws IgniteCheckedException {
+        assertDirectoryExist("binary_meta", subDirName);
+        assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_STORE_PATH, subDirName);
+        assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_ARCHIVE_PATH, subDirName);
+        assertDirectoryExist(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER, subDirName);
+    }
+
+    /**
+     * Checks one folder existence.
+     *
+     * @param subFolderNames sub folders chain array to touch.
+     * @throws IgniteCheckedException if IO error occur.
+     */
+    private void assertDirectoryExist(String... subFolderNames) throws IgniteCheckedException {
+        final File curFolder = new File(U.defaultWorkDirectory());
+
+        assertDirectoryExist(curFolder, subFolderNames);
+    }
+
+
+    /**
+     * Checks one folder existence.
+     *
+     * @param workFolder current work folder.
+     * @param subFolderNames sub folders chain array to touch.
+     * @throws IgniteCheckedException if IO error occur.
+     */
+    private void assertDirectoryExist(final File workFolder, String... subFolderNames) throws IgniteCheckedException {
+        File curFolder = workFolder;
+
+        for (String name : subFolderNames) {
+            curFolder = new File(curFolder, name);
+        }
+
+        final String path;
+
+        try {
+            path = curFolder.getCanonicalPath();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to convert path: [" + curFolder.getAbsolutePath() + "]", e);
+        }
+        assertTrue("Directory " + Arrays.asList(subFolderNames).toString()
+            + " is expected to exist [" + path + "]", curFolder.exists() && curFolder.isDirectory());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
index c160f60..bf8cd85 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Random;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
@@ -68,6 +69,7 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PageDeltaRecord;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMemoryEx;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.TrackingPageIO;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -575,9 +577,11 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
             for (int i = 0; i < 100; i++)
                 cache.put(i, new IndexedObject(i));
 
+            final Object consistentId = ignite.cluster().localNode().consistentId();
+
             stopGrid(1);
 
-            final File cacheDir = cacheDir("partitioned", ignite.context().discovery().consistentId().toString());
+            final File cacheDir = cacheDir("partitioned", consistentId.toString());
 
             final boolean renamed = cacheDir.renameTo(new File(cacheDir.getParent(), "cache-partitioned0"));
 
@@ -605,14 +609,15 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
      * @return Cache dir.
      * @throws IgniteCheckedException If fail.
      */
-    private File cacheDir(final String cacheName, String consId) throws IgniteCheckedException {
-        consId = consId.replaceAll("[\\.:]", "_");
+    private File cacheDir(final String cacheName, final String consId) throws IgniteCheckedException {
+        final String subfolderName
+            = PdsConsistentIdProcessor.genNewStyleSubfolderName(0, UUID.fromString(consId));
 
         final File dbDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
 
         assert dbDir.exists();
 
-        final File consIdDir = new File(dbDir.getAbsolutePath(), consId);
+        final File consIdDir = new File(dbDir.getAbsolutePath(), subfolderName);
 
         assert consIdDir.exists();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index e2f58bd..b9c60b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -30,6 +30,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -64,14 +65,17 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.genNewStyleSubfolderName;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
@@ -87,9 +91,6 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     /** additional cache for testing different combinations of types in WAL */
     private static final String CACHE_ADDL_NAME = "cache1";
 
-    /** Fill wal with some data before iterating. Should be true for non local run */
-    private static final boolean fillWalBeforeTest = true;
-
     /** Delete DB dir before test. */
     private static final boolean deleteBefore = true;
 
@@ -152,7 +153,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
+    @Override protected void beforeTest() throws Exception {
         stopAllGrids();
 
         if (deleteBefore)
@@ -171,8 +172,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private void deleteWorkFiles() throws IgniteCheckedException {
-        if (fillWalBeforeTest)
-            deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
     }
 
     /**
@@ -181,27 +181,23 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     public void testFillWalAndReadRecords() throws Exception {
         final int cacheObjectsToWrite = 10000;
 
-        final String consistentId;
-        if (fillWalBeforeTest) {
-            final Ignite ignite0 = startGrid("node0");
+        final Ignite ignite0 = startGrid("node0");
 
-            ignite0.active(true);
+        ignite0.active(true);
 
-            consistentId = U.maskForFileName(ignite0.cluster().localNode().consistentId().toString());
+        final Serializable consistentId = (Serializable)ignite0.cluster().localNode().consistentId();
+        final String subfolderName = genNewStyleSubfolderName(0, (UUID)consistentId);
 
-            putDummyRecords(ignite0, cacheObjectsToWrite);
+        putDummyRecords(ignite0, cacheObjectsToWrite);
 
-            stopGrid("node0");
-        }
-        else
-            consistentId = "127_0_0_1_47500";
+        stopGrid("node0");
 
         final String workDir = U.defaultWorkDirectory();
         final File db = U.resolveWorkDirectory(workDir, DFLT_STORE_DIR, false);
         final File wal = new File(db, "wal");
         final File walArchive = new File(wal, "archive");
 
-        final MockWalIteratorFactory mockItFactory = new MockWalIteratorFactory(log, PAGE_SIZE, consistentId, WAL_SEGMENTS);
+        final MockWalIteratorFactory mockItFactory = new MockWalIteratorFactory(log, PAGE_SIZE, consistentId, subfolderName, WAL_SEGMENTS);
         final WALIterator it = mockItFactory.iterator(wal, walArchive);
         final int cntUsingMockIter = iterateAndCount(it, false);
 
@@ -209,11 +205,11 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         assert cntUsingMockIter > 0;
         assert cntUsingMockIter > cacheObjectsToWrite;
 
-        final File walArchiveDirWithConsistentId = new File(walArchive, consistentId);
-        final File walWorkDirWithConsistentId = new File(wal, consistentId);
+        final File walArchiveDirWithConsistentId = new File(walArchive, subfolderName);
+        final File walWorkDirWithConsistentId = new File(wal, subfolderName);
 
         final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false);
-        final File binaryMetaWithConsId = new File(binaryMeta, consistentId);
+        final File binaryMetaWithConsId = new File(binaryMeta, subfolderName);
         final File marshaller = U.resolveWorkDirectory(workDir, "marshaller", false);
 
         final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log, PAGE_SIZE, binaryMetaWithConsId, marshaller);
@@ -304,7 +300,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         final IgniteEvents evts = ignite.events();
 
         if (!evts.isEnabled(EVT_WAL_SEGMENT_ARCHIVED))
-            return; //nothing to test
+            assertTrue("nothing to test", false);
 
         evts.localListen(new IgnitePredicate<Event>() {
             @Override public boolean apply(Event e) {
@@ -428,29 +424,23 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         final int cntEntries = 1000;
         final int txCnt = 100;
 
-        final Map<Object, Object> ctrlMap = new HashMap<>();
-        final String consistentId;
-        if (fillWalBeforeTest) {
-            final Ignite ignite0 = startGrid("node0");
-
-            ignite0.active(true);
-
-            final IgniteCache<Object, Object> entries = txPutDummyRecords(ignite0, cntEntries, txCnt);
+        final Ignite ignite0 = startGrid("node0");
 
-            for (Cache.Entry<Object, Object> next : entries) {
-                ctrlMap.put(next.getKey(), next.getValue());
-            }
+        ignite0.active(true);
 
-            consistentId = U.maskForFileName(ignite0.cluster().localNode().consistentId().toString());
+        final IgniteCache<Object, Object> entries = txPutDummyRecords(ignite0, cntEntries, txCnt);
 
-            stopGrid("node0");
+        final Map<Object, Object> ctrlMap = new HashMap<>();
+        for (Cache.Entry<Object, Object> next : entries) {
+            ctrlMap.put(next.getKey(), next.getValue());
         }
-        else
-            consistentId = "127_0_0_1_47500";
+
+        final String subfolderName = genDbSubfolderName(ignite0, 0);
+        stopGrid("node0");
 
         final String workDir = U.defaultWorkDirectory();
         final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false);
-        final File binaryMetaWithConsId = new File(binaryMeta, consistentId);
+        final File binaryMetaWithConsId = new File(binaryMeta, subfolderName);
         final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false);
 
         final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log,
@@ -474,17 +464,28 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
                 }
             }
         };
-        scanIterateAndCount(factory, workDir, consistentId, cntEntries, txCnt, objConsumer, null);
+        scanIterateAndCount(factory, workDir, subfolderName, cntEntries, txCnt, objConsumer, null);
 
         assert ctrlMap.isEmpty() : " Control Map is not empty after reading entries " + ctrlMap;
     }
 
     /**
+     * Generates DB subfolder name for provided node index (local) and UUID (consistent ID)
+     *
+     * @param ignite ignite instance.
+     * @param nodeIdx node index.
+     * @return folder file name
+     */
+    @NotNull private String genDbSubfolderName(Ignite ignite, int nodeIdx) {
+        return genNewStyleSubfolderName(nodeIdx, (UUID)ignite.cluster().localNode().consistentId());
+    }
+
+    /**
      * Scan WAL and WAL archive for logical records and its entries.
      *
      * @param factory WAL iterator factory.
      * @param workDir Ignite work directory.
-     * @param consistentId consistent ID.
+     * @param subfolderName DB subfolder name based on consistent ID.
      * @param expCntEntries minimum expected entries count to find.
      * @param expTxCnt minimum expected transaction count to find.
      * @param objConsumer object handler, called for each object found in logical data records.
@@ -494,7 +495,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     private void scanIterateAndCount(
         final IgniteWalIteratorFactory factory,
         final String workDir,
-        final String consistentId,
+        final String subfolderName,
         final int expCntEntries,
         final int expTxCnt,
         @Nullable final BiConsumer<Object, Object> objConsumer,
@@ -504,11 +505,10 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         final File wal = new File(db, "wal");
         final File walArchive = new File(wal, "archive");
 
-        final File walArchiveDirWithConsistentId = new File(walArchive, consistentId);
+        final File walArchiveDirWithConsistentId = new File(walArchive, subfolderName);
 
         final File[] files = walArchiveDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER);
-
-        assert files != null : "Can't iterate over files [" + walArchiveDirWithConsistentId + "] Directory is N/A";
+        A.notNull(files, "Can't iterate over files [" + walArchiveDirWithConsistentId + "] Directory is N/A");
         final WALIterator iter = factory.iteratorArchiveFiles(files);
 
         final Map<GridCacheVersion, Integer> cntArch = iterateAndCountDataRecord(iter, objConsumer, dataRecordHnd);
@@ -520,8 +520,8 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
 
         log.info("Total tx found loaded using archive directory (file-by-file): " + txCntObservedArch);
 
-        final File walWorkDirWithConsistentId = new File(wal, consistentId);
-        final File[] workFiles = walWorkDirWithConsistentId.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER);
+        final File walWorkDirWithNodeSubDir = new File(wal, subfolderName);
+        final File[] workFiles = walWorkDirWithNodeSubDir.listFiles(FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER);
 
         final WALIterator tuples = factory.iteratorWorkFiles(workFiles);
         final Map<GridCacheVersion, Integer> cntWork = iterateAndCountDataRecord(tuples, objConsumer, dataRecordHnd);
@@ -541,71 +541,66 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
      */
     public void testFillWalWithDifferentTypes() throws Exception {
         int cntEntries = 0;
-        final String consistentId;
 
         final Map<Object, Object> ctrlMap = new HashMap<>();
         final Map<Object, Object> ctrlMapForBinaryObjects = new HashMap<>();
         final Collection<String> ctrlStringsToSearch = new HashSet<>();
         final Collection<String> ctrlStringsForBinaryObjSearch = new HashSet<>();
-        if (fillWalBeforeTest) {
-            final Ignite ignite0 = startGrid("node0");
-            ignite0.active(true);
-
-            final IgniteCache<Object, Object> addlCache = ignite0.getOrCreateCache(CACHE_ADDL_NAME);
-            addlCache.put("1", "2");
-            addlCache.put(1, 2);
-            addlCache.put(1L, 2L);
-            addlCache.put(TestEnum.A, "Enum_As_Key");
-            addlCache.put("Enum_As_Value", TestEnum.B);
-            addlCache.put(TestEnum.C, TestEnum.C);
-
-            addlCache.put("Serializable", new TestSerializable(42));
-            addlCache.put(new TestSerializable(42), "Serializable_As_Key");
-            addlCache.put("Externalizable", new TestExternalizable(42));
-            addlCache.put(new TestExternalizable(42), "Externalizable_As_Key");
-            addlCache.put(292, new IndexedObject(292));
-
-            final String search1 = "SomeUnexpectedStringValueAsKeyToSearch";
-            ctrlStringsToSearch.add(search1);
-            ctrlStringsForBinaryObjSearch.add(search1);
-            addlCache.put(search1, "SearchKey");
-
-            String search2 = "SomeTestStringContainerToBePrintedLongLine";
-            final TestStringContainerToBePrinted val = new TestStringContainerToBePrinted(search2);
-            ctrlStringsToSearch.add(val.toString()); //will validate original toString() was called
-            ctrlStringsForBinaryObjSearch.add(search2);
-            addlCache.put("SearchValue", val);
-
-            String search3 = "SomeTestStringContainerToBePrintedLongLine2";
-            final TestStringContainerToBePrinted key = new TestStringContainerToBePrinted(search3);
-            ctrlStringsToSearch.add(key.toString()); //will validate original toString() was called
-            ctrlStringsForBinaryObjSearch.add(search3); //validate only string itself
-            addlCache.put(key, "SearchKey");
-
-            cntEntries = addlCache.size();
-            for (Cache.Entry<Object, Object> next : addlCache) {
-                ctrlMap.put(next.getKey(), next.getValue());
-            }
+        final Ignite ignite0 = startGrid("node0");
+        ignite0.active(true);
+
+        final IgniteCache<Object, Object> addlCache = ignite0.getOrCreateCache(CACHE_ADDL_NAME);
+        addlCache.put("1", "2");
+        addlCache.put(1, 2);
+        addlCache.put(1L, 2L);
+        addlCache.put(TestEnum.A, "Enum_As_Key");
+        addlCache.put("Enum_As_Value", TestEnum.B);
+        addlCache.put(TestEnum.C, TestEnum.C);
+
+        addlCache.put("Serializable", new TestSerializable(42));
+        addlCache.put(new TestSerializable(42), "Serializable_As_Key");
+        addlCache.put("Externalizable", new TestExternalizable(42));
+        addlCache.put(new TestExternalizable(42), "Externalizable_As_Key");
+        addlCache.put(292, new IndexedObject(292));
+
+        final String search1 = "SomeUnexpectedStringValueAsKeyToSearch";
+        ctrlStringsToSearch.add(search1);
+        ctrlStringsForBinaryObjSearch.add(search1);
+        addlCache.put(search1, "SearchKey");
+
+        String search2 = "SomeTestStringContainerToBePrintedLongLine";
+        final TestStringContainerToBePrinted val = new TestStringContainerToBePrinted(search2);
+        ctrlStringsToSearch.add(val.toString()); //will validate original toString() was called
+        ctrlStringsForBinaryObjSearch.add(search2);
+        addlCache.put("SearchValue", val);
+
+        String search3 = "SomeTestStringContainerToBePrintedLongLine2";
+        final TestStringContainerToBePrinted key = new TestStringContainerToBePrinted(search3);
+        ctrlStringsToSearch.add(key.toString()); //will validate original toString() was called
+        ctrlStringsForBinaryObjSearch.add(search3); //validate only string itself
+        addlCache.put(key, "SearchKey");
+
+        cntEntries = addlCache.size();
+        for (Cache.Entry<Object, Object> next : addlCache) {
+            ctrlMap.put(next.getKey(), next.getValue());
+        }
 
             for (Cache.Entry<Object, Object> next : addlCache) {
                 ctrlMapForBinaryObjects.put(next.getKey(), next.getValue());
             }
 
-            consistentId = U.maskForFileName(ignite0.cluster().localNode().consistentId().toString());
+        final String subfolderName = genDbSubfolderName(ignite0, 0);
 
-            stopGrid("node0");
-        }
-        else
-            consistentId = "127_0_0_1_47500";
+        stopGrid("node0");
 
         final String workDir = U.defaultWorkDirectory();
 
         final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false);
-        final File binaryMetaWithConsId = new File(binaryMeta, consistentId);
+        final File binaryMetaWithNodeSubfolder = new File(binaryMeta, subfolderName);
         final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false);
 
         final IgniteWalIteratorFactory factory = new IgniteWalIteratorFactory(log, PAGE_SIZE,
-            binaryMetaWithConsId,
+            binaryMetaWithNodeSubfolder,
             marshallerMapping);
         final BiConsumer<Object, Object> objConsumer = new BiConsumer<Object, Object>() {
             @Override public void accept(Object key, Object val) {
@@ -634,7 +629,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
                 }
             }
         };
-        scanIterateAndCount(factory, workDir, consistentId, cntEntries, 0, objConsumer, toStrChecker);
+        scanIterateAndCount(factory, workDir, subfolderName, cntEntries, 0, objConsumer, toStrChecker);
 
         assert ctrlMap.isEmpty() : " Control Map is not empty after reading entries: " + ctrlMap;
         assert ctrlStringsToSearch.isEmpty() : " Control Map for strings in entries is not empty after" +
@@ -642,7 +637,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
 
         //Validate same WAL log with flag binary objects only
         final IgniteWalIteratorFactory keepBinFactory = new IgniteWalIteratorFactory(log, PAGE_SIZE,
-            binaryMetaWithConsId,
+            binaryMetaWithNodeSubfolder,
             marshallerMapping,
             true);
         final BiConsumer<Object, Object> binObjConsumer = new BiConsumer<Object, Object>() {
@@ -693,7 +688,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
             }
         };
 
-        final Consumer<DataRecord> binObjToStringChecker = new Consumer<DataRecord>() {
+        final Consumer<DataRecord> binObjToStrChecker = new Consumer<DataRecord>() {
             @Override public void accept(DataRecord record) {
                 String strRepresentation = record.toString();
                 for (Iterator<String> iter = ctrlStringsForBinaryObjSearch.iterator(); iter.hasNext(); ) {
@@ -705,7 +700,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
                 }
             }
         };
-        scanIterateAndCount(keepBinFactory, workDir, consistentId, cntEntries, 0, binObjConsumer, binObjToStringChecker);
+        scanIterateAndCount(keepBinFactory, workDir, subfolderName, cntEntries, 0, binObjConsumer, binObjToStrChecker);
 
         assert ctrlMapForBinaryObjects.isEmpty() : " Control Map is not empty after reading entries: " + ctrlMapForBinaryObjects;
         assert ctrlStringsForBinaryObjSearch.isEmpty() : " Control Map for strings in entries is not empty after" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
index 4030e53..05636eb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/MockWalIteratorFactory.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.persistence.db.wal.reader;
 
 import java.io.File;
+import java.io.Serializable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -29,6 +30,8 @@ import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.jetbrains.annotations.Nullable;
 import org.mockito.Mockito;
@@ -47,25 +50,34 @@ public class MockWalIteratorFactory {
     private final int pageSize;
 
     /** Consistent node id. */
-    private final String consistentId;
+    private final Serializable consistentId;
+
+    /** DB storage subfolder based node index and consistent node ID. */
+    private String subfolderName;
 
     /** Segments count in work dir. */
     private int segments;
 
+
     /**
      * Creates factory
      * @param log Logger.
      * @param pageSize Page size.
      * @param consistentId Consistent id.
+     * @param subfolderName
      * @param segments Segments.
      */
-    public MockWalIteratorFactory(@Nullable IgniteLogger log, int pageSize, String consistentId, int segments) {
+    public MockWalIteratorFactory(@Nullable IgniteLogger log,
+        int pageSize,
+        Serializable consistentId,
+        String subfolderName,
+        int segments) {
         this.log = log == null ? Mockito.mock(IgniteLogger.class) : log;
         this.pageSize = pageSize;
         this.consistentId = consistentId;
+        this.subfolderName = subfolderName;
         this.segments = segments;
     }
-
     /**
      * Creates iterator
      * @param wal WAL directory without node consistent id
@@ -93,10 +105,13 @@ public class MockWalIteratorFactory {
 
         when(ctx.config()).thenReturn(cfg);
         when(ctx.clientNode()).thenReturn(false);
+        when(ctx.pdsFolderResolver()).thenReturn(new PdsFoldersResolver() {
+            @Override public PdsFolderSettings resolveFolders() {
+                return new PdsFolderSettings(new File("."), subfolderName, consistentId, null, false);
+            }
+        });
 
         final GridDiscoveryManager disco = Mockito.mock(GridDiscoveryManager.class);
-
-        when(disco.consistentId()).thenReturn(consistentId);
         when(ctx.discovery()).thenReturn(disco);
 
         final IgniteWriteAheadLogManager mgr = new FileWriteAheadLogManager(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index 29f7255..d92d848 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageE
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsRebalancingOnNotStableTopologyTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsTransactionsHangTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsWholeClusterRestartTest;
+import org.apache.ignite.internal.processors.cache.persistence.db.filename.IgniteUidAsConsistentIdMigrationTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalFlushFailoverTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalHistoryReservationsTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalSerializerVersionTest;
@@ -78,6 +79,9 @@ public class IgnitePdsTestSuite2 extends TestSuite {
 
         suite.addTestSuite(IgnitePdsExchangeDuringCheckpointTest.class);
 
+        // new style folders with generated consistent ID test
+        suite.addTestSuite(IgniteUidAsConsistentIdMigrationTest.class);
+
         suite.addTestSuite(IgniteWalSerializerVersionTest.class);
 
         return suite;


[21/25] ignite git commit: ignite-5937

Posted by sb...@apache.org.
ignite-5937


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

Branch: refs/heads/ignite-5937
Commit: c4f98f367ab24ca22ab8b4f3f1213f365846210f
Parents: d4d2c38
Author: sboikov <sb...@gridgain.com>
Authored: Wed Oct 4 11:32:02 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Oct 4 11:32:02 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/tree/BPlusTree.java       | 31 +++++++++++++++-----
 1 file changed, 23 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4f98f36/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index 5836340..05adb41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -62,6 +62,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.Bool.DONE;
 import static org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree.Bool.FALSE;
@@ -4447,21 +4448,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         abstract void init0();
 
         /**
-         * @param pageAddr
-         * @param io
-         * @param startIdx
-         * @param cnt
-         * @return
+         * @param pageAddr Page address.
+         * @param io IO.
+         * @param startIdx Start index.
+         * @param cnt Number of rows in the buffer.
+         * @return {@code true} If we were able to fetch rows from this page.
          * @throws IgniteCheckedException If failed.
          */
-        abstract boolean fillFromBuffer0(long pageAddr, BPlusIO<L> io, int startIdx, int cnt) throws IgniteCheckedException;
+        abstract boolean fillFromBuffer0(long pageAddr, BPlusIO<L> io, int startIdx, int cnt)
+            throws IgniteCheckedException;
 
         /**
-         * @return
+         * @return {@code True} If we have rows to return after reading the next page.
          * @throws IgniteCheckedException If failed.
          */
         abstract boolean reinitialize0() throws IgniteCheckedException;
 
+        /**
+         * @param readDone {@code True} if traversed all rows.
+         */
         abstract void onNotFound(boolean readDone);
 
         /**
@@ -4584,6 +4589,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
+         * @param lastRow Last read row (to be used as new lower bound).
          * @return {@code true} If we have rows to return after reading the next page.
          * @throws IgniteCheckedException If failed.
          */
@@ -4655,9 +4661,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param lowerBound Lower bound.
          * @param upperBound Upper bound.
+         * @param p Row predicate.
          * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row.
          */
-        FindOneCursor(L lowerBound, L upperBound, RowPredicate<L, T> p, Object x) {
+        FindOneCursor(L lowerBound, L upperBound, @Nullable RowPredicate<L, T> p, Object x) {
             super(lowerBound, upperBound, x);
 
             assert lowerBound != null;
@@ -4666,10 +4673,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             this.p = p;
         }
 
+        /** {@inheritDoc} */
         @Override void init0() {
             // No-op.
         }
 
+        /** {@inheritDoc} */
         @Override boolean fillFromBuffer0(long pageAddr, BPlusIO<L> io, int startIdx, int cnt) throws IgniteCheckedException {
             if (startIdx == 0) // TODO IGNITE-3478: startIdx == 0? can search twice for first item?
                 startIdx = findLowerBound(pageAddr, io, cnt);
@@ -4699,10 +4708,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             return true;
         }
 
+        /** {@inheritDoc} */
         @Override boolean reinitialize0() throws IgniteCheckedException {
             return true;
         }
 
+        /** {@inheritDoc} */
         @Override void onNotFound(boolean readDone) {
             resRow = EMPTY;
         }
@@ -4758,6 +4769,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             super(lowerBound, upperBound, x);
         }
 
+        /** {@inheritDoc} */
         @Override boolean fillFromBuffer0(long pageAddr, BPlusIO<L> io, int startIdx, int cnt) throws IgniteCheckedException {
             if (lowerBound != null && startIdx == 0)
                 startIdx = findLowerBound(pageAddr, io, cnt);
@@ -4784,10 +4796,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             return true;
         }
 
+        /** {@inheritDoc} */
         @Override boolean reinitialize0() throws IgniteCheckedException {
             return next();
         }
 
+        /** {@inheritDoc} */
         @Override void onNotFound(boolean readDone) {
             if (readDone)
                 rows = null;
@@ -4801,6 +4815,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             }
         }
 
+        /** {@inheritDoc} */
         @Override void init0() {
             row = -1;
         }


[16/25] ignite git commit: ignite-3478

Posted by sb...@apache.org.
ignite-3478


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

Branch: refs/heads/ignite-5937
Commit: 07a56028a2150208c79e64c280dc989624c47c47
Parents: 27b2be4 db69e90
Author: sboikov <sb...@gridgain.com>
Authored: Tue Oct 3 17:34:20 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Oct 3 17:34:20 2017 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     |    46 +
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |    88 +-
 .../apache/ignite/configuration/WALMode.java    |    14 +-
 .../internal/binary/BinaryClassDescriptor.java  |     4 +-
 .../affinity/GridAffinityAssignmentCache.java   |     2 +-
 .../cache/GridCacheAffinityManager.java         |     2 +-
 .../processors/cache/GridCacheUtils.java        |     6 +-
 .../dht/GridPartitionedGetFuture.java           |    10 +-
 .../dht/GridPartitionedSingleGetFuture.java     |     6 +-
 .../distributed/near/GridNearGetFuture.java     |     6 +-
 .../wal/FileWriteAheadLogManager.java           |     5 +-
 .../datastreamer/PlatformDataStreamer.java      |    14 +
 .../processors/query/GridQueryProcessor.java    |     6 +-
 .../internal/processors/query/QueryUtils.java   |    17 +
 .../processors/rest/GridRestCommand.java        |     2 +-
 .../processors/rest/GridRestProcessor.java      |     5 +-
 .../ignite/internal/util/IgniteUtils.java       |    43 +-
 .../cache/VisorCacheAggregatedMetrics.java      |    24 +-
 .../binary/BinaryMarshallerSelfTest.java        |   136 +-
 .../processors/query/h2/IgniteH2Indexing.java   |     3 +
 .../query/h2/ddl/DdlStatementsProcessor.java    |    15 +-
 .../query/IgniteSqlNotNullConstraintTest.java   |   179 +-
 .../query/IgniteSqlParameterizedQueryTest.java  |   392 +
 .../IgniteCacheQuerySelfTestSuite.java          |     3 +-
 modules/platforms/.gitignore                    |     3 +-
 .../Cache/CacheConfigurationTest.cs             |     2 +-
 .../Dataload/DataStreamerTest.cs                |    50 +-
 .../Apache.Ignite.Core.csproj                   |     1 +
 .../Datastream/DataStreamerDefaults.cs          |    46 +
 .../Datastream/IDataStreamer.cs                 |    21 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |     2 +
 .../Impl/Binary/BinaryProcessorClient.cs        |     4 +-
 .../Impl/Binary/BinaryReaderExtensions.cs       |    10 +-
 .../Impl/Binary/BinaryUtils.cs                  |    14 +
 .../Impl/Cache/Query/QueryCursorBase.cs         |     3 +
 .../Client/Cache/Query/ClientQueryCursor.cs     |     2 +
 .../Impl/Client/ClientSocket.cs                 |     2 +
 .../Impl/Datastream/DataStreamerImpl.cs         |    45 +-
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |     2 +-
 .../Impl/PlatformJniTarget.cs                   |     6 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |     3 +-
 modules/platforms/dotnet/Apache.Ignite.ndproj   | 11139 +++++++++++++++++
 .../http/jetty/GridJettyRestHandler.java        |    18 +
 .../rest/protocols/http/jetty/favicon.ico       |   Bin 1406 -> 1150 bytes
 .../commands/cache/VisorCacheCommand.scala      |    38 +-
 parent/pom.xml                                  |     1 +
 46 files changed, 12297 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/07a56028/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/07a56028/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------


[20/25] ignite git commit: IGNITE-6285 Enhance persistent store paths handling - Fixes #2775.

Posted by sb...@apache.org.
IGNITE-6285 Enhance persistent store paths handling - Fixes #2775.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-5937
Commit: 62f3c4c52f5aad82e51e93b7981e670090cddd4c
Parents: 855fe4b
Author: dpavlov <dp...@gridgain.com>
Authored: Wed Oct 4 09:46:16 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Oct 4 09:46:16 2017 +0300

----------------------------------------------------------------------
 .../FoldersReuseCompatibilityTest.java          | 224 ++++++
 ...itePersistenceCompatibilityAbstractTest.java |   3 +
 .../IgniteCompatibilityBasicTestSuite.java      |   3 +
 .../apache/ignite/IgniteSystemProperties.java   |   7 +
 .../ignite/internal/GridKernalContext.java      |   6 +
 .../ignite/internal/GridKernalContextImpl.java  |  17 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../discovery/GridDiscoveryManager.java         |  55 +-
 .../cache/binary/BinaryMetadataFileStore.java   |   6 +-
 .../GridCacheDatabaseSharedManager.java         |  91 ++-
 .../IgniteCacheDatabaseSharedManager.java       |  13 +-
 .../persistence/file/FilePageStoreManager.java  |  29 +-
 .../filename/PdsConsistentIdProcessor.java      | 568 +++++++++++++++
 .../persistence/filename/PdsFolderSettings.java | 138 ++++
 .../filename/PdsFoldersResolver.java            |  33 +
 .../wal/FileWriteAheadLogManager.java           |  20 +-
 .../wal/reader/StandaloneGridKernalContext.java |  32 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   4 +-
 .../tcp/internal/TcpDiscoveryNode.java          |  15 +
 .../IgniteUidAsConsistentIdMigrationTest.java   | 712 +++++++++++++++++++
 .../db/wal/IgniteWalRecoveryTest.java           |  13 +-
 .../db/wal/reader/IgniteWalReaderTest.java      | 187 +++--
 .../db/wal/reader/MockWalIteratorFactory.java   |  25 +-
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   4 +
 24 files changed, 1995 insertions(+), 214 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java
new file mode 100644
index 0000000..1775013
--- /dev/null
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/FoldersReuseCompatibilityTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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.compatibility.persistence;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.parseSubFolderName;
+
+/**
+ * Test for new and old style persistent storage folders generation and compatible startup of current ignite version
+ */
+public class FoldersReuseCompatibilityTest extends IgnitePersistenceCompatibilityAbstractTest {
+    /** Cache name for test. */
+    private static final String CACHE_NAME = "dummy";
+
+    /** Key to store in previous version of ignite */
+    private static final String KEY = "ObjectFromPast";
+
+    /** Value to store in previous version of ignite */
+    private static final String VAL = "ValueFromPast";
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        // No-op. super.afterTest();
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        configPersistence(cfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test startup of current ignite version using DB storage folder from previous version of Ignite. Expected to start
+     * successfully with existing DB
+     *
+     * @throws Exception if failed.
+     */
+    public void testFoldersReuseCompatibility_2_2() throws Exception {
+        runFoldersReuse("2.2.0");
+    }
+
+    /**
+     * Test startup of current ignite version using DB storage folder from previous version of Ignite. Expected to start
+     * successfully with existing DB
+     *
+     * @throws Exception if failed.
+     */
+    public void testFoldersReuseCompatibility_2_1() throws Exception {
+        runFoldersReuse("2.1.0");
+    }
+
+    /**
+     * Test startup of current ignite version using DB storage folder from previous version of Ignite. Expected to start
+     * successfully with existing DB
+     *
+     * @param ver 3 digit Ignite version to check compatibility with
+     * @throws Exception if failed.
+     */
+    private void runFoldersReuse(String ver) throws Exception {
+        final IgniteEx grid = startGrid(1, ver, new ConfigurationClosure(), new PostStartupClosure());
+
+        grid.close();
+        stopAllGrids();
+
+        IgniteEx ignite = startGrid(0);
+
+        ignite.active(true);
+        ignite.getOrCreateCache("cache2createdForNewGrid").put("Object", "Value");
+        assertEquals(1, ignite.context().discovery().topologyVersion());
+
+        assertPdsDirsDefaultExist(U.maskForFileName(ignite.cluster().node().consistentId().toString()));
+
+        assertEquals(VAL, ignite.cache(CACHE_NAME).get(KEY));
+
+        assertNodeIndexesInFolder();// should not create any new style directories
+
+        stopAllGrids();
+    }
+
+    /** Started node test actions closure. */
+    private static class PostStartupClosure implements IgniteInClosure<Ignite> {
+        /** {@inheritDoc} */
+        @Override public void apply(Ignite ignite) {
+            ignite.active(true);
+            ignite.getOrCreateCache(CACHE_NAME).put(KEY, VAL);
+        }
+    }
+
+    /** Setup compatible node closure. */
+    private static class ConfigurationClosure implements IgniteInClosure<IgniteConfiguration> {
+        /** {@inheritDoc} */
+        @Override public void apply(IgniteConfiguration cfg) {
+            cfg.setLocalHost("127.0.0.1");
+            TcpDiscoverySpi disco = new TcpDiscoverySpi();
+            disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER);
+
+            cfg.setDiscoverySpi(disco);
+
+            configPersistence(cfg);
+        }
+    }
+
+    /**
+     * Setup persistence for compatible and current version node.
+     *
+     * @param cfg ignite config to setup.
+     */
+    private static void configPersistence(IgniteConfiguration cfg) {
+        final PersistentStoreConfiguration psCfg = new PersistentStoreConfiguration();
+
+        cfg.setPersistentStoreConfiguration(psCfg);
+
+        final MemoryConfiguration memCfg = new MemoryConfiguration();
+        final MemoryPolicyConfiguration memPolCfg = new MemoryPolicyConfiguration();
+
+        memPolCfg.setMaxSize(32 * 1024 * 1024); // we don't need much memory for this test
+        memCfg.setMemoryPolicies(memPolCfg);
+        cfg.setMemoryConfiguration(memCfg);
+    }
+
+    /**
+     * @param indexes expected new style node indexes in folders
+     * @throws IgniteCheckedException if failed
+     */
+    private void assertNodeIndexesInFolder(Integer... indexes) throws IgniteCheckedException {
+        assertEquals(new TreeSet<>(Arrays.asList(indexes)), getAllNodeIndexesInFolder());
+    }
+
+    /**
+     * @return set of all indexes of nodes found in work folder
+     * @throws IgniteCheckedException if failed.
+     */
+    @NotNull private Set<Integer> getAllNodeIndexesInFolder() throws IgniteCheckedException {
+        final File curFolder = new File(U.defaultWorkDirectory(), PdsConsistentIdProcessor.DB_DEFAULT_FOLDER);
+        final Set<Integer> indexes = new TreeSet<>();
+        final File[] files = curFolder.listFiles(PdsConsistentIdProcessor.DB_SUBFOLDERS_NEW_STYLE_FILTER);
+
+        for (File file : files) {
+            final PdsConsistentIdProcessor.FolderCandidate uid
+                = parseSubFolderName(file, log);
+
+            if (uid != null)
+                indexes.add(uid.nodeIndex());
+        }
+
+        return indexes;
+    }
+
+    /**
+     * Checks existence of all storage-related directories
+     *
+     * @param subDirName sub directories name expected
+     * @throws IgniteCheckedException if IO error occur
+     */
+    private void assertPdsDirsDefaultExist(String subDirName) throws IgniteCheckedException {
+        assertDirectoryExist("binary_meta", subDirName);
+        assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_STORE_PATH, subDirName);
+        assertDirectoryExist(PersistentStoreConfiguration.DFLT_WAL_ARCHIVE_PATH, subDirName);
+        assertDirectoryExist(PdsConsistentIdProcessor.DB_DEFAULT_FOLDER, subDirName);
+    }
+
+    /**
+     * Checks one folder existence
+     *
+     * @param subFolderNames subfolders array to touch
+     * @throws IgniteCheckedException if IO error occur
+     */
+    private void assertDirectoryExist(String... subFolderNames) throws IgniteCheckedException {
+        File curFolder = new File(U.defaultWorkDirectory());
+
+        for (String name : subFolderNames) {
+            curFolder = new File(curFolder, name);
+        }
+
+        final String path;
+        try {
+            path = curFolder.getCanonicalPath();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to convert path: [" + curFolder.getAbsolutePath() + "]", e);
+        }
+
+        assertTrue("Directory " + Arrays.asList(subFolderNames).toString()
+            + " is expected to exist [" + path + "]", curFolder.exists() && curFolder.isDirectory());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
index d76b862..f39b6f6 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePersistenceCompatibilityAbstractTest.java
@@ -45,6 +45,9 @@ public abstract class IgnitePersistenceCompatibilityAbstractTest extends IgniteC
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
+        //protection if test failed to finish, e.g. by error
+        stopAllGrids();
+
         assert deleteDefaultDBWorkDirectory() : "Couldn't delete DB work directory.";
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
index b54b396..351a0e7 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
@@ -19,6 +19,7 @@ package org.apache.ignite.compatibility.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.compatibility.persistence.DummyPersistenceCompatibilityTest;
+import org.apache.ignite.compatibility.persistence.FoldersReuseCompatibilityTest;
 
 /**
  * Compatibility tests basic test suite.
@@ -33,6 +34,8 @@ public class IgniteCompatibilityBasicTestSuite {
 
         suite.addTestSuite(DummyPersistenceCompatibilityTest.class);
 
+        suite.addTestSuite(FoldersReuseCompatibilityTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/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 e1e72f7..d7b4de9 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -737,6 +737,13 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_WAL_SERIALIZER_VERSION = "IGNITE_WAL_SERIALIZER_VERSION";
 
     /**
+     * When set to {@code true}, Data store folders are generated only by consistent id, and no consistent ID will be
+     * set based on existing data store folders. This option also enables compatible folder generation mode as it was
+     * before 2.3.
+     */
+    public static final String IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID = "IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 99c7cce..210b401 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
@@ -643,4 +644,9 @@ public interface GridKernalContext extends Iterable<GridComponent> {
      * @return Platform processor.
      */
     public PlatformProcessor platform();
+
+    /**
+     * @return PDS mode folder name resolver, also generates consistent ID in case new folder naming is used
+     */
+    public PdsFoldersResolver pdsFolderResolver();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 07e5970..1f0292c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -49,6 +49,7 @@ import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
@@ -377,6 +378,9 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     /** */
     private volatile boolean disconnected;
 
+    /** PDS mode folder name resolver, also generates consistent ID in case new folder naming is used */
+    private PdsFoldersResolver pdsFolderRslvr;
+
     /**
      * No-arg constructor is required by externalization.
      */
@@ -536,7 +540,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         else if (comp instanceof GridCacheProcessor)
             cacheProc = (GridCacheProcessor)comp;
         else if (comp instanceof GridClusterStateProcessor)
-            stateProc =(GridClusterStateProcessor)comp;
+            stateProc = (GridClusterStateProcessor)comp;
         else if (comp instanceof GridTaskSessionProcessor)
             sesProc = (GridTaskSessionProcessor)comp;
         else if (comp instanceof GridPortProcessor)
@@ -576,9 +580,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         else if (comp instanceof PlatformProcessor)
             platformProc = (PlatformProcessor)comp;
         else if (comp instanceof PoolProcessor)
-            poolProc = (PoolProcessor) comp;
+            poolProc = (PoolProcessor)comp;
         else if (comp instanceof GridMarshallerMappingProcessor)
             mappingProc = (GridMarshallerMappingProcessor)comp;
+        else if (comp instanceof PdsFoldersResolver)
+            pdsFolderRslvr = (PdsFoldersResolver)comp;
         else if (!(comp instanceof DiscoveryNodeValidationProcessor
                 || comp instanceof PlatformPluginProcessor))
             assert (comp instanceof GridPluginComponent) : "Unknown manager class: " + comp.getClass();
@@ -1005,7 +1011,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    public Map<String, ? extends ExecutorService> customExecutors() {
+    @Override public Map<String, ? extends ExecutorService> customExecutors() {
         return customExecSvcs;
     }
 
@@ -1069,6 +1075,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         this.disconnected = disconnected;
     }
 
+    /**{@inheritDoc}*/
+    @Override public PdsFoldersResolver pdsFolderResolver() {
+        return pdsFolderRslvr;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridKernalContextImpl.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/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 f5d736a..759bf64 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
@@ -115,6 +115,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
 import org.apache.ignite.internal.processors.cache.persistence.MemoryPolicy;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
@@ -930,13 +931,14 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             // Assign discovery manager to context before other processors start so they
             // are able to register custom event listener.
-            GridManager discoMgr = new GridDiscoveryManager(ctx);
+            final GridManager discoMgr = new GridDiscoveryManager(ctx);
 
             ctx.add(discoMgr, false);
 
             // Start processors before discovery manager, so they will
             // be able to start receiving messages once discovery completes.
             try {
+                startProcessor(new PdsConsistentIdProcessor(ctx));
                 startProcessor(createComponent(DiscoveryNodeValidationProcessor.class, ctx));
                 startProcessor(new  GridAffinityProcessor(ctx));
                 startProcessor(createComponent(GridSegmentationProcessor.class, ctx));

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/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 527399d..14485d2 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.managers.discovery;
 
+import java.io.Serializable;
 import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
@@ -116,6 +117,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiListener;
 import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
 import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -281,12 +283,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** */
     private final CountDownLatch startLatch = new CountDownLatch(1);
 
-    /** */
-    private Object consistentId;
-
     /** Discovery spi registered flag. */
     private boolean registeredDiscoSpi;
 
+    /** Local node compatibility consistent ID. */
+    private Serializable consistentId;
+
     /** @param ctx Context. */
     public GridDiscoveryManager(GridKernalContext ctx) {
         super(ctx, ctx.config().getDiscoverySpi());
@@ -549,6 +551,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             @Override public void onLocalNodeInitialized(ClusterNode locNode) {
                 for (IgniteInClosure<ClusterNode> lsnr : locNodeInitLsnrs)
                     lsnr.apply(locNode);
+
+                if (locNode instanceof TcpDiscoveryNode) {
+                    final TcpDiscoveryNode node = (TcpDiscoveryNode)locNode;
+
+                    if (consistentId != null)
+                        node.setConsistentId(consistentId);
+                }
             }
 
             @Override public void onDiscovery(
@@ -2017,22 +2026,42 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @return Consistent ID.
+     * @deprecated Use PdsConsistentIdProcessor to get actual consistent ID
      */
-    public Object consistentId() {
-        if (consistentId == null) {
-            try {
-                inject();
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException("Failed to init consistent ID.", e);
-            }
+    @Deprecated
+    public Serializable consistentId() {
+        if (consistentId == null)
+            consistentId = getInjectedDiscoverySpi().consistentId();
+
+        return consistentId;
+    }
 
-            consistentId = getSpi().consistentId();
+    /**
+     * Performs injection of discovery SPI if needed, then provides DiscoverySpi SPI.
+     * Manual injection is required because normal startup of SPI is done after processor started.
+     *
+     * @return Wrapped DiscoverySpi SPI.
+     */
+    private DiscoverySpi getInjectedDiscoverySpi() {
+        try {
+            inject();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to init consistent ID.", e);
         }
+        return getSpi();
+    }
 
-        return consistentId;
+    /**
+     * Sets TCP local node consistent ID. This setter is to be called before node init in SPI
+     *
+     * @param consistentId New value of consistent ID to be used in local node initialization
+     */
+    public void consistentId(final Serializable consistentId) {
+        this.consistentId = consistentId;
     }
 
+
     /** @return Topology version. */
     public long topologyVersion() {
         return topSnap.get().topVer.topologyVersion();

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java
index 2d4114f..420cde5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java
@@ -34,7 +34,7 @@ import org.jetbrains.annotations.Nullable;
  * which may lead to segmentation of nodes from cluster.
  */
 class BinaryMetadataFileStore {
-    /** */
+    /** Link to resolved binary metadata directory. Null for non persistent mode */
     private File workDir;
 
     /** */
@@ -68,14 +68,14 @@ class BinaryMetadataFileStore {
         if (binaryMetadataFileStoreDir != null)
             workDir = binaryMetadataFileStoreDir;
         else {
-            String consId = U.maskForFileName(ctx.discovery().consistentId().toString());
+            final String subFolder = ctx.pdsFolderResolver().resolveFolders().folderName();
 
             workDir = new File(U.resolveWorkDirectory(
                 ctx.config().getWorkDirectory(),
                 "binary_meta",
                 false
             ),
-                consId);
+                subFolder);
         }
 
         U.ensureDirectory(workDir, "directory for serialized binary metadata", log);

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/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 9f1ccb4..2d89942 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
@@ -62,6 +62,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.PersistenceMetrics;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CheckpointWriteOrder;
 import org.apache.ignite.configuration.DataPageEvictionMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -74,6 +75,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
@@ -280,8 +282,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** */
     private final ConcurrentMap<Integer, IgniteInternalFuture> idxRebuildFuts = new ConcurrentHashMap<>();
 
-    /** Lock holder. */
-    private FileLockHolder fileLockHolder;
+    /**
+     * Lock holder for compatible folders mode. Null if lock holder was created at start node. <br>
+     * In this case lock is held on PDS resover manager and it is not required to manage locking here
+     */
+    @Nullable private FileLockHolder fileLockHolder;
 
     /** Lock wait time. */
     private final long lockWaitTime;
@@ -367,7 +372,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         snapshotMgr = cctx.snapshot();
 
-        if (!cctx.kernalContext().clientNode()) {
+        final GridKernalContext kernalCtx = cctx.kernalContext();
+
+        if (!kernalCtx.clientNode()) {
             IgnitePageStoreManager store = cctx.pageStore();
 
             assert store instanceof FilePageStoreManager : "Invalid page store manager was created: " + store;
@@ -379,7 +386,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             if (!U.mkdirs(cpDir))
                 throw new IgniteCheckedException("Could not create directory for checkpoint metadata: " + cpDir);
 
-            fileLockHolder = new FileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log);
+            final FileLockHolder preLocked = kernalCtx.pdsFolderResolver()
+                .resolveFolders()
+                .getLockedFileLockHolder();
+            if (preLocked == null)
+                fileLockHolder = new FileLockHolder(storeMgr.workDir().getPath(), kernalCtx, log);
 
             persStoreMetrics.wal(cctx.wal());
         }
@@ -488,8 +499,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         /* Must be here, because after deactivate we can invoke activate and file lock must be already configured */
         stopping = false;
 
-        if (!cctx.localNode().isClient())
-            fileLockHolder = new FileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log);
+        if (!cctx.localNode().isClient()) {
+            //we replace lock with new instance (only if we're responsible for locking folders)
+            if (fileLockHolder != null)
+                fileLockHolder = new FileLockHolder(storeMgr.workDir().getPath(), cctx.kernalContext(), log);
+        }
     }
 
     /**
@@ -592,20 +606,24 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /** {@inheritDoc} */
     @Override public void lock() throws IgniteCheckedException {
-        if (log.isDebugEnabled())
-            log.debug("Try to capture file lock [nodeId=" +
-                cctx.localNodeId() + " path=" + fileLockHolder.lockPath() + "]");
+        if (fileLockHolder != null) {
+            if (log.isDebugEnabled())
+                log.debug("Try to capture file lock [nodeId=" +
+                    cctx.localNodeId() + " path=" + fileLockHolder.lockPath() + "]");
 
-        fileLockHolder.tryLock(lockWaitTime);
+            fileLockHolder.tryLock(lockWaitTime);
+        }
     }
 
     /** {@inheritDoc} */
     @Override public void unLock() {
-        if (log.isDebugEnabled())
-            log.debug("Release file lock [nodeId=" +
-                cctx.localNodeId() + " path=" + fileLockHolder.lockPath() + "]");
+        if (fileLockHolder != null) {
+            if (log.isDebugEnabled())
+                log.debug("Release file lock [nodeId=" +
+                    cctx.localNodeId() + " path=" + fileLockHolder.lockPath() + "]");
 
-        fileLockHolder.release();
+            fileLockHolder.release();
+        }
     }
 
     /** {@inheritDoc} */
@@ -628,7 +646,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         if (!cctx.kernalContext().clientNode()) {
             unLock();
 
-            fileLockHolder.close();
+            if (fileLockHolder != null)
+                fileLockHolder.close();
         }
 
         unRegistrateMetricsMBean();
@@ -875,7 +894,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             idxRebuildFuts.remove(cacheId, rebuildFut);
 
                             log().info("Finished indexes rebuilding for cache: [name=" + cacheCtx.config().getName()
-                                    + ", grpName=" + cacheCtx.config().getGroupName());
+                                + ", grpName=" + cacheCtx.config().getGroupName());
                         }
                     });
                 }
@@ -1189,7 +1208,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /**
      * For debugging only. TODO: remove.
-     *
      */
     public Map<T2<Integer, Integer>, T2<Long, WALPointer>> reservedForPreloading() {
         return reservedForPreloading;
@@ -2310,7 +2328,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     cpRec.addCacheGroupState(grp.groupId(), state);
                 }
 
-
                 cpPagesTuple = beginAllCheckpoints();
 
                 hasPages = hasPageForWrite(cpPagesTuple.get1());
@@ -2400,7 +2417,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
 
         /**
-         * @return tuple with collections of FullPageIds obtained from each PageMemory and overall number of dirty pages.
+         * @return tuple with collections of FullPageIds obtained from each PageMemory and overall number of dirty
+         * pages.
          */
         private IgniteBiTuple<Collection<GridMultiCollectionWrapper<FullPageId>>, Integer> beginAllCheckpoints() {
             Collection<GridMultiCollectionWrapper<FullPageId>> res = new ArrayList(memoryPolicies().size());
@@ -2792,9 +2810,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     }
 
     /**
-     * Checkpoint history. Holds chronological ordered map with {@link GridCacheDatabaseSharedManager.CheckpointEntry CheckpointEntries}.
-     * Data is loaded from corresponding checkpoint directory.
-     * This directory holds files for checkpoint start and end.
+     * Checkpoint history. Holds chronological ordered map with {@link GridCacheDatabaseSharedManager.CheckpointEntry
+     * CheckpointEntries}. Data is loaded from corresponding checkpoint directory. This directory holds files for
+     * checkpoint start and end.
      */
     @SuppressWarnings("PublicInnerClass")
     public class CheckpointHistory {
@@ -3117,7 +3135,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /**
      *
      */
-    private static class FileLockHolder {
+    public static class FileLockHolder implements AutoCloseable {
         /** Lock file name. */
         private static final String lockFileName = "lock";
 
@@ -3130,8 +3148,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         /** Lock. */
         private FileLock lock;
 
-        /** Id. */
-        private GridKernalContext ctx;
+        /** Kernal context to generate Id of locked node in file. */
+        @NotNull private GridKernalContext ctx;
 
         /** Logger. */
         private IgniteLogger log;
@@ -3139,7 +3157,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         /**
          * @param path Path.
          */
-        private FileLockHolder(String path, GridKernalContext ctx, IgniteLogger log) {
+        public FileLockHolder(String path, @NotNull GridKernalContext ctx, IgniteLogger log) {
             try {
                 file = Paths.get(path, lockFileName).toFile();
 
@@ -3168,7 +3186,14 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             sb.a("[").a(ctx.localNodeId().toString()).a("]");
 
             //write ip addresses
-            sb.a(ctx.discovery().localNode().addresses());
+            final GridDiscoveryManager discovery = ctx.discovery();
+
+            if (discovery != null) { //discovery may be not up and running
+                final ClusterNode node = discovery.localNode();
+
+                if (node != null)
+                    sb.a(node.addresses());
+            }
 
             //write ports
             sb.a("[");
@@ -3264,17 +3289,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             return content;
         }
 
-        /**
-         *
-         */
-        private void release() {
+        /** Releases file lock */
+        public void release() {
             U.releaseQuiet(lock);
         }
 
-        /**
-         *
-         */
-        private void close() {
+        /** Closes file channel */
+        public void close() {
             U.closeQuiet(lockFile);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/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 6ea6eff..d7682f0 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
@@ -50,6 +50,7 @@ import org.apache.ignite.internal.processors.cache.persistence.evict.NoOpPageEvi
 import org.apache.ignite.internal.processors.cache.persistence.evict.PageEvictionTracker;
 import org.apache.ignite.internal.processors.cache.persistence.evict.Random2LruPageEvictionTracker;
 import org.apache.ignite.internal.processors.cache.persistence.evict.RandomLruPageEvictionTracker;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeListImpl;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
@@ -939,17 +940,21 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
      *
      * @throws IgniteCheckedException If resolving swap directory fails.
      */
-    @Nullable protected File buildAllocPath(MemoryPolicyConfiguration plc) throws IgniteCheckedException {
+    @Nullable private File buildAllocPath(MemoryPolicyConfiguration plc) throws IgniteCheckedException {
         String path = plc.getSwapFilePath();
 
         if (path == null)
             return null;
 
-        String consId = String.valueOf(cctx.discovery().consistentId());
+        final PdsFolderSettings folderSettings = cctx.kernalContext().pdsFolderResolver().resolveFolders();
+        final String folderName;
 
-        consId = consId.replaceAll("[:,\\.]", "_");
+        if(folderSettings.isCompatible())
+            folderName = String.valueOf(folderSettings.consistentId()).replaceAll("[:,\\.]", "_");
+        else
+            folderName = folderSettings.folderName();
 
-        return buildPath(path, consId);
+        return buildPath(path, folderName);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/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 a20bda1..ed82127 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
@@ -47,6 +47,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
@@ -93,7 +94,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     /** */
     private PersistentStoreConfiguration pstCfg;
 
-    /** Absolute directory for file page store */
+    /** Absolute directory for file page store. Includes consistent id based folder. */
     private File storeWorkDir;
 
     /** */
@@ -117,29 +118,13 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
 
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
-        if (cctx.kernalContext().clientNode())
+        final GridKernalContext ctx = cctx.kernalContext();
+        if (ctx.clientNode())
             return;
 
-        String consId = U.maskForFileName(cctx.kernalContext().discovery().consistentId().toString());
-
-        if (pstCfg.getPersistentStorePath() != null) {
-            File workDir0 = new File(pstCfg.getPersistentStorePath());
-
-            if (!workDir0.isAbsolute())
-                workDir0 = U.resolveWorkDirectory(
-                    igniteCfg.getWorkDirectory(),
-                    pstCfg.getPersistentStorePath(),
-                    false
-                );
+        final PdsFolderSettings folderSettings = ctx.pdsFolderResolver().resolveFolders();
 
-            storeWorkDir = new File(workDir0, consId);
-        }
-        else
-            storeWorkDir = new File(U.resolveWorkDirectory(
-                igniteCfg.getWorkDirectory(),
-                DFLT_STORE_DIR,
-                false
-            ), consId);
+        storeWorkDir = new File(folderSettings.persistentStoreRootPath(), folderSettings.folderName());
 
         U.ensureDirectory(storeWorkDir, "page store work directory", log);
     }
@@ -551,7 +536,7 @@ public class FilePageStoreManager extends GridCacheSharedManagerAdapter implemen
     }
 
     /**
-     * @return Store work dir.
+     * @return Store work dir. Includes consistent-id based folder
      */
     public File workDir() {
         return storeWorkDir;

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
new file mode 100644
index 0000000..c73a952
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsConsistentIdProcessor.java
@@ -0,0 +1,568 @@
+/*
+ * 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.filename;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+import java.io.Serializable;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID;
+import static org.apache.ignite.IgniteSystemProperties.getBoolean;
+
+/**
+ * Component for resolving PDS storage file names, also used for generating consistent ID for case PDS mode is enabled
+ */
+public class PdsConsistentIdProcessor extends GridProcessorAdapter implements PdsFoldersResolver {
+    /** Database subfolders constant prefix. */
+    private static final String DB_FOLDER_PREFIX = "node";
+
+    /** Node index and uid separator in subfolders name. */
+    private static final String NODEIDX_UID_SEPARATOR = "-";
+
+    /** Constant node subfolder prefix and node index pattern (nodeII, where II - node index as decimal integer) */
+    private static final String NODE_PATTERN = DB_FOLDER_PREFIX + "[0-9]*" + NODEIDX_UID_SEPARATOR;
+
+    /** Uuid as string pattern. */
+    private static final String UUID_STR_PATTERN = "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
+
+    /**
+     * Subdir (nodeII-UID, where II - node index as decimal integer, UID - string representation of consistent ID)
+     * pattern.
+     */
+    private static final String SUBDIR_PATTERN = NODE_PATTERN + UUID_STR_PATTERN;
+
+    /** Database subfolders for new style filter. */
+    public static final FileFilter DB_SUBFOLDERS_NEW_STYLE_FILTER = new FileFilter() {
+        @Override public boolean accept(File pathname) {
+            return pathname.isDirectory() && pathname.getName().matches(SUBDIR_PATTERN);
+        }
+    };
+
+    /** Database subfolders for old style filter. */
+    private static final FileFilter DB_SUBFOLDERS_OLD_STYLE_FILTER = new FileFilter() {
+        @Override public boolean accept(File pathname) {
+            return pathname.isDirectory()
+                && !"wal".equals(pathname.getName())
+                && !pathname.getName().matches(SUBDIR_PATTERN);
+        }
+    };
+
+    /** Database default folder. */
+    public static final String DB_DEFAULT_FOLDER = "db";
+
+    /** Config. */
+    private IgniteConfiguration cfg;
+
+    /** Logger. */
+    private IgniteLogger log;
+
+    /** Context. */
+    private GridKernalContext ctx;
+
+    /** Cached folder settings. */
+    private PdsFolderSettings settings;
+
+    /**
+     * Creates folders resolver
+     *
+     * @param ctx Context.
+     */
+    public PdsConsistentIdProcessor(final GridKernalContext ctx) {
+        super(ctx);
+
+        this.cfg = ctx.config();
+        this.log = ctx.log(PdsFoldersResolver.class);
+        this.ctx = ctx;
+    }
+
+    /**
+     * Prepares compatible PDS folder settings. No locking is performed, consistent ID is not overridden.
+     *
+     * @param pstStoreBasePath DB storage base path or null if persistence is not enabled.
+     * @param consistentId compatibility consistent ID
+     * @return PDS folder settings compatible with previous versions.
+     */
+    private PdsFolderSettings compatibleResolve(
+        @Nullable final File pstStoreBasePath,
+        @NotNull final Serializable consistentId) {
+
+        if (cfg.getConsistentId() != null) {
+            // compatible mode from configuration is used fot this case, no locking, no consitent id change
+            return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId());
+        }
+
+        return new PdsFolderSettings(pstStoreBasePath, consistentId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PdsFolderSettings resolveFolders() throws IgniteCheckedException {
+        if (settings == null) {
+            settings = prepareNewSettings();
+
+            if (!settings.isCompatible()) {
+                if (log.isInfoEnabled())
+                    log.info("Consistent ID used for local node is [" + settings.consistentId() + "] " +
+                        "according to persistence data storage folders");
+
+                ctx.discovery().consistentId(settings.consistentId());
+            }
+        }
+        return settings;
+    }
+
+    /**
+     * Creates new settings when we don't have cached one.
+     *
+     * @return new settings with prelocked directory (if appropriate).
+     * @throws IgniteCheckedException if IO failed.
+     */
+    private PdsFolderSettings prepareNewSettings() throws IgniteCheckedException {
+        final File pstStoreBasePath = resolvePersistentStoreBasePath();
+        //here deprecated method is used to get compatible version of consistentId
+        final Serializable consistentId = ctx.discovery().consistentId();
+
+        if (!cfg.isPersistentStoreEnabled())
+            return compatibleResolve(pstStoreBasePath, consistentId);
+
+        if (getBoolean(IGNITE_DATA_STORAGE_FOLDER_BY_CONSISTENT_ID, false))
+            return compatibleResolve(pstStoreBasePath, consistentId);
+
+        // compatible mode from configuration is used fot this case
+        if (cfg.getConsistentId() != null) {
+            // compatible mode from configuration is used fot this case, no locking, no consistent id change
+            return new PdsFolderSettings(pstStoreBasePath, cfg.getConsistentId());
+        }
+        // The node scans the work directory and checks if there is a folder matching the consistent ID.
+        // If such a folder exists, we start up with this ID (compatibility mode)
+        final String subFolder = U.maskForFileName(consistentId.toString());
+
+        final GridCacheDatabaseSharedManager.FileLockHolder oldStyleFolderLockHolder = tryLock(new File(pstStoreBasePath, subFolder));
+
+        if (oldStyleFolderLockHolder != null)
+            return new PdsFolderSettings(pstStoreBasePath,
+                subFolder,
+                consistentId,
+                oldStyleFolderLockHolder,
+                true);
+
+        final File[] oldStyleFolders = pstStoreBasePath.listFiles(DB_SUBFOLDERS_OLD_STYLE_FILTER);
+
+        if (oldStyleFolders != null && oldStyleFolders.length != 0) {
+            for (File folder : oldStyleFolders) {
+                final String path = getPathDisplayableInfo(folder);
+
+                U.warn(log, "There is other non-empty storage folder under storage base directory [" + path + "]");
+            }
+        }
+
+        for (FolderCandidate next : getNodeIndexSortedCandidates(pstStoreBasePath)) {
+            final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = tryLock(next.subFolderFile());
+
+            if (fileLockHolder != null) {
+                if (log.isInfoEnabled())
+                    log.info("Successfully locked persistence storage folder [" + next.subFolderFile() + "]");
+
+                return new PdsFolderSettings(pstStoreBasePath,
+                    next.subFolderFile().getName(),
+                    next.uuid(),
+                    fileLockHolder,
+                    false);
+            }
+        }
+
+        // was not able to find free slot, allocating new
+        final GridCacheDatabaseSharedManager.FileLockHolder rootDirLock = lockRootDirectory(pstStoreBasePath);
+
+        try {
+            final List<FolderCandidate> sortedCandidates = getNodeIndexSortedCandidates(pstStoreBasePath);
+            final int nodeIdx = sortedCandidates.isEmpty() ? 0 : (sortedCandidates.get(sortedCandidates.size() - 1).nodeIndex() + 1);
+
+            return generateAndLockNewDbStorage(pstStoreBasePath, nodeIdx);
+        }
+        finally {
+            rootDirLock.release();
+            rootDirLock.close();
+        }
+    }
+
+    /**
+     * Calculate overall folder size.
+     *
+     * @param dir directory to scan.
+     * @return total size in bytes.
+     */
+    private static FolderParams folderSize(File dir) {
+        final FolderParams params = new FolderParams();
+
+        visitFolder(dir, params);
+
+        return params;
+    }
+
+    /**
+     * Scans provided directory and its sub dirs, collects found metrics.
+     *
+     * @param dir directory to start scan from.
+     * @param params input/output.
+     */
+    private static void visitFolder(final File dir, final FolderParams params) {
+        for (File file : dir.listFiles()) {
+            if (file.isDirectory())
+                visitFolder(file, params);
+            else {
+                params.size += file.length();
+                params.lastModified = Math.max(params.lastModified, dir.lastModified());
+            }
+        }
+    }
+
+    /**
+     * @param folder folder to scan.
+     * @return folder displayable information.
+     */
+    @NotNull private String getPathDisplayableInfo(final File folder) {
+        final SB res = new SB();
+
+        res.a(getCanonicalPath(folder));
+        res.a(", ");
+        final FolderParams params = folderSize(folder);
+
+        res.a(params.size);
+        res.a(" bytes, modified ");
+        final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("MM/dd/yyyy hh:mm a");
+
+        res.a(simpleDateFormat.format(params.lastModified));
+        res.a(" ");
+
+        return res.toString();
+    }
+
+    /**
+     * Returns the canonical pathname string of this abstract pathname.
+     *
+     * @param file path to convert.
+     * @return canonical pathname or at leas absolute if convert to canonical failed.
+     */
+    @NotNull private String getCanonicalPath(final File file) {
+        try {
+            return file.getCanonicalPath();
+        }
+        catch (IOException ignored) {
+            return file.getAbsolutePath();
+        }
+    }
+
+    /**
+     * Pad start of string with provided character.
+     *
+     * @param str sting to pad.
+     * @param minLength expected length.
+     * @param padChar padding character.
+     * @return padded string.
+     */
+    private static String padStart(String str, int minLength, char padChar) {
+        A.notNull(str, "String should not be empty");
+        if (str.length() >= minLength)
+            return str;
+
+        final SB sb = new SB(minLength);
+
+        for (int i = str.length(); i < minLength; ++i)
+            sb.a(padChar);
+
+        sb.a(str);
+
+        return sb.toString();
+
+    }
+
+    /**
+     * Creates new DB storage folder.
+     *
+     * @param pstStoreBasePath DB root path.
+     * @param nodeIdx next node index to use in folder name.
+     * @return new settings to be used in this node.
+     * @throws IgniteCheckedException if failed.
+     */
+    @NotNull private PdsFolderSettings generateAndLockNewDbStorage(final File pstStoreBasePath,
+        final int nodeIdx) throws IgniteCheckedException {
+
+        final UUID uuid = UUID.randomUUID();
+        final String consIdBasedFolder = genNewStyleSubfolderName(nodeIdx, uuid);
+        final File newRandomFolder = U.resolveWorkDirectory(pstStoreBasePath.getAbsolutePath(), consIdBasedFolder, false); //mkdir here
+        final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = tryLock(newRandomFolder);
+
+        if (fileLockHolder != null) {
+            if (log.isInfoEnabled())
+                log.info("Successfully created new persistent storage folder [" + newRandomFolder + "]");
+
+            return new PdsFolderSettings(pstStoreBasePath, consIdBasedFolder, uuid, fileLockHolder, false);
+        }
+        throw new IgniteCheckedException("Unable to lock file generated randomly [" + newRandomFolder + "]");
+    }
+
+    /**
+     * Generates DB subfolder name for provided node index (local) and UUID (consistent ID)
+     *
+     * @param nodeIdx node index.
+     * @param uuid consistent ID.
+     * @return folder file name
+     */
+    @NotNull public static String genNewStyleSubfolderName(final int nodeIdx, final UUID uuid) {
+        final String uuidAsStr = uuid.toString();
+
+        assert uuidAsStr.matches(UUID_STR_PATTERN);
+
+        final String nodeIdxPadded = padStart(Integer.toString(nodeIdx), 2, '0');
+
+        return DB_FOLDER_PREFIX + nodeIdxPadded + NODEIDX_UID_SEPARATOR + uuidAsStr;
+    }
+
+    /**
+     * Acquires lock to root storage directory, used to lock root directory in case creating new files is required.
+     *
+     * @param pstStoreBasePath rood DB dir to lock
+     * @return locked directory, should be released and closed later
+     * @throws IgniteCheckedException if failed
+     */
+    @NotNull private GridCacheDatabaseSharedManager.FileLockHolder lockRootDirectory(File pstStoreBasePath)
+        throws IgniteCheckedException {
+
+        GridCacheDatabaseSharedManager.FileLockHolder rootDirLock;
+        int retry = 0;
+
+        while ((rootDirLock = tryLock(pstStoreBasePath)) == null) {
+            if (retry > 600)
+                throw new IgniteCheckedException("Unable to start under DB storage path [" + pstStoreBasePath + "]" +
+                    ". Lock is being held to root directory");
+            retry++;
+        }
+
+        return rootDirLock;
+    }
+
+    /**
+     * @param pstStoreBasePath root storage folder to scan.
+     * @return empty list if there is no files in folder to test. Non null value is returned for folder having
+     * applicable new style files. Collection is sorted ascending according to node ID, 0 node index is coming first.
+     */
+    @Nullable private List<FolderCandidate> getNodeIndexSortedCandidates(File pstStoreBasePath) {
+        final File[] files = pstStoreBasePath.listFiles(DB_SUBFOLDERS_NEW_STYLE_FILTER);
+
+        if (files == null)
+            return Collections.emptyList();
+
+        final List<FolderCandidate> res = new ArrayList<>();
+
+        for (File file : files) {
+            final FolderCandidate candidate = parseFileName(file);
+
+            if (candidate != null)
+                res.add(candidate);
+        }
+        Collections.sort(res, new Comparator<FolderCandidate>() {
+            @Override public int compare(FolderCandidate c1, FolderCandidate c2) {
+                return Integer.compare(c1.nodeIndex(), c2.nodeIndex());
+            }
+        });
+
+        return res;
+    }
+
+    /**
+     * Tries to lock subfolder within storage root folder.
+     *
+     * @param dbStoreDirWithSubdirectory DB store directory, is to be absolute and should include consistent ID based
+     * sub folder.
+     * @return non null holder if lock was successful, null in case lock failed. If directory does not exist method will
+     * always fail to lock.
+     */
+    private GridCacheDatabaseSharedManager.FileLockHolder tryLock(File dbStoreDirWithSubdirectory) {
+        if (!dbStoreDirWithSubdirectory.exists())
+            return null;
+
+        final String path = dbStoreDirWithSubdirectory.getAbsolutePath();
+        final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder
+            = new GridCacheDatabaseSharedManager.FileLockHolder(path, ctx, log);
+
+        try {
+            fileLockHolder.tryLock(1000);
+
+            return fileLockHolder;
+        }
+        catch (IgniteCheckedException e) {
+            U.closeQuiet(fileLockHolder);
+
+            if (log.isInfoEnabled())
+                log.info("Unable to acquire lock to file [" + path + "], reason: " + e.getMessage());
+
+            return null;
+        }
+    }
+
+    /**
+     * @return DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent
+     * store configuration. Null if persistence is not enabled. Returned folder is created automatically.
+     * @throws IgniteCheckedException if I/O failed.
+     */
+    @Nullable private File resolvePersistentStoreBasePath() throws IgniteCheckedException {
+        final PersistentStoreConfiguration pstCfg = cfg.getPersistentStoreConfiguration();
+
+        if (pstCfg == null)
+            return null;
+
+        final String pstPath = pstCfg.getPersistentStorePath();
+
+        return U.resolveWorkDirectory(
+            cfg.getWorkDirectory(),
+            pstPath != null ? pstPath : DB_DEFAULT_FOLDER,
+            false
+        );
+
+    }
+
+    /**
+     * @param subFolderFile new style folder name to parse
+     * @return Pair of UUID and node index
+     */
+    private FolderCandidate parseFileName(@NotNull final File subFolderFile) {
+        return parseSubFolderName(subFolderFile, log);
+    }
+
+    /**
+     * @param subFolderFile new style file to parse.
+     * @param log Logger.
+     * @return Pair of UUID and node index.
+     */
+    @Nullable public static FolderCandidate parseSubFolderName(
+        @NotNull final File subFolderFile,
+        @NotNull final IgniteLogger log) {
+
+        final String fileName = subFolderFile.getName();
+        final Matcher matcher = Pattern.compile(NODE_PATTERN).matcher(fileName);
+        if (!matcher.find())
+            return null;
+
+        int uidStart = matcher.end();
+
+        try {
+            final String uid = fileName.substring(uidStart);
+            final UUID uuid = UUID.fromString(uid);
+            final String substring = fileName.substring(DB_FOLDER_PREFIX.length(), uidStart - NODEIDX_UID_SEPARATOR.length());
+            final int idx = Integer.parseInt(substring);
+
+            return new FolderCandidate(subFolderFile, idx, uuid);
+        }
+        catch (Exception e) {
+            U.warn(log, "Unable to parse new style file format from [" + subFolderFile.getAbsolutePath() + "]: " + e);
+
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
+        if (settings != null) {
+            final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder = settings.getLockedFileLockHolder();
+
+            if (fileLockHolder != null) {
+                fileLockHolder.release();
+                fileLockHolder.close();
+            }
+        }
+        super.stop(cancel);
+    }
+
+    /** Path metrics */
+    private static class FolderParams {
+        /** Overall size in bytes. */
+        private long size;
+
+        /** Last modified. */
+        private long lastModified;
+    }
+
+    /**
+     * Represents parsed new style file and encoded parameters in this file name
+     */
+    public static class FolderCandidate {
+        /** Absolute file path pointing to DB subfolder within DB storage root folder. */
+        private final File subFolderFile;
+
+        /** Node index (local, usually 0 if multiple nodes are not started at local PC). */
+        private final int nodeIdx;
+
+        /** Uuid contained in file name, is to be set as consistent ID. */
+        private final UUID uuid;
+
+        /**
+         * @param subFolderFile Absolute file path pointing to DB subfolder.
+         * @param nodeIdx Node index.
+         * @param uuid Uuid.
+         */
+        public FolderCandidate(File subFolderFile, int nodeIdx, UUID uuid) {
+            this.subFolderFile = subFolderFile;
+            this.nodeIdx = nodeIdx;
+            this.uuid = uuid;
+        }
+
+        /**
+         * @return Node index (local, usually 0 if multiple nodes are not started at local PC).
+         */
+        public int nodeIndex() {
+            return nodeIdx;
+        }
+
+        /**
+         * @return Uuid contained in file name, is to be set as consistent ID.
+         */
+        public Serializable uuid() {
+            return uuid;
+        }
+
+        /**
+         * @return Absolute file path pointing to DB subfolder within DB storage root folder.
+         */
+        public File subFolderFile() {
+            return subFolderFile;
+        }
+    }
+}
+
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
new file mode 100644
index 0000000..20fb5ca
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFolderSettings.java
@@ -0,0 +1,138 @@
+/*
+ * 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.filename;
+
+import java.io.File;
+import java.io.Serializable;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Class holds information required for folder generation for ignite persistent store
+ */
+public class PdsFolderSettings {
+    /**
+     * DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent store
+     * configuration. <br>
+     * Note WAL storage may be configured outside this path.<br>
+     * This value may be null if persistence is not enabled.
+     */
+    @Nullable private final File persistentStoreRootPath;
+
+    /** Sub folder name containing consistent ID and optionally node index. */
+    private final String folderName;
+
+    /** Consistent id to be set to local node. */
+    private final Serializable consistentId;
+
+    /**
+     * File lock holder with prelocked db directory. For non compatible mode this holder contains prelocked work
+     * directory. This value is to be used at activate instead of locking. <br> May be null in case preconfigured
+     * consistent ID is used or in case lock holder was already taken by other processor.
+     */
+    @Nullable private final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder;
+
+    /**
+     * Indicates if compatible mode is enabled, in that case all sub folders are generated from consistent ID without
+     * 'node' and node index prefix. In compatible mode there is no overriding for consistent ID is done.
+     */
+    private final boolean compatible;
+
+    /**
+     * Creates settings in for new PST(DB) folder mode.
+     *
+     * @param persistentStoreRootPath Persistent store root path or null if non PDS mode.
+     * @param folderName Sub folder name containing consistent ID and optionally node index.
+     * @param consistentId Consistent id.
+     * @param fileLockHolder File lock holder with prelocked db directory.
+     * @param compatible Compatible mode flag.
+     */
+    public PdsFolderSettings(@Nullable final File persistentStoreRootPath,
+        final String folderName,
+        final Serializable consistentId,
+        @Nullable final GridCacheDatabaseSharedManager.FileLockHolder fileLockHolder,
+        final boolean compatible) {
+
+        this.consistentId = consistentId;
+        this.folderName = folderName;
+        this.fileLockHolder = fileLockHolder;
+        this.compatible = compatible;
+        this.persistentStoreRootPath = persistentStoreRootPath;
+    }
+
+    /**
+     * Creates settings for compatible mode. Folder name is consistent ID (masked), no node prefix is added.
+     *
+     * @param persistentStoreRootPath root DB path.
+     * @param consistentId node consistent ID.
+     */
+    public PdsFolderSettings(
+        @Nullable final File persistentStoreRootPath,
+        @NotNull final Serializable consistentId) {
+
+        this.consistentId = consistentId;
+        this.compatible = true;
+        this.folderName = U.maskForFileName(consistentId.toString());
+        this.persistentStoreRootPath = persistentStoreRootPath;
+        this.fileLockHolder = null;
+    }
+
+    /**
+     * @return sub folders name based on consistent ID. In compatible mode this is escaped consistent ID, in new mode
+     * this is UUID based folder name.
+     */
+    public String folderName() {
+        return folderName;
+    }
+
+    /**
+     * @return Consistent id to be set to local node.
+     */
+    public Serializable consistentId() {
+        return consistentId;
+    }
+
+    /**
+     * @return flag indicating if compatible mode is enabled for folder generation. In that case all sub folders names are
+     * generated from consistent ID without 'node' and node index prefix. In compatible mode there is no overriding for
+     * consistent ID is done for cluster node. Locking files is independent to compatibility mode.
+     */
+    public boolean isCompatible() {
+        return compatible;
+    }
+
+    /**
+     * Returns already locked file lock holder to lock file in {@link #persistentStoreRootPath}. Unlock and close this
+     * lock is not required.
+     *
+     * @return File lock holder with prelocked db directory.
+     */
+    @Nullable public GridCacheDatabaseSharedManager.FileLockHolder getLockedFileLockHolder() {
+        return fileLockHolder;
+    }
+
+    /**
+     * @return DB storage absolute root path resolved as 'db' folder in Ignite work dir (by default) or using persistent
+     * store configuration. Note WAL storage may be configured outside this path. May return null for non pds mode.
+     */
+    @Nullable public File persistentStoreRootPath() {
+        return persistentStoreRootPath;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
new file mode 100644
index 0000000..cefaa04
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/filename/PdsFoldersResolver.java
@@ -0,0 +1,33 @@
+/*
+ * 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.filename;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Resolves folders for PDS mode, may have side effect as setting random UUID as local node consistent ID.
+ */
+public interface PdsFoldersResolver {
+    /**
+     * Prepares and caches PDS folder settings. Subsequent call to this method will provide same settings.
+     *
+     * @return PDS folder settings, consistentID and prelocked DB file lock.
+     * @throws IgniteCheckedException if failed.
+     */
+    public PdsFolderSettings resolveFolders() throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/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 c8715aa..6a75dd2 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
@@ -64,6 +64,7 @@ import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabase
 import org.apache.ignite.internal.processors.cache.persistence.PersistenceMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.PureJavaCrc32;
 import org.apache.ignite.internal.processors.cache.persistence.wal.record.HeaderRecord;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordDataV1Serializer;
@@ -162,7 +163,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** WAL archive directory (including consistent ID as subfolder) */
     private File walArchiveDir;
 
-    /** Serializer of latest version. */
+    /** Serializer of latest version, used to read header record and for write records */
     private RecordSerializer serializer;
 
     /** Serializer latest version to use. */
@@ -258,25 +259,21 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         if (!cctx.kernalContext().clientNode()) {
-            String consId = consistentId();
-
-            A.notNullOrEmpty(consId, "consistentId");
-
-            consId = U.maskForFileName(consId);
+            final PdsFolderSettings resolveFolders = cctx.kernalContext().pdsFolderResolver().resolveFolders();
 
             checkWalConfiguration();
 
             walWorkDir = initDirectory(
                 psCfg.getWalStorePath(),
                 PersistentStoreConfiguration.DFLT_WAL_STORE_PATH,
-                consId,
+                resolveFolders.folderName(),
                 "write ahead log work directory"
             );
 
             walArchiveDir = initDirectory(
                 psCfg.getWalArchivePath(),
                 PersistentStoreConfiguration.DFLT_WAL_ARCHIVE_PATH,
-                consId,
+                resolveFolders.folderName(),
                 "write ahead log archive directory"
             );
 
@@ -317,13 +314,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
         }
     }
 
-    /**
-     * @return Consistent ID.
-     */
-    protected String consistentId() {
-        return cctx.discovery().consistentId().toString();
-    }
-
     /** {@inheritDoc} */
     @Override protected void stop0(boolean cancel) {
         final GridTimeoutProcessor.CancelableTask schedule = backgroundFlushSchedule;

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index 07be8b4..e234766 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.persistence.wal.reader;
 
 import java.io.File;
+import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.util.Iterator;
 import java.util.List;
@@ -47,6 +48,8 @@ import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFoldersResolver;
+import org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderSettings;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
@@ -79,6 +82,7 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.StripedExecutor;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.PluginNotFoundException;
 import org.apache.ignite.plugin.PluginProvider;
@@ -106,7 +110,7 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     @Nullable private IgniteCacheObjectProcessor cacheObjProcessor;
 
     /** Marshaller context implementation. */
-    private MarshallerContextImpl marshallerContext;
+    private MarshallerContextImpl marshallerCtx;
 
     /**
      * @param log Logger.
@@ -130,13 +134,13 @@ public class StandaloneGridKernalContext implements GridKernalContext {
             throw new IllegalStateException("Must not fail on empty providers list.", e);
         }
 
-        this.marshallerContext = new MarshallerContextImpl(null);
+        this.marshallerCtx = new MarshallerContextImpl(null);
         this.cfg = prepareIgniteConfiguration();
         this.cacheObjProcessor = binaryMetadataFileStoreDir != null ? binaryProcessor(this, binaryMetadataFileStoreDir) : null;
 
         if (marshallerMappingFileStoreDir != null) {
-            marshallerContext.setMarshallerMappingFileStoreDir(marshallerMappingFileStoreDir);
-            marshallerContext.onMarshallerProcessorStarted(this, null);
+            marshallerCtx.setMarshallerMappingFileStoreDir(marshallerMappingFileStoreDir);
+            marshallerCtx.onMarshallerProcessorStarted(this, null);
         }
     }
 
@@ -176,7 +180,7 @@ public class StandaloneGridKernalContext implements GridKernalContext {
         PersistentStoreConfiguration pstCfg = new PersistentStoreConfiguration();
         cfg.setPersistentStoreConfiguration(pstCfg);
 
-        marshaller.setContext(marshallerContext);
+        marshaller.setContext(marshallerCtx);
 
         return cfg;
     }
@@ -392,11 +396,7 @@ public class StandaloneGridKernalContext implements GridKernalContext {
 
     /** {@inheritDoc} */
     @Override public GridDiscoveryManager discovery() {
-        return new GridDiscoveryManager(StandaloneGridKernalContext.this) {
-            @Override public Object consistentId() {
-                return ""; // some non null value is required
-            }
-        };
+        return new GridDiscoveryManager(this);
     }
 
     /** {@inheritDoc} */
@@ -579,7 +579,7 @@ public class StandaloneGridKernalContext implements GridKernalContext {
 
     /** {@inheritDoc} */
     @Override public MarshallerContextImpl marshallerContext() {
-        return marshallerContext;
+        return marshallerCtx;
     }
 
     /** {@inheritDoc} */
@@ -598,6 +598,16 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     }
 
     /** {@inheritDoc} */
+    @Override public PdsFoldersResolver pdsFolderResolver() {
+        return new PdsFoldersResolver() {
+            /** {@inheritDoc} */
+            @Override public PdsFolderSettings resolveFolders() {
+                return new PdsFolderSettings(new File("."), U.maskForFileName(""));
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
     @NotNull @Override public Iterator<GridComponent> iterator() {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index e6eaa8e..eb8ee30 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -966,10 +966,10 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi {
 
             initAddresses();
 
-            Serializable cfgId = ignite.configuration().getConsistentId();
+            final Serializable cfgId = ignite.configuration().getConsistentId();
 
             if (cfgId == null) {
-                List<String> sortedAddrs = new ArrayList<>(addrs.get1());
+                final List<String> sortedAddrs = new ArrayList<>(addrs.get1());
 
                 Collections.sort(sortedAddrs);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/62f3c4c5/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index 20fb6c5..38c2a1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@ -225,6 +225,21 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
         return consistentId;
     }
 
+    /**
+     * Sets consistent globally unique node ID which survives node restarts.
+     *
+     * @param consistentId Consistent globally unique node ID.
+     */
+    public void setConsistentId(Serializable consistentId) {
+        this.consistentId = consistentId;
+
+        final Map<String, Object> map = new HashMap<>(attrs);
+
+        map.put(ATTR_NODE_CONSISTENT_ID, consistentId);
+
+        attrs = Collections.unmodifiableMap(map);
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <T> T attribute(String name) {