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

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

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-6305
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",