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 2016/09/23 15:26:56 UTC

[22/24] ignite git commit: IGNITE-3938 Implemented escaping for SQL identifiers. Added tests for escaped mode.

IGNITE-3938 Implemented escaping for SQL identifiers. Added tests for escaped mode.


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

Branch: refs/heads/ignite-comm-opts2
Commit: 63888bd99b8a84cc961b226efba9964dcc292f4c
Parents: a97483a
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Sep 22 16:22:40 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Sep 22 16:22:40 2016 +0700

----------------------------------------------------------------------
 .../store/jdbc/CacheAbstractJdbcStore.java      | 104 +++++++++++++++----
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   |  46 ++++++--
 .../store/jdbc/dialect/BasicJdbcDialect.java    |   8 +-
 .../cache/store/jdbc/dialect/DB2Dialect.java    |   6 +-
 .../cache/store/jdbc/dialect/H2Dialect.java     |   5 +-
 .../cache/store/jdbc/dialect/JdbcDialect.java   |  18 +++-
 .../cache/store/jdbc/dialect/MySQLDialect.java  |   9 +-
 .../cache/store/jdbc/dialect/OracleDialect.java |   5 +-
 .../store/jdbc/dialect/SQLServerDialect.java    |  14 ++-
 .../CacheJdbcPojoStoreAbstractSelfTest.java     |  48 ++++++---
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java |   2 +-
 ...reBinaryMarshallerWithSqlEscapeSelfTest.java |  28 +++++
 ...dbcPojoStoreOptimizedMarshallerSelfTest.java |   2 +-
 ...ptimizedMarshallerWithSqlEscapeSelfTest.java |  28 +++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +
 .../src/test/config/jdbc-pojo-store-builtin.xml |  12 +--
 .../src/test/config/jdbc-pojo-store-obj.xml     |  12 +--
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java |  12 +--
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |  14 ++-
 .../store/jdbc/CachePojoStoreXmlSelfTest.java   |  13 ++-
 .../CachePojoStoreXmlWithSqlEscapeSelfTest.java |  28 +++++
 .../testsuites/IgniteSpringTestSuite.java       |   5 +-
 22 files changed, 328 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
index fe8a50b..a9a8ce1 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheAbstractJdbcStore.java
@@ -187,6 +187,9 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
     /** Hash calculator.  */
     protected JdbcTypeHasher hasher = JdbcTypeDefaultHasher.INSTANCE;
 
