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

[32/35] ignite git commit: IGNITE-3815 Added check that key type registered in store or cache. Added test. IGNITE-3816 Fixed column index search logic and added propper exception.

IGNITE-3815 Added check that key type registered in store or cache. Added test.
IGNITE-3816 Fixed column index search logic and added propper exception.


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

Branch: refs/heads/ignite-3199-1
Commit: 98914fef1565dee660b1d743c45c91f3c0bf0afe
Parents: 409f043
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Sep 14 15:59:42 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Sep 14 15:59:42 2016 +0700

----------------------------------------------------------------------
 .../store/jdbc/CacheAbstractJdbcStore.java      | 71 +++++++++++++-------
 .../cache/store/jdbc/CacheJdbcPojoStore.java    | 22 +++---
 .../CacheJdbcPojoStoreAbstractSelfTest.java     | 70 ++++++++++++++++---
 ...heJdbcPojoStoreBinaryMarshallerSelfTest.java | 12 ++--
 4 files changed, 126 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/98914fef/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 aad05e0..a33a1e6 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
@@ -574,8 +574,8 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
         try {
             if (kind == TypeKind.BUILT_IN) {
                 if (flds.length != 1)
-                    throw new CacheException("More than one field for built in type [cache=" +  U.maskName(cacheName) +
-                        ", type=" + typeName + " ]");
+                    throw new CacheException("More than one field for built in type " +
+                        "[cache=" + U.maskName(cacheName) + ", type=" + typeName + " ]");
 
                 JdbcTypeField field = flds[0];
 
@@ -588,16 +588,16 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
             else
                 for (JdbcTypeField field : flds) {
                     if (field.getDatabaseFieldName() == null)
-                        throw new CacheException("Missing database name in mapping description [cache=" +
-                            U.maskName(cacheName) + ", type=" + typeName + " ]");
+                        throw new CacheException("Missing database name in mapping description " +
+                            "[cache=" + U.maskName(cacheName) + ", type=" + typeName + " ]");
 
                     if (field.getJavaFieldName() == null)
-                        throw new CacheException("Missing field name in mapping description [cache=" +
-                            U.maskName(cacheName) + ", type=" + typeName + " ]");
+                        throw new CacheException("Missing field name in mapping description " +
+                            "[cache=" + U.maskName(cacheName) + ", type=" + typeName + " ]");
 
                     if (field.getJavaFieldType() == null)
-                        throw new CacheException("Missing field type in mapping description [cache=" +
-                            U.maskName(cacheName) + ", type=" + typeName + " ]");
+                        throw new CacheException("Missing field type in mapping description " +
+                            "[cache=" + U.maskName(cacheName) + ", type=" + typeName + " ]");
                 }
         }
         catch (ClassNotFoundException e) {
@@ -778,6 +778,23 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
         return em;
     }
 
+    /**
+     * Find column index by database name.
+     *
+     * @param loadColIdxs Select query columns indexes.
+     * @param dbName Column name in database.
+     * @return Column index.
+     * @throws IllegalStateException if column not found.
+     */
+    protected Integer columnIndex(Map<String, Integer> loadColIdxs, String dbName) {
+        Integer colIdx = loadColIdxs.get(dbName.toUpperCase());
+
+        if (colIdx == null)
+            throw new IllegalStateException("Failed to find column index for database field: " + dbName);
+
+        return colIdx;
+    }
+
     /** {@inheritDoc} */
     @Override public void loadCache(final IgniteBiInClosure<K, V> clo, @Nullable Object... args)
         throws CacheLoaderException {
@@ -800,7 +817,15 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
                     log.debug("Start loading entries from db using user queries from arguments...");
 
                 for (int i = 0; i < args.length; i += 2) {
-                    String keyType = args[i].toString();
+                    final String keyType = args[i].toString();
+
+                    if (!F.exist(mappings.values(), new IgnitePredicate<EntryMapping>() {
+                        @Override public boolean apply(EntryMapping em) {
+                            return em.keyType().equals(keyType);
+                        }
+                    }))
+                        throw new CacheLoaderException("Provided key type is not found in store or cache configuration " +
+                            "[cache=" + U.maskName(cacheName) + ", key=" + keyType + "]");
 
                     String selQry = args[i + 1].toString();
 
@@ -827,7 +852,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
                             if (rs.next()) {
                                 if (log.isDebugEnabled())
-                                    log.debug("Multithread loading entries from db [cache=" +  U.maskName(cacheName) +
+                                    log.debug("Multithread loading entries from db [cache=" + U.maskName(cacheName) +
                                         ", keyType=" + em.keyType() + " ]");
 
                                 int keyCnt = em.keyCols.size();
@@ -856,8 +881,8 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
                             }
                         }
                         catch (SQLException e) {
-                            log.warning("Failed to load entries from db in multithreaded mode [cache=" +  U.maskName(cacheName) +
-                                ", keyType=" + em.keyType() + " ]", e);
+                            log.warning("Failed to load entries from db in multithreaded mode " +
+                                "[cache=" + U.maskName(cacheName) + ", keyType=" + em.keyType() + " ]", e);
                         }
                         finally {
                             U.closeQuiet(conn);
@@ -865,7 +890,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
                     }
 
                     if (log.isDebugEnabled())
-                        log.debug("Single thread loading entries from db [cache=" +  U.maskName(cacheName) +
+                        log.debug("Single thread loading entries from db [cache=" + U.maskName(cacheName) +
                             ", keyType=" + em.keyType() + " ]");
 
                     futs.add(pool.submit(loadCacheFull(em, clo)));
@@ -876,7 +901,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
                 U.get(fut);
 
             if (log.isDebugEnabled())
-                log.debug("Cache loaded from db: " +  U.maskName(cacheName));
+                log.debug("Cache loaded from db: " + U.maskName(cacheName));
         }
         catch (IgniteCheckedException e) {
             throw new CacheLoaderException("Failed to load cache: " + U.maskName(cacheName), e.getCause());
@@ -1143,7 +1168,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
                         if (currKeyTypeId == null || !currKeyTypeId.equals(keyTypeId)) {
                             if (mergeStmt != null) {
                                 if (log.isDebugEnabled())
-                                    log.debug("Write entries to db [cache=" +  U.maskName(cacheName) +
+                                    log.debug("Write entries to db [cache=" + U.maskName(cacheName) +
                                         ", keyType=" + em.keyType() + ", cnt=" + prepared + "]");
 
                                 executeBatch(em, mergeStmt, "writeAll", fromIdx, prepared, lazyEntries);
@@ -1168,7 +1193,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
                         if (++prepared % batchSize == 0) {
                             if (log.isDebugEnabled())
-                                log.debug("Write entries to db [cache=" +  U.maskName(cacheName) +
+                                log.debug("Write entries to db [cache=" + U.maskName(cacheName) +
                                     ", keyType=" + em.keyType() + ", cnt=" + prepared + "]");
 
                             executeBatch(em, mergeStmt, "writeAll", fromIdx, prepared, lazyEntries);
@@ -1181,7 +1206,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
                     if (mergeStmt != null && prepared % batchSize != 0) {
                         if (log.isDebugEnabled())
-                            log.debug("Write entries to db [cache=" +  U.maskName(cacheName) +
+                            log.debug("Write entries to db [cache=" + U.maskName(cacheName) +
                                 ", keyType=" + em.keyType() + ", cnt=" + prepared + "]");
 
                         executeBatch(em, mergeStmt, "writeAll", fromIdx, prepared, lazyEntries);
@@ -1194,8 +1219,8 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
             }
             else {
                 if (log.isDebugEnabled())
-                    log.debug("Write entries to db one by one using update and insert statements [cache=" +
-                        U.maskName(cacheName) + ", cnt=" + entries.size() + "]");
+                    log.debug("Write entries to db one by one using update and insert statements " +
+                        "[cache=" + U.maskName(cacheName) + ", cnt=" + entries.size() + "]");
 
                 PreparedStatement insStmt = null;
 
@@ -1359,7 +1384,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
                 if (!currKeyTypeId.equals(keyTypeId)) {
                     if (log.isDebugEnabled())
-                        log.debug("Delete entries from db [cache=" +  U.maskName(cacheName) +
+                        log.debug("Delete entries from db [cache=" + U.maskName(cacheName) +
                             ", keyType=" + em.keyType() + ", cnt=" + prepared + "]");
 
                     executeBatch(em, delStmt, "deleteAll", fromIdx, prepared, lazyKeys);
@@ -1377,7 +1402,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
                 if (++prepared % batchSize == 0) {
                     if (log.isDebugEnabled())
-                        log.debug("Delete entries from db [cache=" +  U.maskName(cacheName) +
+                        log.debug("Delete entries from db [cache=" + U.maskName(cacheName) +
                             ", keyType=" + em.keyType() + ", cnt=" + prepared + "]");
 
                     executeBatch(em, delStmt, "deleteAll", fromIdx, prepared, lazyKeys);
@@ -1390,7 +1415,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
 
             if (delStmt != null && prepared % batchSize != 0) {
                 if (log.isDebugEnabled())
-                    log.debug("Delete entries from db [cache=" +  U.maskName(cacheName) +
+                    log.debug("Delete entries from db [cache=" + U.maskName(cacheName) +
                         ", keyType=" + em.keyType() + ", cnt=" + prepared + "]");
 
                 executeBatch(em, delStmt, "deleteAll", fromIdx, prepared, lazyKeys);
@@ -1766,7 +1791,7 @@ public abstract class CacheAbstractJdbcStore<K, V> implements CacheStore<K, V>,
             int idx = 1;
 
             for (String col : cols)
-                loadColIdxs.put(col, idx++);
+                loadColIdxs.put(col.toUpperCase(), idx++);
 
             loadCacheQry = dialect.loadCacheQuery(fullTblName, cols);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/98914fef/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
index b9a3118..798b84a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStore.java
@@ -151,7 +151,9 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
         try {
             JdbcTypeField field = fields[0];
 
-            return getColumnValue(rs, loadColIdxs.get(field.getDatabaseFieldName()), field.getJavaFieldType());
+            Integer colIdx = columnIndex(loadColIdxs, field.getDatabaseFieldName());
+
+            return getColumnValue(rs, colIdx, field.getJavaFieldType());
         }
         catch (SQLException e) {
             throw new CacheLoaderException("Failed to read object of class: " + typeName, e);
@@ -195,9 +197,9 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                     throw new IllegalStateException("Failed to find property in POJO class [type=" + typeName +
                         ", prop=" + fldJavaName + "]");
 
-                String fldDbName = fld.getDatabaseFieldName();
+                String dbName = fld.getDatabaseFieldName();
 
-                Integer colIdx = loadColIdxs.get(fldDbName);
+                Integer colIdx = columnIndex(loadColIdxs, dbName);
 
                 try {
                     Object colVal = getColumnValue(rs, colIdx, fld.getJavaFieldType());
@@ -207,12 +209,12 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
                     }
                     catch (Exception e) {
                         throw new CacheLoaderException("Failed to set property in POJO class [type=" + typeName +
-                            ", prop=" + fldJavaName + ", col=" + colIdx + ", dbName=" + fldDbName + "]", e);
+                            ", prop=" + fldJavaName + ", col=" + colIdx + ", dbName=" + dbName + "]", e);
                     }
                 }
                 catch (SQLException e) {
                     throw new CacheLoaderException("Failed to read object property [type= " + typeName +
-                        ", prop=" + fldJavaName + ", col=" + colIdx + ", dbName=" + fldDbName + "]", e);
+                        ", prop=" + fldJavaName + ", col=" + colIdx + ", dbName=" + dbName + "]", e);
                 }
             }
 
@@ -244,7 +246,7 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
             Collection<Object> hashValues = calcHash ? new ArrayList<>(hashFields.size()) : null;
 
             for (JdbcTypeField field : fields) {
-                Integer colIdx = loadColIdxs.get(field.getDatabaseFieldName());
+                Integer colIdx = columnIndex(loadColIdxs, field.getDatabaseFieldName());
 
                 Object colVal = getColumnValue(rs, colIdx, field.getJavaFieldType());
 
@@ -370,8 +372,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
          *
          * @param obj Object to get property value from.
          * @return Property value.
-         * @throws IllegalAccessException
-         * @throws InvocationTargetException
+         * @throws IllegalAccessException If failed to get value from property or failed access to property via reflection.
+         * @throws InvocationTargetException If failed access to property via reflection.
          */
         private Object get(Object obj) throws IllegalAccessException, InvocationTargetException {
             if (getter != null)
@@ -388,8 +390,8 @@ public class CacheJdbcPojoStore<K, V> extends CacheAbstractJdbcStore<K, V> {
          *
          * @param obj Object to set property value to.
          * @param val New property value to set.
-         * @throws IllegalAccessException
-         * @throws InvocationTargetException
+         * @throws IllegalAccessException If failed to set property value or failed access to property via reflection.
+         * @throws InvocationTargetException If failed access to property via reflection.
          */
         private void set(Object obj, Object val) throws IllegalAccessException, InvocationTargetException {
             if (setter != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/98914fef/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 41a6136..e8592d7 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
@@ -26,6 +26,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Types;
 import java.util.Random;
+import javax.cache.integration.CacheLoaderException;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.store.jdbc.dialect.H2Dialect;
 import org.apache.ignite.cache.store.jdbc.model.Person;
@@ -77,7 +78,7 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
 
     /**
      * @return Connection to test in-memory H2 database.
-     * @throws SQLException
+     * @throws SQLException if failed to connect.
      */
     protected Connection getConnection() throws SQLException {
         return DriverManager.getConnection(DFLT_CONN_URL, "sa", "");
@@ -164,9 +165,9 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
 
         storeTypes[0].setValueType("org.apache.ignite.cache.store.jdbc.model.Organization" + (noValClasses ? "1" : ""));
         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, "Id", Integer.class, "id"),
+            new JdbcTypeField(Types.VARCHAR, "Name", String.class, "name"),
+            new JdbcTypeField(Types.VARCHAR, "City", String.class, "city"));
 
         storeTypes[1] = new JdbcType();
         storeTypes[1].setCacheName(CACHE_NAME);
@@ -273,7 +274,7 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
      * @param noKeyCls {@code True} if keys classes are not on class path.
      * @param noValCls {@code True} if values classes are not on class path.
      * @param trn {@code True} if cache should be started in transactional mode.
-     * @throws Exception
+     * @throws Exception If failed to start grid.
      */
     protected void startTestGrid(boolean builtin, boolean noKeyCls, boolean noValCls, boolean trn) throws Exception {
         builtinKeys = builtin;
@@ -287,7 +288,7 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
     /**
      * Check that data was loaded correctly.
      */
-    protected void checkCacheContent() {
+    protected void checkCacheLoad() {
         IgniteCache<Object, Object> c1 = grid().cache(CACHE_NAME);
 
         c1.loadCache(null);
@@ -296,12 +297,32 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
     }
 
     /**
+     * Check that data was loaded correctly.
+     */
+    protected void checkCacheLoadWithSql() {
+        IgniteCache<Object, Object> c1 = grid().cache(CACHE_NAME);
+
+        c1.loadCache(null, "org.apache.ignite.cache.store.jdbc.model.PersonKey", "select id, org_id, name, birthday from Person");
+
+        assertEquals(PERSON_CNT, c1.size());
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testLoadCache() throws Exception {
         startTestGrid(false, false, false, false);
 
-        checkCacheContent();
+        checkCacheLoad();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheWithSql() throws Exception {
+        startTestGrid(false, false, false, false);
+
+        checkCacheLoadWithSql();
     }
 
     /**
@@ -310,7 +331,16 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
     public void testLoadCacheTx() throws Exception {
         startTestGrid(false, false, false, true);
 
-        checkCacheContent();
+        checkCacheLoad();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheWithSqlTx() throws Exception {
+        startTestGrid(false, false, false, true);
+
+        checkCacheLoadWithSql();
     }
 
     /**
@@ -319,7 +349,7 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
     public void testLoadCachePrimitiveKeys() throws Exception {
         startTestGrid(true, false, false, false);
 
-        checkCacheContent();
+        checkCacheLoad();
     }
 
     /**
@@ -328,7 +358,7 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
     public void testLoadCachePrimitiveKeysTx() throws Exception {
         startTestGrid(true, false, false, true);
 
-        checkCacheContent();
+        checkCacheLoad();
     }
 
     /**
@@ -431,4 +461,24 @@ public abstract class CacheJdbcPojoStoreAbstractSelfTest extends GridCommonAbstr
 
         checkPut();
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadNotRegisteredType() throws Exception {
+        startTestGrid(false, false, false, false);
+
+        IgniteCache<Object, Object> c1 = grid().cache(CACHE_NAME);
+
+        try {
+            c1.loadCache(null, "PersonKeyWrong", "SELECT * FROM Person");
+        }
+        catch (CacheLoaderException e) {
+            String msg = e.getMessage();
+
+            assertTrue("Unexpected exception: " + msg,
+                ("Provided key type is not found in store or cache configuration " +
+                    "[cache=" + CACHE_NAME + ", key=PersonKeyWrong]").equals(msg));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/98914fef/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 b825a24..f998027 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
@@ -35,7 +35,7 @@ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoSto
     public void testLoadCacheNoKeyClasses() throws Exception {
         startTestGrid(false, true, false, false);
 
-        checkCacheContent();
+        checkCacheLoad();
     }
 
     /**
@@ -44,7 +44,7 @@ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoSto
     public void testLoadCacheNoKeyClassesTx() throws Exception {
         startTestGrid(false, true, false, true);
 
-        checkCacheContent();
+        checkCacheLoad();
     }
 
     /**
@@ -53,7 +53,7 @@ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoSto
     public void testLoadCacheNoValueClasses() throws Exception {
         startTestGrid(false, false, true, false);
 
-        checkCacheContent();
+        checkCacheLoad();
     }
 
     /**
@@ -62,7 +62,7 @@ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoSto
     public void testLoadCacheNoValueClassesTx() throws Exception {
         startTestGrid(false, false, true, true);
 
-        checkCacheContent();
+        checkCacheLoad();
     }
 
     /**
@@ -71,7 +71,7 @@ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoSto
     public void testLoadCacheNoKeyAndValueClasses() throws Exception {
         startTestGrid(false, true, true, false);
 
-        checkCacheContent();
+        checkCacheLoad();
     }
 
     /**
@@ -80,6 +80,6 @@ public class CacheJdbcPojoStoreBinaryMarshallerSelfTest extends CacheJdbcPojoSto
     public void testLoadCacheNoKeyAndValueClassesTx() throws Exception {
         startTestGrid(false, true, true, true);
 
-        checkCacheContent();
+        checkCacheLoad();
     }
 }