+    /** Flag indicating that table and field names should be escaped in all SQL queries created by JDBC POJO store. */
+    private boolean sqlEscapeAll;
+
     /**
      * Get field value from object for use as query parameter.
      *
@@ -727,7 +730,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
                     checkTypeConfiguration(cacheName, valKind, valType, type.getValueFields());
 
-                    entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, keyKind, valKind));
+                    entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, keyKind, valKind, sqlEscapeAll));
 
                     // Add one more binding to binary typeId for POJOs,
                     // because object could be passed to store in binary format.
@@ -736,7 +739,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
                         valKind = valKind == TypeKind.POJO ? TypeKind.BINARY : valKind;
 
-                        entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, TypeKind.BINARY, valKind));
+                        entryMappings.put(keyTypeId, new EntryMapping(cacheName, dialect, type, TypeKind.BINARY, valKind, sqlEscapeAll));
                     }
                 }
 
@@ -1675,6 +1678,28 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
     }
 
     /**
+     * If {@code true} all the SQL table and field names will be escaped with double quotes like
+     * ({@code "tableName"."fieldsName"}). This enforces case sensitivity for field names and
+     * also allows having special characters in table and field names.
+     *
+     * @return Flag value.
+     */
+    public boolean isSqlEscapeAll() {
+        return sqlEscapeAll;
+    }
+
+    /**
+     * If {@code true} all the SQL table and field names will be escaped with double quotes like
+     * ({@code "tableName"."fieldsName"}). This enforces case sensitivity for field names and
+     * also allows having special characters in table and field names.
+     *
+     * @param sqlEscapeAll Flag value.
+     */
+    public void setSqlEscapeAll(boolean sqlEscapeAll) {
+        this.sqlEscapeAll = sqlEscapeAll;
+    }
+
+    /**
      * @return Ignite instance.
      */
     protected Ignite ignite() {
@@ -1740,9 +1765,15 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
         /** Database key columns. */
         private final Collection<String> keyCols;
 
+        /** Database key columns prepared for building SQL queries.. */
+        private final Collection<String> sqlKeyCols;
+
         /** Database unique value columns. */
         private final Collection<String> cols;
 
+        /** Database unique value columns prepared for building SQL queries. */
+        private final Collection<String> sqlCols;
+
         /** Select query columns index. */
         private final Map<String, Integer> loadColIdxs;
 
@@ -1761,13 +1792,34 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
         /** Full table name. */
         private final String fullTblName;
 
+        /** Full table name prepared for building SQL queries. */
+        private final String sqlFullTblName;
+
+        /**
+         * Escape collection of column names.
+         * @param dialect Database dialect.
+         * @param cols Columns.
+         * @return Collection of escaped names.
+         */
+        private static Collection<String> escape(JdbcDialect dialect, Collection<String> cols) {
+            Collection<String> res = new ArrayList<>(cols.size());
+
+            for (String col : cols)
+                res.add(dialect.escape(col));
+
+            return res;
+        }
+
         /**
          * @param cacheName Cache name.
          * @param dialect JDBC dialect.
          * @param typeMeta Type metadata.
+         * @param keyKind Type kind.
+         * @param valKind Value kind.
+         * @param escape Escape SQL identifiers flag.
          */
         public EntryMapping(@Nullable String cacheName, JdbcDialect dialect, JdbcType typeMeta,
-            TypeKind keyKind, TypeKind valKind) {
+            TypeKind keyKind, TypeKind valKind, boolean escape) {
             this.cacheName = cacheName;
 
             this.dialect = dialect;
@@ -1794,11 +1846,9 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
             String tblName = typeMeta.getDatabaseTable();
 
-            fullTblName = F.isEmpty(schema) ? tblName : schema + "." + tblName;
-
-            Collection<String> uniqValCols = databaseColumns(uniqValFlds);
+            Collection<String> uniqueValCols = databaseColumns(uniqValFlds);
 
-            cols = F.concat(false, keyCols, uniqValCols);
+            cols = F.concat(false, keyCols, uniqueValCols);
 
             loadColIdxs = U.newHashMap(cols.size());
 
@@ -1807,23 +1857,41 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
             for (String col : cols)
                 loadColIdxs.put(col.toUpperCase(), idx++);
 
-            loadCacheQry = dialect.loadCacheQuery(fullTblName, cols);
+            fullTblName = F.isEmpty(schema) ? tblName : schema + "." + tblName;
+
+            Collection<String> sqlUniqueValCols;
+
+            if (escape) {
+                sqlFullTblName = F.isEmpty(schema) ? dialect.escape(tblName) : dialect.escape(schema) + "." + dialect.escape(tblName);
+
+                sqlCols = escape(dialect, cols);
+                sqlKeyCols = escape(dialect, keyCols);
+                sqlUniqueValCols = escape(dialect, uniqueValCols);
+            }
+            else {
+                sqlFullTblName = fullTblName;
+                sqlCols = cols;
+                sqlKeyCols = keyCols;
+                sqlUniqueValCols = uniqueValCols;
+            }
+
+            loadCacheQry = dialect.loadCacheQuery(sqlFullTblName, sqlCols);
 
-            loadCacheSelRangeQry = dialect.loadCacheSelectRangeQuery(fullTblName, keyCols);
+            loadCacheSelRangeQry = dialect.loadCacheSelectRangeQuery(sqlFullTblName, sqlKeyCols);
 
-            loadQrySingle = dialect.loadQuery(fullTblName, keyCols, cols, 1);
+            loadQrySingle = dialect.loadQuery(sqlFullTblName, sqlKeyCols, sqlCols, 1);
 
-            maxKeysPerStmt = dialect.getMaxParameterCount() / keyCols.size();
+            maxKeysPerStmt = dialect.getMaxParameterCount() / sqlKeyCols.size();
 
-            loadQry = dialect.loadQuery(fullTblName, keyCols, cols, maxKeysPerStmt);
+            loadQry = dialect.loadQuery(sqlFullTblName, sqlKeyCols, sqlCols, maxKeysPerStmt);
 
-            insQry = dialect.insertQuery(fullTblName, keyCols, uniqValCols);
+            insQry = dialect.insertQuery(sqlFullTblName, sqlKeyCols, sqlUniqueValCols);
 
-            updQry = dialect.updateQuery(fullTblName, keyCols, uniqValCols);
+            updQry = dialect.updateQuery(sqlFullTblName, sqlKeyCols, sqlUniqueValCols);
 
-            mergeQry = dialect.mergeQuery(fullTblName, keyCols, uniqValCols);
+            mergeQry = dialect.mergeQuery(sqlFullTblName, sqlKeyCols, sqlUniqueValCols);
 
-            remQry = dialect.removeQuery(fullTblName, keyCols);
+            remQry = dialect.removeQuery(sqlFullTblName, sqlKeyCols);
         }
 
         /**
@@ -1884,7 +1952,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
             if (keyCnt == 1)
                 return loadQrySingle;
 
-            return dialect.loadQuery(fullTblName, keyCols, cols, keyCnt);
+            return dialect.loadQuery(sqlFullTblName, sqlKeyCols, sqlCols, keyCnt);
         }
 
         /**
@@ -1895,7 +1963,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
          * @return Query with range.
          */
         protected String loadCacheRangeQuery(boolean appendLowerBound, boolean appendUpperBound) {
-            return dialect.loadCacheRangeQuery(fullTblName, keyCols, cols, appendLowerBound, appendUpperBound);
+            return dialect.loadCacheRangeQuery(sqlFullTblName, sqlKeyCols, sqlCols, appendLowerBound, appendUpperBound);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
index 0fca3ab..8bc6031 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
@@ -111,7 +111,7 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
     public static final int DFLT_PARALLEL_LOAD_CACHE_MINIMUM_THRESHOLD = 512;
 
     /** Maximum batch size for writeAll and deleteAll operations. */
-    private int batchSizw = DFLT_BATCH_SIZE;
+    private int batchSize = DFLT_BATCH_SIZE;
 
     /** Name of data source bean. */
     private String dataSrcBean;
@@ -123,7 +123,7 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
     private int maxPoolSize = Runtime.getRuntime().availableProcessors();
 
     /** Maximum write attempts in case of database error. */
-    private int maxWriteAttempts = DFLT_WRITE_ATTEMPTS;
+    private int maxWrtAttempts = DFLT_WRITE_ATTEMPTS;
 
     /** Parallel load cache minimum threshold. If {@code 0} then load sequentially. */
     private int parallelLoadCacheMinThreshold = DFLT_PARALLEL_LOAD_CACHE_MINIMUM_THRESHOLD;
@@ -140,6 +140,9 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
     /** Data source factory. */
     private Factory<DataSource> dataSrcFactory;
 
+    /** Flag indicating that table and field names should be escaped in all SQL queries created by JDBC POJO store. */
+    private boolean sqlEscapeAll;
+
     /** Application context. */
     @SpringApplicationContextResource
     private transient Object appCtx;
@@ -148,12 +151,14 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
     @Override public CacheJdbcPojoStore<K, V> create() {
         CacheJdbcPojoStore<K, V> store = new CacheJdbcPojoStore<>();
 
-        store.setBatchSize(batchSizw);
+        store.setBatchSize(batchSize);
         store.setDialect(dialect);
         store.setMaximumPoolSize(maxPoolSize);
-        store.setMaximumWriteAttempts(maxWriteAttempts);
+        store.setMaximumWriteAttempts(maxWrtAttempts);
         store.setParallelLoadCacheMinimumThreshold(parallelLoadCacheMinThreshold);
         store.setTypes(types);
+        store.setHasher(hasher);
+        store.setSqlEscapeAll(sqlEscapeAll);
 
         if (dataSrc != null)
             store.setDataSource(dataSrc);
@@ -201,7 +206,7 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
      * @return Maximum batch size.
      */
     public int getBatchSize() {
-        return batchSizw;
+        return batchSize;
     }
 
     /**
@@ -211,7 +216,7 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
      * @return {@code This} for chaining.
      */
     public CacheJdbcPojoStoreFactory setBatchSize(int batchSize) {
-        this.batchSizw = batchSize;
+        this.batchSize = batchSize;
 
         return this;
     }
@@ -285,7 +290,7 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
      * @return Maximum number of write attempts.
      */
     public int getMaximumWriteAttempts() {
-        return maxWriteAttempts;
+        return maxWrtAttempts;
     }
 
     /**
@@ -295,7 +300,7 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
      * @return {@code This} for chaining.
      */
     public CacheJdbcPojoStoreFactory setMaximumWriteAttempts(int maxWrtAttempts) {
-        this.maxWriteAttempts = maxWrtAttempts;
+        this.maxWrtAttempts = maxWrtAttempts;
 
         return this;
     }
@@ -386,6 +391,31 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheAbstractJdb
         return this;
     }
 
+    /**
+     * If {@code true} all the SQL table and field names will be escaped with double quotes like
+     * ({@code "tableName"."fieldsName"}). This enforces case sensitivity for field names and
+     * also allows having special characters in table and field names.
+     *
+     * @return Flag value.
+     */
+    public boolean isSqlEscapeAll() {
+        return sqlEscapeAll;
+    }
+
+    /**
+     * If {@code true} all the SQL table and field names will be escaped with double quotes like
+     * ({@code "tableName"."fieldsName"}). This enforces case sensitivity for field names and
+     * also allows having special characters in table and field names.
+     *
+     * @param sqlEscapeAll Flag value.
+     * @return {@code this} for chaining.
+     */
+    public CacheJdbcPojoStoreFactory<K, V> setSqlEscapeAll(boolean sqlEscapeAll) {
+        this.sqlEscapeAll = sqlEscapeAll;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheJdbcPojoStoreFactory.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
index cd9c986..5fa564b 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
@@ -149,6 +149,11 @@ public class BasicJdbcDialect implements JdbcDialect {
     }
 
     /** {@inheritDoc} */
+    @Override public String escape(String ident) {
+        return '"' + ident + '"';
+    }
+
+    /** {@inheritDoc} */
     @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection<String> keyCols) {
         String cols = mkString(keyCols, ",");
 
@@ -245,8 +250,7 @@ public class BasicJdbcDialect implements JdbcDialect {
     }
 
     /** {@inheritDoc} */
-    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols,
-        Collection<String> uniqCols) {
+    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols, Collection<String> uniqCols) {
         return "";
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java
index 2a92bdd..4bae14a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/DB2Dialect.java
@@ -34,9 +34,7 @@ public class DB2Dialect extends BasicJdbcDialect {
     }
 
     /** {@inheritDoc} */
-    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols,
-        Collection<String> uniqCols) {
-
+    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols, Collection<String> uniqCols) {
         Collection<String> cols = F.concat(false, keyCols, uniqCols);
 
         String colsLst = mkString(cols, ", ");
@@ -68,4 +66,4 @@ public class DB2Dialect extends BasicJdbcDialect {
                 "  INSERT (%s) VALUES (%s)", fullTblName, repeat("?", cols.size(), "", ",", ""), colsLst,
             match, setCols, colsLst, valuesCols);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java
index 94d27a9..019a4b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/H2Dialect.java
@@ -33,11 +33,10 @@ public class H2Dialect extends BasicJdbcDialect {
     }
 
     /** {@inheritDoc} */
-    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols,
-        Collection<String> uniqCols) {
+    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols, Collection<String> uniqCols) {
         Collection<String> cols = F.concat(false, keyCols, uniqCols);
 
         return String.format("MERGE INTO %s (%s) KEY (%s) VALUES(%s)", fullTblName, mkString(cols, ","),
             mkString(keyCols, ","), repeat("?", cols.size(), "", ", ", ""));
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java
index 9daa00b..8c153b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/JdbcDialect.java
@@ -25,7 +25,13 @@ import java.util.Collection;
  */
 public interface JdbcDialect extends Serializable {
     /**
-     * Construct select count query.
+     * @param ident SQL identifier to escape.
+     * @return Escaped SQL identifier.
+     */
+    public String escape(String ident);
+
+    /**
+     * Construct query to get ranges bounds.
      *
      * @param fullTblName Full table name.
      * @param keyCols Database key columns for order.
@@ -34,7 +40,7 @@ public interface JdbcDialect extends Serializable {
     public String loadCacheSelectRangeQuery(String fullTblName, Collection<String> keyCols);
 
     /**
-     * Construct select count query.
+     * Construct load cache query over specified range.
      *
      * @param fullTblName Full table name.
      * @param keyCols Database key columns for order.
@@ -43,8 +49,8 @@ public interface JdbcDialect extends Serializable {
      * @param appendUpperBound Need add upper bound for range.
      * @return Query for select count.
      */
-    public String loadCacheRangeQuery(String fullTblName,
-        Collection<String> keyCols, Iterable<String> uniqCols, boolean appendLowerBound, boolean appendUpperBound);
+    public String loadCacheRangeQuery(String fullTblName, Collection<String> keyCols, Iterable<String> uniqCols,
+        boolean appendLowerBound, boolean appendUpperBound);
 
     /**
      * Construct load cache query.
@@ -73,6 +79,7 @@ public interface JdbcDialect extends Serializable {
      * @param fullTblName Full table name.
      * @param keyCols Database key columns.
      * @param valCols Database value columns.
+     * @return Insert query.
      */
     public String insertQuery(String fullTblName, Collection<String> keyCols, Collection<String> valCols);
 
@@ -82,6 +89,7 @@ public interface JdbcDialect extends Serializable {
      * @param fullTblName Full table name.
      * @param keyCols Database key columns.
      * @param valCols Database value columns.
+     * @return Update query.
      */
     public String updateQuery(String fullTblName, Collection<String> keyCols, Iterable<String> valCols);
 
@@ -96,7 +104,7 @@ public interface JdbcDialect extends Serializable {
      * @param fullTblName Full table name.
      * @param keyCols Database key columns.
      * @param uniqCols Database unique value columns.
-     * @return Put query.
+     * @return Merge query.
      */
     public String mergeQuery(String fullTblName, Collection<String> keyCols, Collection<String> uniqCols);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java
index 84e6d05..1a5730b 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/MySQLDialect.java
@@ -29,6 +29,11 @@ public class MySQLDialect extends BasicJdbcDialect {
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
+    @Override public String escape(String ident) {
+        return '`' + ident + '`';
+    }
+
+    /** {@inheritDoc} */
     @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection<String> keyCols) {
         String cols = mkString(keyCols, ",");
 
@@ -43,9 +48,7 @@ public class MySQLDialect extends BasicJdbcDialect {
     }
 
     /** {@inheritDoc} */
-    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols,
-        Collection<String> uniqCols) {
-
+    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols, Collection<String> uniqCols) {
         Collection<String> cols = F.concat(false, keyCols, uniqCols);
 
         String updPart = mkString(uniqCols, new C1<String, String>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java
index b7a0400..a9efe0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/OracleDialect.java
@@ -34,8 +34,7 @@ public class OracleDialect extends BasicJdbcDialect {
     }
 
     /** {@inheritDoc} */
-    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols,
-        Collection<String> uniqCols) {
+    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols, Collection<String> uniqCols) {
         Collection<String> cols = F.concat(false, keyCols, uniqCols);
 
         String colsLst = mkString(cols, ", ");
@@ -72,4 +71,4 @@ public class OracleDialect extends BasicJdbcDialect {
             " WHEN NOT MATCHED THEN" +
             "  INSERT (%s) VALUES (%s)", fullTblName, selCols, match, setCols, colsLst, valuesCols);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java
index 9831aa8..883918f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/SQLServerDialect.java
@@ -29,13 +29,17 @@ public class SQLServerDialect extends BasicJdbcDialect {
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
+    @Override public String escape(String ident) {
+        return '[' + ident + ']';
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean hasMerge() {
         return true;
     }
 
     /** {@inheritDoc} */
-    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols,
-        Collection<String> uniqCols) {
+    @Override public String mergeQuery(String fullTblName, Collection<String> keyCols, Collection<String> uniqCols) {
         Collection<String> cols = F.concat(false, keyCols, uniqCols);
 
         String colsLst = mkString(cols, ", ");
@@ -64,7 +68,7 @@ public class SQLServerDialect extends BasicJdbcDialect {
                 " WHEN MATCHED THEN" +
                 "  UPDATE SET %s" +
                 " WHEN NOT MATCHED THEN" +
-                "  INSERT (%s) VALUES (%s);", fullTblName, repeat("?", cols.size(), "", ",", ""), colsLst,
-            match, setCols, colsLst, valuesCols);
+                "  INSERT (%s) VALUES (%s);", fullTblName, repeat("?", cols.size(), "", ",", ""),
+            colsLst, match, setCols, colsLst, valuesCols);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java
index e8592d7..83065f1 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreAbstractSelfTest.java
@@ -46,7 +46,7 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
 /**
- * Class for {@code PojoCacheStore} tests.
+ * Class for {@link CacheJdbcPojoStore} tests.
  */
 public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstractTest {
     /** IP finder. */
@@ -77,6 +77,13 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
     protected static boolean noValClasses;
 
     /**
+     * @return Flag indicating that all internal SQL queries should use escaped identifiers.
+     */
+    protected boolean sqlEscapeAll(){
+        return false;
+    }
+
+    /**
      * @return Connection to test in-memory H2 database.
      * @throws SQLException if failed to connect.
      */
@@ -164,10 +171,13 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
         }
 
         storeTypes[0].setValueType("org.apache.ignite.cache.store.jdbc.model.Organization" + (noValClasses ? "1" : ""));
+
+        boolean escape = sqlEscapeAll();
+
         storeTypes[0].setValueFields(
-            new JdbcTypeField(Types.INTEGER, "Id", Integer.class, "id"),
-            new JdbcTypeField(Types.VARCHAR, "Name", String.class, "name"),
-            new JdbcTypeField(Types.VARCHAR, "City", String.class, "city"));
+            new JdbcTypeField(Types.INTEGER, escape ? "ID" : "Id", Integer.class, "id"),
+            new JdbcTypeField(Types.VARCHAR, escape ? "NAME" : "Name", String.class, "name"),
+            new JdbcTypeField(Types.VARCHAR, escape ? "CITY" : "City", String.class, "city"));
 
         storeTypes[1] = new JdbcType();
         storeTypes[1].setCacheName(CACHE_NAME);
@@ -210,6 +220,7 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
         storeFactory.setDialect(new H2Dialect());
         storeFactory.setTypes(storeTypes());
         storeFactory.setDataSourceFactory(new H2DataSourceFactory()); // H2 DataSource factory.
+        storeFactory.setSqlEscapeAll(sqlEscapeAll());
 
         cc.setCacheStoreFactory(storeFactory);
         cc.setReadThrough(true);
@@ -228,8 +239,6 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
     protected void fillSampleDatabase(Connection conn) throws SQLException {
         info("Start to fill sample database...");
 
-        Random rnd = new Random();
-
         PreparedStatement orgStmt = conn.prepareStatement("INSERT INTO Organization(id, name, city) VALUES (?, ?, ?)");
 
         for (int i = 0; i < ORGANIZATION_CNT; i++) {
@@ -249,6 +258,8 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
         PreparedStatement prnStmt = conn.prepareStatement(
             "INSERT INTO Person(id, org_id, birthday, name) VALUES (?, ?, ?, ?)");
 
+        Random rnd = new Random();
+
         for (int i = 0; i < PERSON_CNT; i++) {
             prnStmt.setInt(1, i);
             prnStmt.setInt(2, i % 100);
@@ -366,7 +377,7 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
      *
      * @throws Exception If failed.
      */
-    private void checkPut() throws Exception {
+    private void checkPutRemove() throws Exception {
         IgniteCache<Object, Person> c1 = grid().cache(CACHE_NAME);
 
         Connection conn = getConnection();
@@ -419,6 +430,13 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
 
             assertFalse("Unexpected more data in result set", rs.next());
 
+            // Test remove.
+            c1.remove(key);
+
+            rs = stmt.executeQuery();
+
+            assertFalse("Unexpected non-empty result set", rs.next());
+
             U.closeQuiet(rs);
         }
         finally {
@@ -429,37 +447,37 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
     /**
      * @throws Exception If failed.
      */
-    public void testPutBuiltIn() throws Exception {
+    public void testPutRemoveBuiltIn() throws Exception {
         startTestGrid(true, false, false, false);
 
-        checkPut();
+        checkPutRemove();
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPut() throws Exception {
+    public void testPutRemove() throws Exception {
         startTestGrid(false, false, false, false);
 
-        checkPut();
+        checkPutRemove();
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPutTxBuiltIn() throws Exception {
+    public void testPutRemoveTxBuiltIn() throws Exception {
         startTestGrid(true, false, false, true);
 
-        checkPut();
+        checkPutRemove();
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPutTx() throws Exception {
+    public void testPutRemoveTx() throws Exception {
         startTestGrid(false, false, false, true);
 
-        checkPut();
+        checkPutRemove();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java
index f998027..14c743c 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerSelfTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 
 /**
- * Class for {@code PojoCacheStore} tests.
+ * Test for {@link CacheJdbcPojoStore} with binary marshaller.
  */
 public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoStoreAbstractSelfTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.java
new file mode 100644
index 0000000..829fffa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.cache.store.jdbc;
+
+/**
+ * Test for {@link CacheJdbcPojoStore} with binary marshaller and enabled SQL escaping.
+ */
+public class CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest extends CacheJdbcPojoStoreBinaryMarshallerSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean sqlEscapeAll(){
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java
index f40f7d7..3f6c9b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerSelfTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 
 /**
- * Class for {@code PojoCacheStore} tests.
+ * Test for {@link CacheJdbcPojoStore} with optimized marshaller.
  */
 public class CacheJdbcPojoStoreOptimizedMarshallerSelfTest extends CacheJdbcPojoStoreAbstractSelfTest {
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.java
new file mode 100644
index 0000000..d1ce726
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.cache.store.jdbc;
+
+/**
+ * Test for {@link CacheJdbcPojoStore} with optimized marshaller and enabled SQL escaping.
+ */
+public class CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest extends CacheJdbcPojoStoreOptimizedMarshallerSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean sqlEscapeAll(){
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 9240ef5..5ad4cb8 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -32,8 +32,10 @@ import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest;
 import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest;
 import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerSelfTest;
+import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreMultitreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerSelfTest;
+import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
@@ -243,7 +245,9 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheJdbcBlobStoreMultithreadedSelfTest.class);
         suite.addTestSuite(CacheJdbcPojoStoreTest.class);
         suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerSelfTest.class);
+        suite.addTestSuite(CacheJdbcPojoStoreOptimizedMarshallerWithSqlEscapeSelfTest.class);
         suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerSelfTest.class);
+        suite.addTestSuite(CacheJdbcPojoStoreBinaryMarshallerWithSqlEscapeSelfTest.class);
         suite.addTestSuite(CacheJdbcPojoStoreMultitreadedSelfTest.class);
         suite.addTestSuite(GridCacheBalancingStoreSelfTest.class);
         suite.addTestSuite(GridCacheAffinityApiSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml
index 3480ece..d62e76b 100644
--- a/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml
+++ b/modules/spring/src/test/config/jdbc-pojo-store-builtin.xml
@@ -70,7 +70,7 @@
                                         <property name="keyFields">
                                             <list>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="id"/>
+                                                    <property name="databaseFieldName" value="ID"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.INTEGER"/>
                                                     </property>
@@ -83,7 +83,7 @@
                                         <property name="valueFields">
                                             <list>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="name"/>
+                                                    <property name="databaseFieldName" value="NAME"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.VARCHAR"/>
                                                     </property>
@@ -91,7 +91,7 @@
                                                     <property name="javaFieldType" value="java.lang.String"/>
                                                 </bean>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="city"/>
+                                                    <property name="databaseFieldName" value="CITY"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.VARCHAR"/>
                                                     </property>
@@ -112,7 +112,7 @@
                                         <property name="keyFields">
                                             <list>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="id"/>
+                                                    <property name="databaseFieldName" value="ID"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.INTEGER"/>
                                                     </property>
@@ -125,7 +125,7 @@
                                         <property name="valueFields">
                                             <list>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="name"/>
+                                                    <property name="databaseFieldName" value="NAME"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.VARCHAR"/>
                                                     </property>
@@ -133,7 +133,7 @@
                                                     <property name="javaFieldType" value="java.lang.String"/>
                                                 </bean>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="birthday"/>
+                                                    <property name="databaseFieldName" value="BIRTHDAY"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.DATE"/>
                                                     </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/spring/src/test/config/jdbc-pojo-store-obj.xml
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml
index ee761d0..83e0548 100644
--- a/modules/spring/src/test/config/jdbc-pojo-store-obj.xml
+++ b/modules/spring/src/test/config/jdbc-pojo-store-obj.xml
@@ -70,7 +70,7 @@
                                         <property name="keyFields">
                                             <list>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="id"/>
+                                                    <property name="databaseFieldName" value="ID"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.INTEGER"/>
                                                     </property>
@@ -83,7 +83,7 @@
                                         <property name="valueFields">
                                             <list>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="name"/>
+                                                    <property name="databaseFieldName" value="NAME"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.VARCHAR"/>
                                                     </property>
@@ -91,7 +91,7 @@
                                                     <property name="javaFieldType" value="java.lang.String"/>
                                                 </bean>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="city"/>
+                                                    <property name="databaseFieldName" value="CITY"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.VARCHAR"/>
                                                     </property>
@@ -112,7 +112,7 @@
                                         <property name="keyFields">
                                             <list>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="id"/>
+                                                    <property name="databaseFieldName" value="ID"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.INTEGER"/>
                                                     </property>
@@ -125,7 +125,7 @@
                                         <property name="valueFields">
                                             <list>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="name"/>
+                                                    <property name="databaseFieldName" value="NAME"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.VARCHAR"/>
                                                     </property>
@@ -133,7 +133,7 @@
                                                     <property name="javaFieldType" value="java.lang.String"/>
                                                 </bean>
                                                 <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
-                                                    <property name="databaseFieldName" value="birthday"/>
+                                                    <property name="databaseFieldName" value="BIRTHDAY"/>
                                                     <property name="databaseFieldType">
                                                         <util:constant static-field="java.sql.Types.DATE"/>
                                                     </property>

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java
index da70042..6ecf67f 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactorySelfTest.java
@@ -34,7 +34,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.h2.jdbcx.JdbcDataSource;
 
 /**
- * Test for Cache jdbc blob store factory.
+ * Test for Cache JDBC blob store factory.
  */
 public class CacheJdbcBlobStoreFactorySelfTest extends GridCommonAbstractTest {
     /** Cache name. */
@@ -107,16 +107,16 @@ public class CacheJdbcBlobStoreFactorySelfTest extends GridCommonAbstractTest {
 
     /**
      * @param cache Ignite cache.
-     * @param dataSrcClass Data source class.
+     * @param dataSrcCls Data source class.
      * @throws Exception If store parameters is not the same as in configuration xml.
      */
-    private void checkStore(IgniteCache<Integer, String> cache, Class<?> dataSrcClass) throws Exception {
+    private void checkStore(IgniteCache<Integer, String> cache, Class<?> dataSrcCls) throws Exception {
         CacheJdbcBlobStore store = (CacheJdbcBlobStore) cache.getConfiguration(CacheConfiguration.class).
             getCacheStoreFactory().create();
 
         assertEquals(USER_NAME, GridTestUtils.getFieldValue(store, CacheJdbcBlobStore.class, "user"));
 
-        assertEquals(dataSrcClass,
+        assertEquals(dataSrcCls,
             GridTestUtils.getFieldValue(store, CacheJdbcBlobStore.class, "dataSrc").getClass());
     }
 
@@ -135,7 +135,7 @@ public class CacheJdbcBlobStoreFactorySelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public Connection getConnection(String username, String password) throws SQLException {
+        @Override public Connection getConnection(String username, String pwd) throws SQLException {
             return null;
         }
 
@@ -174,4 +174,4 @@ public class CacheJdbcBlobStoreFactorySelfTest extends GridCommonAbstractTest {
             return false;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
index dfa1452..e354935 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactorySelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.cache.store.jdbc;
 
-import java.io.Serializable;
 import java.util.Collection;
 import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
@@ -32,7 +31,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.h2.jdbcx.JdbcDataSource;
 
 /**
- * Test for Cache jdbc blob store factory.
+ * Test for Cache JDBC POJO store factory.
  */
 public class CacheJdbcPojoStoreFactorySelfTest extends GridCommonAbstractTest {
     /** Cache name. */
@@ -117,14 +116,14 @@ public class CacheJdbcPojoStoreFactorySelfTest extends GridCommonAbstractTest {
 
     /**
      * @param cache Ignite cache.
-     * @param dataSrcClass Data source class.
+     * @param dataSrcCls Data source class.
      * @throws Exception If store parameters is not the same as in configuration xml.
      */
-    private void checkStore(IgniteCache<Integer, String> cache, Class<?> dataSrcClass) throws Exception {
+    private void checkStore(IgniteCache<Integer, String> cache, Class<?> dataSrcCls) throws Exception {
         CacheJdbcPojoStore store = (CacheJdbcPojoStore)cache.getConfiguration(CacheConfiguration.class).
             getCacheStoreFactory().create();
 
-        assertEquals(dataSrcClass,
+        assertEquals(dataSrcCls,
             GridTestUtils.getFieldValue(store, CacheAbstractJdbcStore.class, "dataSrc").getClass());
     }
 
@@ -133,6 +132,11 @@ public class CacheJdbcPojoStoreFactorySelfTest extends GridCommonAbstractTest {
      */
     public static class DummyDialect implements JdbcDialect {
         /** {@inheritDoc} */
+        @Override public String escape(String ident) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
         @Override public String loadCacheSelectRangeQuery(String fullTblName, Collection<String> keyCols) {
             return null;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java
index c712f88..7de0cfe 100644
--- a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.cache.store.jdbc;
 
 import java.net.URL;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteComponentType;
 import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
@@ -25,13 +26,14 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.Marshaller;
 
 /**
- * Tests for {@code PojoCacheStore} created via XML.
+ * Tests for {@link CacheJdbcPojoStore} created via XML.
  */
 public class CachePojoStoreXmlSelfTest extends CacheJdbcPojoStoreAbstractSelfTest {
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        String path = builtinKeys ?  "modules/spring/src/test/config/jdbc-pojo-store-builtin.xml" :
-            "modules/spring/src/test/config/jdbc-pojo-store-obj.xml";
+        String path = builtinKeys
+            ? "modules/spring/src/test/config/jdbc-pojo-store-builtin.xml"
+            : "modules/spring/src/test/config/jdbc-pojo-store-obj.xml";
 
         URL url = U.resolveIgniteUrl(path);
 
@@ -39,6 +41,11 @@ public class CachePojoStoreXmlSelfTest extends CacheJdbcPojoStoreAbstractSelfTes
 
         IgniteConfiguration cfg = spring.loadConfigurations(url).get1().iterator().next();
 
+        if (sqlEscapeAll()) {
+            for (CacheConfiguration ccfg : cfg.getCacheConfiguration())
+                ((CacheJdbcPojoStoreFactory)ccfg.getCacheStoreFactory()).setSqlEscapeAll(true);
+        }
+
         cfg.setGridName(gridName);
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlWithSqlEscapeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlWithSqlEscapeSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlWithSqlEscapeSelfTest.java
new file mode 100644
index 0000000..e801682
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/jdbc/CachePojoStoreXmlWithSqlEscapeSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.cache.store.jdbc;
+
+/**
+ * Tests for {@link CacheJdbcPojoStore} created via XML.
+ */
+public class CachePojoStoreXmlWithSqlEscapeSelfTest extends CachePojoStoreXmlSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean sqlEscapeAll() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/63888bd9/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
index 67b117d..c4a4b75 100644
--- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
+++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
@@ -22,6 +22,8 @@ import org.apache.ignite.cache.spring.GridSpringCacheManagerSelfTest;
 import org.apache.ignite.cache.spring.SpringCacheManagerContextInjectionTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactorySelfTest;
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactorySelfTest;
+import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlSelfTest;
+import org.apache.ignite.cache.store.jdbc.CachePojoStoreXmlWithSqlEscapeSelfTest;
 import org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListenerSelfTest;
 import org.apache.ignite.internal.GridFactorySelfTest;
 import org.apache.ignite.internal.GridSpringBeanSerializationSelfTest;
@@ -64,8 +66,9 @@ public class IgniteSpringTestSuite extends TestSuite {
         suite.addTestSuite(CacheSpringStoreSessionListenerSelfTest.class);
 
         suite.addTestSuite(CacheJdbcBlobStoreFactorySelfTest.class);
-
         suite.addTestSuite(CacheJdbcPojoStoreFactorySelfTest.class);
+        suite.addTestSuite(CachePojoStoreXmlSelfTest.class);
+        suite.addTestSuite(CachePojoStoreXmlWithSqlEscapeSelfTest.class);
 
         suite.addTest(new TestSuite(GridSpringTransactionManagerSelfTest.class));