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

[01/29] ignite git commit: IGNITE-6554 Atomic cache remove operations are not logged to WAL - Fixes #2800.

Repository: ignite
Updated Branches:
  refs/heads/ignite-5937 5c195c3be -> c1b2c03dc


IGNITE-6554 Atomic cache remove operations are not logged to WAL - Fixes #2800.

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


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

Branch: refs/heads/ignite-5937
Commit: 20033c7324c49212ad9543e13e9a5ac1e4bf5b68
Parents: 62f3c4c
Author: dpavlov <dp...@gridgain.com>
Authored: Wed Oct 4 15:05:48 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Oct 4 15:05:48 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |   2 +
 .../db/wal/reader/IgniteWalReaderTest.java      | 159 ++++++++++++++++++-
 2 files changed, 158 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/20033c73/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 7b60b9c..54b8dc3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -4486,6 +4486,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (updateCntr != null)
                 updateCntr0 = updateCntr;
 
+            entry.logUpdate(op, null, newVer, 0, updateCntr0);
+
             if (oldVal != null) {
                 assert !entry.deletedUnlocked();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/20033c73/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index b9c60b2..6db2784 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -24,6 +24,7 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.io.Serializable;
 import java.util.Collection;
+import java.util.EnumMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -61,6 +62,7 @@ import org.apache.ignite.internal.pagemem.wal.record.TxRecord;
 import org.apache.ignite.internal.pagemem.wal.record.UnwrapDataEntry;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.persistence.wal.reader.IgniteWalIteratorFactory;
@@ -75,8 +77,9 @@ import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED;
-import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.genNewStyleSubfolderName;
+import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsConsistentIdProcessor.genNewStyleSubfolderName;
 
 /**
  * Test suite for WAL segments reader and event generator.
@@ -109,6 +112,9 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
      */
     private int archiveIncompleteSegmentAfterInactivityMs;
 
+    /** Custom wal mode. */
+    private WALMode customWalMode;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         final IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -143,7 +149,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
         pCfg.setWalHistorySize(1);
         pCfg.setWalSegmentSize(1024 * 1024);
         pCfg.setWalSegments(WAL_SEGMENTS);
-        pCfg.setWalMode(WALMode.BACKGROUND);
+        pCfg.setWalMode(customWalMode != null ? customWalMode : WALMode.BACKGROUND);
 
         if (archiveIncompleteSegmentAfterInactivityMs > 0)
             pCfg.setWalAutoArchiveAfterInactivity(archiveIncompleteSegmentAfterInactivityMs);
@@ -709,6 +715,128 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Creates and fills cache with data.
+     *
+     * @param ig Ignite instance.
+     * @param mode Cache Atomicity Mode.
+     */
+    private void createCache2(Ignite ig, CacheAtomicityMode mode) {
+        if (log.isInfoEnabled())
+            log.info("Populating the cache...");
+
+        final CacheConfiguration<Integer, Organization> cfg = new CacheConfiguration<>("Org" + "11");
+        cfg.setAtomicityMode(mode);
+        final IgniteCache<Integer, Organization> cache = ig.getOrCreateCache(cfg).withKeepBinary();
+
+        try (Transaction tx = ig.transactions().txStart()) {
+            for (int i = 0; i < 10; i++) {
+
+                cache.put(i, new Organization(i, "Organization-" + i));
+
+                if (i % 2 == 0)
+                    cache.put(i, new Organization(i, "Organization-updated-" + i));
+
+                if (i % 5 == 0)
+                    cache.remove(i);
+            }
+            tx.commit();
+        }
+
+    }
+
+    /**
+     * Test if DELETE operation can be found for transactional cache after mixed cache operations including remove().
+     *
+     * @throws Exception if failed.
+     */
+    public void testRemoveOperationPresentedForDataEntry() throws Exception {
+        runRemoveOperationTest(CacheAtomicityMode.TRANSACTIONAL);
+    }
+
+    /**
+     * Test if DELETE operation can be found for atomic cache after mixed cache operations including remove().
+     *
+     * @throws Exception if failed.
+     */
+    public void testRemoveOperationPresentedForDataEntryForAtomic() throws Exception {
+        runRemoveOperationTest(CacheAtomicityMode.ATOMIC);
+    }
+
+
+    /**
+     * Test if DELETE operation can be found after mixed cache operations including remove().
+     *
+     * @throws Exception if failed.
+     * @param mode Cache Atomicity Mode.
+     */
+    private void runRemoveOperationTest(CacheAtomicityMode mode) throws Exception {
+        final Ignite ignite = startGrid("node0");
+
+        ignite.active(true);
+        createCache2(ignite, mode);
+        ignite.active(false);
+
+        final String subfolderName = genDbSubfolderName(ignite, 0);
+
+        stopGrid("node0");
+
+        final String workDir = U.defaultWorkDirectory();
+        final IgniteWalIteratorFactory factory = createWalIteratorFactory(subfolderName, workDir);
+
+        final StringBuilder builder = new StringBuilder();
+        final Map<GridCacheOperation, Integer> operationsFound = new EnumMap<>(GridCacheOperation.class);
+
+        scanIterateAndCount(factory, workDir, subfolderName, 0, 0, null, new Consumer<DataRecord>() {
+            @Override public void accept(DataRecord dataRecord) {
+                final List<DataEntry> entries = dataRecord.writeEntries();
+
+                builder.append("{");
+                for (DataEntry entry : entries) {
+                    final GridCacheOperation op = entry.op();
+                    final Integer cnt = operationsFound.get(op);
+
+                    operationsFound.put(op, cnt == null ? 1 : (cnt + 1));
+
+                    if (entry instanceof UnwrapDataEntry) {
+                        final UnwrapDataEntry entry1 = (UnwrapDataEntry)entry;
+
+                        builder.append(entry1.op()).append(" for ").append(entry1.unwrappedKey());
+                        final GridCacheVersion ver = entry.nearXidVersion();
+
+                        builder.append(", ");
+
+                        if (ver != null)
+                            builder.append("tx=").append(ver).append(", ");
+                    }
+                }
+
+                builder.append("}\n");
+            }
+        });
+
+        final Integer deletesFound = operationsFound.get(DELETE);
+
+        if (log.isInfoEnabled())
+            log.info(builder.toString());
+
+        assertTrue("Delete operations should be found in log: " + operationsFound,
+            deletesFound != null && deletesFound > 0);
+    }
+
+    @NotNull private IgniteWalIteratorFactory createWalIteratorFactory(String subfolderName,
+        String workDir) throws IgniteCheckedException {
+        final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false);
+        final File binaryMetaWithConsId = new File(binaryMeta, subfolderName);
+        final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false);
+
+        return new IgniteWalIteratorFactory(log,
+            PAGE_SIZE,
+            binaryMetaWithConsId,
+            marshallerMapping);
+    }
+
+
+    /**
      * @param values collection with numbers
      * @return sum of numbers
      */
@@ -776,7 +904,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
                             "; Key: " + unwrappedKeyObj +
                             "; Value: " + unwrappedValObj);
 
-                        if (cacheObjHnd != null && unwrappedKeyObj != null || unwrappedValObj != null)
+                        if (cacheObjHnd != null && (unwrappedKeyObj != null || unwrappedValObj != null))
                             cacheObjHnd.accept(unwrappedKeyObj, unwrappedValObj);
 
                         final Integer entriesUnderTx = entriesUnderTxFound.get(globalTxId);
@@ -967,4 +1095,29 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
                 '}';
         }
     }
+
+    /** Test class for storing in ignite */
+    private static class Organization {
+        /** Key. */
+        private final int key;
+        /** Name. */
+        private final String name;
+
+        /**
+         * @param key Key.
+         * @param name Name.
+         */
+        public Organization(int key, String name) {
+            this.key = key;
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "Organization{" +
+                "key=" + key +
+                ", name='" + name + '\'' +
+                '}';
+        }
+    }
 }


[14/29] ignite git commit: IGNITE-6553 Standalone WAL iterator fails to handle WAL delete data records - Fixes #2797.

Posted by sb...@apache.org.
IGNITE-6553 Standalone WAL iterator fails to handle WAL delete data records - Fixes #2797.


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

Branch: refs/heads/ignite-5937
Commit: 49b835812a607a116c4dbc99ce60ed1684229b34
Parents: 474479c
Author: dpavlov <dp...@gridgain.com>
Authored: Thu Oct 5 18:10:25 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Oct 5 18:11:58 2017 +0300

----------------------------------------------------------------------
 .../internal/pagemem/wal/record/DataEntry.java  | 13 ++++++------
 .../pagemem/wal/record/UnwrapDataEntry.java     | 22 +++++++++++++-------
 .../reader/StandaloneWalRecordsIterator.java    | 11 +++++++---
 .../db/wal/reader/IgniteWalReaderTest.java      | 14 ++++++++++---
 4 files changed, 41 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/49b83581/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java
index d4e0b9f..cb6b482 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataEntry.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Represents Data Entry ({@link #key}, {@link #val value}) pair update {@link #op operation} in WAL log.
@@ -32,13 +33,13 @@ public class DataEntry {
     @GridToStringInclude
     protected int cacheId;
 
-    /** Cache object key */
+    /** Cache object key. */
     protected KeyCacheObject key;
 
-    /** Cache object value */
-    protected CacheObject val;
+    /** Cache object value. May be {@code} null for {@link GridCacheOperation#DELETE} */
+    @Nullable protected CacheObject val;
 
-    /** Entry operation performed */
+    /** Entry operation performed. */
     @GridToStringInclude
     protected GridCacheOperation op;
 
@@ -67,7 +68,7 @@ public class DataEntry {
     /**
      * @param cacheId Cache ID.
      * @param key Key.
-     * @param val Value.
+     * @param val Value or null for delete operation.
      * @param op Operation.
      * @param nearXidVer Near transaction version.
      * @param writeVer Write version.
@@ -78,7 +79,7 @@ public class DataEntry {
     public DataEntry(
         int cacheId,
         KeyCacheObject key,
-        CacheObject val,
+        @Nullable CacheObject val,
         GridCacheOperation op,
         GridCacheVersion nearXidVer,
         GridCacheVersion writeVer,

http://git-wip-us.apache.org/repos/asf/ignite/blob/49b83581/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
index 678539d..dbcc651 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
@@ -31,13 +31,13 @@ public class UnwrapDataEntry extends DataEntry {
     /** Cache object value context. Context is used for unwrapping objects. */
     private final CacheObjectValueContext cacheObjValCtx;
 
-    /** Keep binary. This flag disables converting of non primitive types (BinaryObjects) */
+    /** Keep binary. This flag disables converting of non primitive types (BinaryObjects). */
     private boolean keepBinary;
 
     /**
      * @param cacheId Cache ID.
      * @param key Key.
-     * @param val Value.
+     * @param val Value or null for delete operation.
      * @param op Operation.
      * @param nearXidVer Near transaction version.
      * @param writeVer Write version.
@@ -45,7 +45,7 @@ public class UnwrapDataEntry extends DataEntry {
      * @param partId Partition ID.
      * @param partCnt Partition counter.
      * @param cacheObjValCtx cache object value context for unwrapping objects.
-     * @param keepBinary disable unwrapping for non primitive objects, Binary Objects would be returned instead
+     * @param keepBinary disable unwrapping for non primitive objects, Binary Objects would be returned instead.
      */
     public UnwrapDataEntry(
         final int cacheId,
@@ -66,39 +66,47 @@ public class UnwrapDataEntry extends DataEntry {
 
     /**
      * Unwraps key value from cache key object into primitive boxed type or source class. If client classes were used
-     * in key, call of this method requires classes to be available in classpath
+     * in key, call of this method requires classes to be available in classpath.
      *
-     * @return Key which was placed into cache. Or null if failed
+     * @return Key which was placed into cache. Or null if failed to convert.
      */
     public Object unwrappedKey() {
         try {
             if (keepBinary && key instanceof BinaryObject)
                 return key;
+
             Object unwrapped = key.value(cacheObjValCtx, false);
+
             if (unwrapped instanceof BinaryObject) {
                 if (keepBinary)
                     return unwrapped;
                 unwrapped = ((BinaryObject)unwrapped).deserialize();
             }
+
             return unwrapped;
         }
         catch (Exception e) {
             cacheObjValCtx.kernalContext().log(UnwrapDataEntry.class)
                 .error("Unable to convert key [" + key + "]", e);
+
             return null;
         }
     }
 
     /**
      * Unwraps value value from cache value object into primitive boxed type or source class. If client classes were
-     * used in key, call of this method requires classes to be available in classpath
+     * used in key, call of this method requires classes to be available in classpath.
      *
-     * @return Value which was placed into cache. Or null if failed
+     * @return Value which was placed into cache. Or null for delete operation or for failure.
      */
     public Object unwrappedValue() {
         try {
+            if (val == null)
+                return null;
+
             if (keepBinary && val instanceof BinaryObject)
                 return val;
+
             return val.value(cacheObjValCtx, false);
         }
         catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/49b83581/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
index 42bb410..a4d9e95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -335,12 +335,17 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
 
         if (dataEntry instanceof LazyDataEntry) {
             final LazyDataEntry lazyDataEntry = (LazyDataEntry)dataEntry;
+
             key = processor.toKeyCacheObject(fakeCacheObjCtx,
                 lazyDataEntry.getKeyType(),
                 lazyDataEntry.getKeyBytes());
-            val = processor.toCacheObject(fakeCacheObjCtx,
-                lazyDataEntry.getValType(),
-                lazyDataEntry.getValBytes());
+
+            final byte type = lazyDataEntry.getValType();
+
+            val = type == 0 ? null :
+                processor.toCacheObject(fakeCacheObjCtx,
+                    type,
+                    lazyDataEntry.getValBytes());
         }
         else {
             key = dataEntry.key();

http://git-wip-us.apache.org/repos/asf/ignite/blob/49b83581/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index 93df8b2..79387e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -356,6 +356,7 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
 
         cache0.putAll(values);
     }
+
     /**
      * Puts provided number of records to fill WAL under transactions
      *
@@ -912,8 +913,16 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
             deletesFound != null && deletesFound > 0);
     }
 
-    @NotNull private IgniteWalIteratorFactory createWalIteratorFactory(String subfolderName,
-        String workDir) throws IgniteCheckedException {
+    /**
+     * @param subfolderName Subfolder name.
+     * @param workDir Work directory.
+     * @return WAL iterator factory.
+     * @throws IgniteCheckedException If failed.
+     */
+    @NotNull private IgniteWalIteratorFactory createWalIteratorFactory(
+        String subfolderName,
+        String workDir
+    ) throws IgniteCheckedException {
         final File binaryMeta = U.resolveWorkDirectory(workDir, "binary_meta", false);
         final File binaryMetaWithConsId = new File(binaryMeta, subfolderName);
         final File marshallerMapping = U.resolveWorkDirectory(workDir, "marshaller", false);
@@ -924,7 +933,6 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
             marshallerMapping);
     }
 
-
     /**
      * @param values collection with numbers
      * @return sum of numbers


[11/29] ignite git commit: IGNITE-6556: JDBC thin: fixed setSchema() case sensitivity handling. This closes #2805.

Posted by sb...@apache.org.
IGNITE-6556: JDBC thin: fixed setSchema() case sensitivity handling. This closes #2805.


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

Branch: refs/heads/ignite-5937
Commit: 3c355e9f6e8c4c38bb14ac3d3c6ccfee378af1ab
Parents: df3c407
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Oct 5 16:45:31 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 5 16:45:31 2017 +0300

----------------------------------------------------------------------
 .../JdbcThinAbstractDmlStatementSelfTest.java   |  2 +-
 .../thin/JdbcThinAutoCloseServerCursorTest.java |  8 ++--
 .../jdbc/thin/JdbcThinComplexQuerySelfTest.java |  2 +-
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |  4 ++
 .../JdbcThinDynamicIndexAbstractSelfTest.java   |  2 +-
 .../jdbc/thin/JdbcThinEmptyCacheSelfTest.java   |  2 +-
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  4 +-
 .../JdbcThinMissingLongArrayResultsTest.java    |  2 +-
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  |  5 ++-
 .../thin/JdbcThinPreparedStatementSelfTest.java |  2 +-
 .../jdbc/thin/JdbcThinResultSetSelfTest.java    |  2 +-
 .../jdbc/thin/JdbcThinStatementSelfTest.java    | 46 +++++++++++---------
 .../internal/jdbc/thin/JdbcThinConnection.java  |  6 +--
 .../jdbc/thin/JdbcThinPreparedStatement.java    |  5 ++-
 .../internal/jdbc/thin/JdbcThinStatement.java   |  9 +++-
 15 files changed, 59 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractDmlStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractDmlStatementSelfTest.java
index 607f557..afe5e2e 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractDmlStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAbstractDmlStatementSelfTest.java
@@ -69,7 +69,7 @@ public abstract class JdbcThinAbstractDmlStatementSelfTest extends JdbcThinAbstr
 
         conn = DriverManager.getConnection(URL);
 
-        conn.setSchema(DEFAULT_CACHE_NAME);
+        conn.setSchema('"' + DEFAULT_CACHE_NAME + '"');
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAutoCloseServerCursorTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAutoCloseServerCursorTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAutoCloseServerCursorTest.java
index eff504b..db4ed3f 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAutoCloseServerCursorTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinAutoCloseServerCursorTest.java
@@ -115,7 +115,7 @@ public class JdbcThinAutoCloseServerCursorTest extends JdbcThinAbstractSelfTest
             cache.put(person.id, person);
 
         try (Connection conn = DriverManager.getConnection(URL)) {
-            conn.setSchema(CACHE_NAME);
+            conn.setSchema('"' + CACHE_NAME + '"');
 
             String sqlText = "select * from Person";
 
@@ -198,7 +198,7 @@ public class JdbcThinAutoCloseServerCursorTest extends JdbcThinAbstractSelfTest
      */
     public void testInsert() throws Exception {
         try (Connection conn = DriverManager.getConnection(URL)) {
-            conn.setSchema(CACHE_NAME);
+            conn.setSchema('"' + CACHE_NAME + '"');
 
             String sqlText = "insert into Person (_key, id, name, age) values (?, ?, ?, ?)";
 
@@ -233,7 +233,7 @@ public class JdbcThinAutoCloseServerCursorTest extends JdbcThinAbstractSelfTest
         cache.put(1, p);
 
         try (Connection conn = DriverManager.getConnection(URL)) {
-            conn.setSchema(CACHE_NAME);
+            conn.setSchema('"' + CACHE_NAME + '"');
 
             String sqlText = "update Person set age = age + 1";
 
@@ -258,7 +258,7 @@ public class JdbcThinAutoCloseServerCursorTest extends JdbcThinAbstractSelfTest
         cache.put(1, p);
 
         try (Connection conn = DriverManager.getConnection(URL)) {
-            conn.setSchema(CACHE_NAME);
+            conn.setSchema('"' + CACHE_NAME + '"');
 
             String sqlText = "delete Person where age = ?";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexQuerySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexQuerySelfTest.java
index 1714ab1..ad1e312 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexQuerySelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinComplexQuerySelfTest.java
@@ -113,7 +113,7 @@ public class JdbcThinComplexQuerySelfTest extends JdbcThinAbstractSelfTest {
     @Override protected void beforeTest() throws Exception {
         Connection conn = DriverManager.getConnection(URL);
 
-        conn.setSchema("pers");
+        conn.setSchema("\"pers\"");
 
         stmt = conn.createStatement();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
index 17ce686..fbbec0d 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
@@ -1633,6 +1633,10 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
 
             conn.setSchema(schema);
 
+            assertEquals(schema.toUpperCase(), conn.getSchema());
+
+            conn.setSchema('"' + schema + '"');
+
             assertEquals(schema, conn.getSchema());
 
             conn.close();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java
index 5089894..dbe93a4 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDynamicIndexAbstractSelfTest.java
@@ -309,7 +309,7 @@ public abstract class JdbcThinDynamicIndexAbstractSelfTest extends JdbcThinAbstr
         assertEquals(expSize, cache().size());
 
         try (Statement stmt = conn.createStatement()) {
-            conn.setSchema(DEFAULT_CACHE_NAME);
+            conn.setSchema('"' + DEFAULT_CACHE_NAME + '"');
 
             try (ResultSet rs = stmt.executeQuery("SELECT COUNT(*) from Person")) {
                 assertEquals(expSize, getSingleValue(rs));

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinEmptyCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinEmptyCacheSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinEmptyCacheSelfTest.java
index 1ede536..41b4338 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinEmptyCacheSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinEmptyCacheSelfTest.java
@@ -87,7 +87,7 @@ public class JdbcThinEmptyCacheSelfTest extends JdbcThinAbstractSelfTest {
     @Override protected void beforeTest() throws Exception {
         Connection conn = DriverManager.getConnection(URL);
 
-        conn.setSchema(CACHE_NAME);
+        conn.setSchema('"' + CACHE_NAME + '"');
 
         stmt = conn.createStatement();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index abbe4e1..6c20de0 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -156,7 +156,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     public void testResultSetMetaData() throws Exception {
         Connection conn = DriverManager.getConnection(URL);
 
-        conn.setSchema("pers");
+        conn.setSchema("\"pers\"");
 
         Statement stmt = conn.createStatement();
 
@@ -528,7 +528,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
      */
     public void testParametersMetadata() throws Exception {
         try (Connection conn = DriverManager.getConnection(URL)) {
-            conn.setSchema("pers");
+            conn.setSchema("\"pers\"");
 
             PreparedStatement stmt = conn.prepareStatement("select orgId from Person p where p.name > ? and p.orgId > ?");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMissingLongArrayResultsTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMissingLongArrayResultsTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMissingLongArrayResultsTest.java
index 1a53ab7..1f6e2c0 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMissingLongArrayResultsTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMissingLongArrayResultsTest.java
@@ -171,7 +171,7 @@ public class JdbcThinMissingLongArrayResultsTest extends JdbcThinAbstractSelfTes
     @SuppressWarnings({"EmptyTryBlock", "unused"})
     public void testDefaults() throws Exception {
         try (Connection conn = DriverManager.getConnection(URL)) {
-            conn.setSchema(CACHE_NAME);
+            conn.setSchema('"' + CACHE_NAME + '"');
 
             try (PreparedStatement st = conn.prepareStatement("SELECT * FROM VALUE")) {
                 ResultSet rs = st.executeQuery();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
index cab28f4..a1be582 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinNoDefaultSchemaTest.java
@@ -219,10 +219,13 @@ public class JdbcThinNoDefaultSchemaTest extends JdbcThinAbstractSelfTest {
                 }
             }, SQLException.class, "Failed to parse query");
 
-            conn.setSchema("cache1");
+            conn.setSchema("\"cache1\"");
 
             Statement stmt = conn.createStatement();
 
+            //Must not affects previous created statements.
+            conn.setSchema("invalid_schema");
+
             stmt.execute("select t._key, t._val from Integer t");
 
             ResultSet rs = stmt.getResultSet();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java
index 85eb1d3..c577853 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinPreparedStatementSelfTest.java
@@ -148,7 +148,7 @@ public class JdbcThinPreparedStatementSelfTest extends JdbcThinAbstractSelfTest
     @Override protected void beforeTest() throws Exception {
         conn = DriverManager.getConnection(URL);
 
-        conn.setSchema(DEFAULT_CACHE_NAME);
+        conn.setSchema('"' + DEFAULT_CACHE_NAME + '"');
 
         assert conn != null;
         assert !conn.isClosed();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinResultSetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinResultSetSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinResultSetSelfTest.java
index 5a3c5df..fd46cda 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinResultSetSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinResultSetSelfTest.java
@@ -120,7 +120,7 @@ public class JdbcThinResultSetSelfTest extends JdbcThinAbstractSelfTest {
     @Override protected void beforeTest() throws Exception {
         Connection conn = DriverManager.getConnection(URL);
 
-        conn.setSchema(DEFAULT_CACHE_NAME);
+        conn.setSchema('"' + DEFAULT_CACHE_NAME + '"');
 
         stmt = conn.createStatement();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java
index 5309465..82c0512 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinStatementSelfTest.java
@@ -104,7 +104,7 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
     @Override protected void beforeTest() throws Exception {
         conn = DriverManager.getConnection(URL);
 
-        conn.setSchema(DEFAULT_CACHE_NAME);
+        conn.setSchema('"' + DEFAULT_CACHE_NAME + '"');
 
         stmt = conn.createStatement();
 
@@ -444,6 +444,8 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
     public void testExecuteQueryMultipleOnlyDml() throws Exception {
         conn.setSchema(null);
 
+        Statement stmt0 = conn.createStatement();
+
         int stmtCnt = 10;
 
         StringBuilder sql = new StringBuilder("drop table if exists test; create table test(ID int primary key, NAME varchar(20)); ");
@@ -451,24 +453,24 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
         for (int i = 0; i < stmtCnt; ++i)
             sql.append("insert into test (ID, NAME) values (" + i + ", 'name_" + i +"'); ");
 
-        assert !stmt.execute(sql.toString());
+        assert !stmt0.execute(sql.toString());
 
         // DROP TABLE statement
-        assert stmt.getResultSet() == null;
-        assert stmt.getUpdateCount() == 0;
+        assert stmt0.getResultSet() == null;
+        assert stmt0.getUpdateCount() == 0;
 
         // CREATE TABLE statement
-        assert stmt.getResultSet() == null;
-        assert stmt.getUpdateCount() == 0;
+        assert stmt0.getResultSet() == null;
+        assert stmt0.getUpdateCount() == 0;
 
         for (int i = 0; i < stmtCnt; ++i) {
-            assert stmt.getMoreResults();
+            assert stmt0.getMoreResults();
 
-            assert stmt.getResultSet() == null;
-            assert stmt.getUpdateCount() == 1;
+            assert stmt0.getResultSet() == null;
+            assert stmt0.getUpdateCount() == 1;
         }
 
-        assert !stmt.getMoreResults();
+        assert !stmt0.getMoreResults();
     }
 
     /**
@@ -477,6 +479,8 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
     public void testExecuteQueryMultipleMixed() throws Exception {
         conn.setSchema(null);
 
+        Statement stmt0 = conn.createStatement();
+
         int stmtCnt = 10;
 
         StringBuilder sql = new StringBuilder("drop table if exists test; create table test(ID int primary key, NAME varchar(20)); ");
@@ -488,29 +492,29 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
                 sql.append(" select * from test where id < " + i + "; ");
         }
 
-        assert !stmt.execute(sql.toString());
+        assert !stmt0.execute(sql.toString());
 
         // DROP TABLE statement
-        assert stmt.getResultSet() == null;
-        assert stmt.getUpdateCount() == 0;
+        assert stmt0.getResultSet() == null;
+        assert stmt0.getUpdateCount() == 0;
 
         // CREATE TABLE statement
-        assert stmt.getResultSet() == null;
-        assert stmt.getUpdateCount() == 0;
+        assert stmt0.getResultSet() == null;
+        assert stmt0.getUpdateCount() == 0;
 
         boolean notEmptyResult = false;
 
         for (int i = 0; i < stmtCnt; ++i) {
-            assert stmt.getMoreResults();
+            assert stmt0.getMoreResults();
 
             if (i % 2 == 0) {
-                assert stmt.getResultSet() == null;
-                assert stmt.getUpdateCount() == 1;
+                assert stmt0.getResultSet() == null;
+                assert stmt0.getUpdateCount() == 1;
             }
             else {
-                assert stmt.getUpdateCount() == -1;
+                assert stmt0.getUpdateCount() == -1;
 
-                ResultSet rs = stmt.getResultSet();
+                ResultSet rs = stmt0.getResultSet();
 
                 int rowsCnt = 0;
 
@@ -526,7 +530,7 @@ public class JdbcThinStatementSelfTest extends JdbcThinAbstractSelfTest {
 
         assert notEmptyResult;
 
-        assert !stmt.getMoreResults();
+        assert !stmt0.getMoreResults();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
index d804750..5afed4e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
@@ -168,7 +168,7 @@ public class JdbcThinConnection implements Connection {
 
         checkCursorOptions(resSetType, resSetConcurrency, resSetHoldability);
 
-        JdbcThinStatement stmt  = new JdbcThinStatement(this, resSetHoldability);
+        JdbcThinStatement stmt  = new JdbcThinStatement(this, resSetHoldability, schema);
 
         if (timeout > 0)
             stmt.timeout(timeout);
@@ -197,7 +197,7 @@ public class JdbcThinConnection implements Connection {
         if (sql == null)
             throw new SQLException("SQL string cannot be null.");
 
-        JdbcThinPreparedStatement stmt = new JdbcThinPreparedStatement(this, sql, resSetHoldability);
+        JdbcThinPreparedStatement stmt = new JdbcThinPreparedStatement(this, sql, resSetHoldability, schema);
 
         if (timeout > 0)
             stmt.timeout(timeout);
@@ -592,7 +592,7 @@ public class JdbcThinConnection implements Connection {
     @Override public void setSchema(String schema) throws SQLException {
         ensureNotClosed();
 
-        this.schema = schema;
+        this.schema = normalizeSchema(schema);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
index fb2810d..23d3bbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
@@ -65,9 +65,10 @@ public class JdbcThinPreparedStatement extends JdbcThinStatement implements Prep
      * @param conn Connection.
      * @param sql SQL query.
      * @param resHoldability Result set holdability.
+     * @param schema Schema name.
      */
-    JdbcThinPreparedStatement(JdbcThinConnection conn, String sql, int resHoldability) {
-        super(conn, resHoldability);
+    JdbcThinPreparedStatement(JdbcThinConnection conn, String sql, int resHoldability, String schema) {
+        super(conn, resHoldability, schema);
 
         this.sql = sql;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c355e9f/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
index 603545b..d29df93 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
@@ -55,6 +55,9 @@ public class JdbcThinStatement implements Statement {
     /** JDBC Connection implementation. */
     protected JdbcThinConnection conn;
 
+    /** Schema name. */
+    private final String schema;
+
     /** Closed flag. */
     private boolean closed;
 
@@ -87,12 +90,14 @@ public class JdbcThinStatement implements Statement {
      *
      * @param conn JDBC connection.
      * @param resHoldability Result set holdability.
+     * @param schema Schema name.
      */
-    JdbcThinStatement(JdbcThinConnection conn, int resHoldability) {
+    JdbcThinStatement(JdbcThinConnection conn, int resHoldability, String schema) {
         assert conn != null;
 
         this.conn = conn;
         this.resHoldability = resHoldability;
+        this.schema = schema;
     }
 
     /** {@inheritDoc} */
@@ -122,7 +127,7 @@ public class JdbcThinStatement implements Statement {
         if (sql == null || sql.isEmpty())
             throw new SQLException("SQL query is empty.");
 
-        JdbcResult res0 = conn.sendRequest(new JdbcQueryExecuteRequest(stmtType, conn.getSchema(), pageSize,
+        JdbcResult res0 = conn.sendRequest(new JdbcQueryExecuteRequest(stmtType, schema, pageSize,
             maxRows, sql, args == null ? null : args.toArray(new Object[args.size()])));
 
         assert res0 != null;


[06/29] ignite git commit: Adjusted WAL docs

Posted by sb...@apache.org.
Adjusted WAL docs


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

Branch: refs/heads/ignite-5937
Commit: 8bd51d8af815f91089405dfdde3b1b81af7b2e59
Parents: e74163a
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Oct 4 13:51:53 2017 -0700
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Oct 4 13:51:53 2017 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/configuration/WALMode.java  | 9 +++++----
 1 file changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


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


[03/29] ignite git commit: IGNITE-6547 Support logging timestamp for WAL tx and data records - Fixes #2792.

Posted by sb...@apache.org.
IGNITE-6547 Support logging timestamp for WAL tx and data records - Fixes #2792.

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


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

Branch: refs/heads/ignite-5937
Commit: e6e1ca9a5a9155a550258b112415b65845d6bcef
Parents: 78f77b1
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Wed Oct 4 18:54:49 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Oct 4 18:54:49 2017 +0300

----------------------------------------------------------------------
 .../internal/pagemem/wal/record/DataRecord.java |  20 +-
 .../pagemem/wal/record/TimeStampRecord.java     |  57 ++++++
 .../internal/pagemem/wal/record/TxRecord.java   |  52 +++--
 .../reader/StandaloneWalRecordsIterator.java    |   2 +-
 .../wal/serializer/RecordDataV1Serializer.java  |   6 +-
 .../wal/serializer/RecordDataV2Serializer.java  |  49 ++++-
 .../wal/serializer/TxRecordSerializer.java      |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     |   3 +-
 .../db/wal/IgniteWalSerializerVersionTest.java  | 205 ++++++++++++++++++-
 9 files changed, 365 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java
index 0e92383..ac569bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/DataRecord.java
@@ -27,7 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
  * This record contains information about operation we want to do.
  * Contains operation type (put, remove) and (Key, Value, Version) for each {@link DataEntry}
  */
-public class DataRecord extends WALRecord {
+public class DataRecord extends TimeStampRecord {
     /** */
     @GridToStringInclude
     private List<DataEntry> writeEntries;
@@ -59,6 +59,24 @@ public class DataRecord extends WALRecord {
     }
 
     /**
+     * @param writeEntry Write entry.
+     * @param timestamp TimeStamp.
+     */
+    public DataRecord(DataEntry writeEntry, long timestamp) {
+        this(Collections.singletonList(writeEntry), timestamp);
+    }
+
+    /**
+     * @param writeEntries Write entries.
+     * @param timestamp TimeStamp.
+     */
+    public DataRecord(List<DataEntry> writeEntries, long timestamp) {
+        super(timestamp);
+
+        this.writeEntries = writeEntries;
+    }
+
+    /**
      * @return Collection of write entries.
      */
     public List<DataEntry> writeEntries() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TimeStampRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TimeStampRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TimeStampRecord.java
new file mode 100644
index 0000000..3f29dfd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TimeStampRecord.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagemem.wal.record;
+
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Base class for records with timeStamp.
+ * All records which support timeStamp should be inherited from {@code TimeStampRecord}.
+ */
+public abstract class TimeStampRecord extends WALRecord {
+    /** Timestamp. */
+    protected long timestamp;
+
+    /**
+     *
+     */
+    protected TimeStampRecord() {
+        timestamp = U.currentTimeMillis();
+    }
+
+    /**
+     * @param timestamp TimeStamp.
+     */
+    protected TimeStampRecord(long timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    /**
+     * @param timestamp TimeStamp.
+     */
+    public void timestamp(long timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    /**
+     * @return TimeStamp.
+     */
+    public long timestamp() {
+        return timestamp;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TxRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TxRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TxRecord.java
index ce1e28e..f933fa9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TxRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/TxRecord.java
@@ -28,7 +28,7 @@ import org.jetbrains.annotations.Nullable;
  * Logical data record indented for transaction (tx) related actions.<br>
  * This record is marker of begin, prepare, commit, and rollback transactions.
  */
-public class TxRecord extends WALRecord {
+public class TxRecord extends TimeStampRecord {
     /** Transaction state. */
     private TransactionState state;
 
@@ -49,28 +49,51 @@ public class TxRecord extends WALRecord {
     /** If transaction is remote, primary node for this backup node. */
     @Nullable private Object primaryNode;
 
-    /** Timestamp of Tx state change. */
-    private long timestamp;
-
     /**
      *
      * @param state Transaction state.
      * @param nearXidVer Transaction id.
      * @param writeVer Transaction entries write topology version.
      * @param participatingNodes Primary -> Backup nodes participating in transaction.
+     * @param primaryNode Primary node.
      */
-    public TxRecord(TransactionState state,
-                    GridCacheVersion nearXidVer,
-                    GridCacheVersion writeVer,
-                    @Nullable Map<Object, Collection<Object>> participatingNodes,
-                    @Nullable Object primaryNode,
-                    long timestamp) {
+    public TxRecord(
+        TransactionState state,
+        GridCacheVersion nearXidVer,
+        GridCacheVersion writeVer,
+        @Nullable Map<Object, Collection<Object>> participatingNodes,
+        @Nullable Object primaryNode
+    ) {
+        this.state = state;
+        this.nearXidVer = nearXidVer;
+        this.writeVer = writeVer;
+        this.participatingNodes = participatingNodes;
+        this.primaryNode = primaryNode;
+    }
+
+    /**
+     * @param state Transaction state.
+     * @param nearXidVer Transaction id.
+     * @param writeVer Transaction entries write topology version.
+     * @param participatingNodes Primary -> Backup nodes participating in transaction.
+     * @param primaryNode Primary node.
+     * @param timestamp TimeStamp.
+     */
+    public TxRecord(
+        TransactionState state,
+        GridCacheVersion nearXidVer,
+        GridCacheVersion writeVer,
+        @Nullable Map<Object, Collection<Object>> participatingNodes,
+        @Nullable Object primaryNode,
+        long timestamp
+    ) {
+        super(timestamp);
+
         this.state = state;
         this.nearXidVer = nearXidVer;
         this.writeVer = writeVer;
         this.participatingNodes = participatingNodes;
         this.primaryNode = primaryNode;
-        this.timestamp = timestamp;
     }
 
     /** {@inheritDoc} */
@@ -148,13 +171,6 @@ public class TxRecord extends WALRecord {
         return primaryNode;
     }
 
-    /**
-     * @return Timestamp of Tx state change in millis.
-     */
-    public long timestamp() {
-        return timestamp;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(TxRecord.class, this, "super", super.toString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
index f1258a0..24b2148 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -305,7 +305,7 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
 
             postProcessedEntries.add(postProcessedEntry);
         }
-        return new DataRecord(postProcessedEntries);
+        return new DataRecord(postProcessedEntries, dataRec.timestamp());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
index 8b5e6ba..e583df3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV1Serializer.java
@@ -384,7 +384,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
                 for (int i = 0; i < entryCnt; i++)
                     entries.add(readDataEntry(in));
 
-                res = new DataRecord(entries);
+                res = new DataRecord(entries, 0L);
 
                 break;
 
@@ -1322,7 +1322,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @param buf Buffer to write to.
      * @param entry Data entry.
      */
-    private static void putDataEntry(ByteBuffer buf, DataEntry entry) throws IgniteCheckedException {
+    static void putDataEntry(ByteBuffer buf, DataEntry entry) throws IgniteCheckedException {
         buf.putInt(entry.cacheId());
 
         if (!entry.key().putValue(buf))
@@ -1390,7 +1390,7 @@ public class RecordDataV1Serializer implements RecordDataSerializer {
      * @param in Input to read from.
      * @return Read entry.
      */
-    private DataEntry readDataEntry(ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException {
+    DataEntry readDataEntry(ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException {
         int cacheId = in.readInt();
 
         int keySize = in.readInt();

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
index 2b55c5f..c02f36e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordDataV2Serializer.java
@@ -19,7 +19,11 @@ package org.apache.ignite.internal.processors.cache.persistence.wal.serializer;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
+import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferBackedDataInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.RecordDataSerializer;
@@ -46,12 +50,35 @@ public class RecordDataV2Serializer implements RecordDataSerializer {
         if (record instanceof HeaderRecord)
             throw new UnsupportedOperationException("Getting size of header records is forbidden since version 2 of serializer");
 
-        return delegateSerializer.size(record);
+        switch (record.type()) {
+            case DATA_RECORD:
+                return delegateSerializer.size(record) + 8/*timestamp*/;
+
+            default:
+                return delegateSerializer.size(record);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public WALRecord readRecord(WALRecord.RecordType type, ByteBufferBackedDataInput in) throws IOException, IgniteCheckedException {
-        return delegateSerializer.readRecord(type, in);
+    @Override public WALRecord readRecord(
+        WALRecord.RecordType type,
+        ByteBufferBackedDataInput in
+    ) throws IOException, IgniteCheckedException {
+        switch (type) {
+            case DATA_RECORD:
+                int entryCnt = in.readInt();
+                long timeStamp = in.readLong();
+
+                List<DataEntry> entries = new ArrayList<>(entryCnt);
+
+                for (int i = 0; i < entryCnt; i++)
+                    entries.add(delegateSerializer.readDataEntry(in));
+
+                return new DataRecord(entries, timeStamp);
+
+            default:
+                return delegateSerializer.readRecord(type, in);
+        }
     }
 
     /** {@inheritDoc} */
@@ -59,6 +86,20 @@ public class RecordDataV2Serializer implements RecordDataSerializer {
         if (record instanceof HeaderRecord)
             throw new UnsupportedOperationException("Writing header records is forbidden since version 2 of serializer");
 
-        delegateSerializer.writeRecord(record, buf);
+        switch (record.type()) {
+            case DATA_RECORD:
+                DataRecord dataRec = (DataRecord)record;
+
+                buf.putInt(dataRec.writeEntries().size());
+                buf.putLong(dataRec.timestamp());
+
+                for (DataEntry dataEntry : dataRec.writeEntries())
+                    RecordDataV1Serializer.putDataEntry(buf, dataEntry);
+
+                break;
+
+            default:
+                delegateSerializer.writeRecord(record, buf);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/TxRecordSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/TxRecordSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/TxRecordSerializer.java
index 448bdbc..e8b324d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/TxRecordSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/TxRecordSerializer.java
@@ -75,9 +75,8 @@ public class TxRecordSerializer {
 
                 buf.putInt(backupNodes.size());
 
-                for (Object backupNode : backupNodes) {
+                for (Object backupNode : backupNodes)
                     writeConsistentId(backupNode, buf);
-                }
             }
         }
         else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index b5178b5..00c637e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -1116,8 +1116,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
                             nearXidVersion(),
                             writeVersion(),
                             participatingNodes,
-                            remote() ? nodeId() : null,
-                            U.currentTimeMillis()
+                            remote() ? nodeId() : null
                     );
 
                     try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e6e1ca9a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
index f31d0f9..ddf74c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalSerializerVersionTest.java
@@ -17,23 +17,38 @@
 
 package org.apache.ignite.internal.processors.cache.persistence.db.wal;
 
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.PersistentStoreConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.pagemem.wal.WALIterator;
+import org.apache.ignite.internal.pagemem.wal.WALPointer;
+import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
+import org.apache.ignite.internal.pagemem.wal.record.DataRecord;
+import org.apache.ignite.internal.pagemem.wal.record.TimeStampRecord;
+import org.apache.ignite.internal.pagemem.wal.record.TxRecord;
+import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.processors.cache.persistence.wal.RecordSerializer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV2Serializer;
 import org.apache.ignite.internal.util.typedef.internal.GPC;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.TransactionState;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_WAL_SERIALIZER_VERSION;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
  *
@@ -46,7 +61,7 @@ public class IgniteWalSerializerVersionTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(name);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
 
         cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration());
 
@@ -101,4 +116,192 @@ public class IgniteWalSerializerVersionTest extends GridCommonAbstractTest {
 
         stopGrid();
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCheckDifferentSerializerVersionsAndLogTimestamp() throws Exception {
+        IgniteCallable<List<WALRecord>> recordsFactory = new IgniteCallable<List<WALRecord>>() {
+            @Override public List<WALRecord> call() throws Exception {
+                WALRecord rec0 = new DataRecord(Collections.<DataEntry>emptyList());
+
+                WALRecord rec1 = new TxRecord(
+                    TransactionState.PREPARED,
+                    null,
+                    null,
+                    null,
+                    null
+                );
+
+                return Arrays.asList(rec0, rec1);
+            }
+        };
+
+        long time0 = U.currentTimeMillis();
+
+        check(new Checker(
+            1,
+            RecordV1Serializer.class,
+            recordsFactory,
+            Arrays.asList(0L, time0)
+        ));
+
+        long time1 = U.currentTimeMillis();
+
+        check(new Checker(
+            2,
+            RecordV2Serializer.class,
+            recordsFactory,
+            Arrays.asList(time1, time1)
+        ));
+    }
+
+    /**
+     *
+     */
+    public static class Checker {
+        /** */
+        private final int serializerVer;
+
+        /** */
+        private final Class serializer;
+
+        /** */
+        private final List<Long> timeStamps;
+
+        /** */
+        private final IgniteCallable<List<WALRecord>> recordsToWrite;
+
+        /**
+         *
+         */
+        public Checker(
+            int serializerVer,
+            Class serializer,
+            IgniteCallable<List<WALRecord>> recordsToWrite,
+            List<Long> timeStamps) {
+            this.serializerVer = serializerVer;
+            this.serializer = serializer;
+            this.timeStamps = timeStamps;
+            this.recordsToWrite = recordsToWrite;
+        }
+
+        /**
+         *
+         */
+        public int serializerVersion() {
+            return serializerVer;
+        }
+
+        /**
+         *
+         */
+        public Class serializer() {
+            return serializer;
+        }
+
+        /**
+         *
+         */
+        public List<Long> getTimeStamps() {
+            return timeStamps;
+        }
+
+        /**
+         *
+         */
+        public List<WALRecord> recordsToWrite() throws Exception {
+            return recordsToWrite.call();
+        }
+
+        /**
+         *
+         */
+        public void assertRecords(long exp, WALRecord act) {
+            if (act instanceof TimeStampRecord) {
+                TimeStampRecord act0 = (TimeStampRecord)act;
+
+                if (exp == 0L)
+                    assertTrue(act0.timestamp() == 0L);
+                else{
+                    long diff = Math.abs(exp - act0.timestamp());
+
+                    assertTrue(String.valueOf(diff), diff < 10_000);
+                }
+            }
+            else
+                fail();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void check(Checker checker) throws Exception {
+        System.setProperty(IGNITE_WAL_SERIALIZER_VERSION, Integer.toString(checker.serializerVersion()));
+
+        IgniteEx ig0 = (IgniteEx)startGrid();
+
+        ig0.active(true);
+
+        IgniteWriteAheadLogManager wal = ig0.context().cache().context().wal();
+
+        RecordSerializer ser0 = U.field(wal, "serializer");
+
+        assertTrue(ser0.getClass().getName().equals(checker.serializer().getName()));
+
+        List<WALRecord> recs = checker.recordsToWrite();
+
+        assertTrue(!recs.isEmpty());
+
+        WALPointer p = null;
+
+        for (WALRecord rec : recs) {
+            WALPointer p0 = wal.log(rec);
+
+            if (p == null)
+                p = p0;
+        }
+
+        wal.fsync(null);
+
+        Iterator<Long> itToCheck = checker.getTimeStamps().iterator();
+
+        try (WALIterator it = wal.replay(p)) {
+            while (it.hasNext()) {
+                IgniteBiTuple<WALPointer, WALRecord> tup0 = it.next();
+
+                checker.assertRecords(itToCheck.next(), tup0.get2());
+            }
+        }
+
+        stopGrid();
+
+        System.clearProperty(IGNITE_WAL_SERIALIZER_VERSION);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        deleteWorkFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        deleteWorkFiles();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void deleteWorkFiles() throws IgniteCheckedException {
+        deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false));
+    }
 }


[12/29] ignite git commit: IGNITE-5739 Fixed Ignite node crash on deactivation

Posted by sb...@apache.org.
IGNITE-5739 Fixed Ignite node crash on deactivation


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

Branch: refs/heads/ignite-5937
Commit: 3b1cad2156e6230760fface6c7cd00d3f3526255
Parents: 3c355e9
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Oct 5 17:37:04 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Oct 5 17:37:04 2017 +0300

----------------------------------------------------------------------
 .../GridCacheDatabaseSharedManager.java           |  2 ++
 .../cache/persistence/pagemem/PageMemoryImpl.java | 18 ++++++++++++++++++
 2 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3b1cad21/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 33f065e..9a2e028 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -540,6 +540,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
         try {
             cctx.kernalContext().config().getMBeanServer().unregisterMBean(persistenceMetricsMbeanName);
+
+            persistenceMetricsMbeanName = null;
         }
         catch (Throwable e) {
             U.error(log, "Failed to unregister " + MBEAN_NAME + " MBean.", e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b1cad21/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 1da17b5..95b81ad 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -237,6 +237,9 @@ public class PageMemoryImpl implements PageMemoryEx {
     /** */
     private MemoryMetricsImpl memMetrics;
 
+    /** */
+    private volatile boolean closed;
+
     /**
      * @param directMemoryProvider Memory allocator to use.
      * @param sharedCtx Cache shared context.
@@ -358,6 +361,15 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         U.shutdownNow(getClass(), asyncRunner, log);
 
+        closed = true;
+
+        for (Segment seg : segments) {
+            // Make sure all threads have left the lock.
+            seg.writeLock().lock();
+
+            seg.writeLock().unlock();
+        }
+
         directMemoryProvider.shutdown();
     }
 
@@ -1093,6 +1105,9 @@ public class PageMemoryImpl implements PageMemoryEx {
                 seg.readLock().lock();
 
                 try {
+                    if (closed)
+                        continue;
+
                     total += seg.loadedPages.size();
                 }
                 finally {
@@ -1114,6 +1129,9 @@ public class PageMemoryImpl implements PageMemoryEx {
             seg.readLock().lock();
 
             try {
+                if (closed)
+                    continue;
+
                 total += seg.acquiredPages();
             }
             finally {


[23/29] ignite git commit: Fixed JavaDoc again.

Posted by sb...@apache.org.
Fixed JavaDoc again.


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

Branch: refs/heads/ignite-5937
Commit: 87e7b32d506a0b3caa93c9da2be1ea51f20ae96d
Parents: e9764f1
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 9 11:34:51 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 9 11:34:51 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/wal/serializer/RecordV2Serializer.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/87e7b32d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
index a06b6d7..98804d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
@@ -39,7 +39,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.wal.serial
  * Record V2 serializer.
  * Stores records in following format:
  * <ul>
- * <li>Record type from {@link WALRecord.RecordType} incremented by 1</li>
+ * <li>Record type from {@code WALRecord.RecordType} incremented by 1</li>
  * <li>WAL pointer to double check consistency</li>
  * <li>Record length</li>
  * <li>Data</li>


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

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

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java


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

Branch: refs/heads/ignite-5937
Commit: c553638a48594d110472aad27ba8218f7e92f663
Parents: 0bbbbe2 fd53c1a
Author: sboikov <sb...@gridgain.com>
Authored: Mon Oct 9 16:27:14 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 9 16:27:14 2017 +0300

----------------------------------------------------------------------
 assembly/release-fabric-base.xml                |   1 -
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |  28 +-
 .../internal/jdbc2/JdbcStatementSelfTest.java   | 130 +++++++-
 .../JdbcThinAbstractDmlStatementSelfTest.java   |   2 +-
 .../thin/JdbcThinAutoCloseServerCursorTest.java |   8 +-
 .../jdbc/thin/JdbcThinComplexQuerySelfTest.java |   2 +-
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |   4 +
 .../JdbcThinDynamicIndexAbstractSelfTest.java   |   2 +-
 .../jdbc/thin/JdbcThinEmptyCacheSelfTest.java   |   2 +-
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  39 +--
 .../JdbcThinMissingLongArrayResultsTest.java    |   2 +-
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  |   5 +-
 .../thin/JdbcThinPreparedStatementSelfTest.java |   2 +-
 .../jdbc/thin/JdbcThinResultSetSelfTest.java    |   2 +-
 .../jdbc/thin/JdbcThinStatementSelfTest.java    |  46 +--
 .../org/apache/ignite/IgniteJdbcDriver.java     |   9 +-
 .../apache/ignite/configuration/WALMode.java    |   9 +-
 .../connection/GridClientNioTcpConnection.java  |   2 +
 .../internal/jdbc/thin/JdbcThinConnection.java  |   6 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     |   3 +-
 .../jdbc/thin/JdbcThinPreparedStatement.java    |   5 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   |   9 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |   4 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |  13 +
 .../internal/jdbc2/JdbcDatabaseMetadata.java    | 124 +++++---
 .../jdbc2/JdbcQueryMultipleStatementsTask.java  | 167 ++++++++++
 .../ignite/internal/jdbc2/JdbcQueryTask.java    | 154 +++-------
 .../internal/jdbc2/JdbcQueryTaskResult.java     | 120 ++++++++
 .../ignite/internal/jdbc2/JdbcQueryTaskV3.java  |  94 ++++++
 .../ignite/internal/jdbc2/JdbcResultSet.java    | 175 ++++++++---
 .../ignite/internal/jdbc2/JdbcStatement.java    | 270 +++++++++--------
 .../internal/jdbc2/JdbcStatementResultInfo.java |  73 +++++
 .../jdbc2/JdbcStreamedPreparedStatement.java    |  19 +-
 .../ignite/internal/pagemem/wal/WALPointer.java |   4 +-
 .../internal/pagemem/wal/record/DataEntry.java  |  13 +-
 .../internal/pagemem/wal/record/DataRecord.java |  20 +-
 .../pagemem/wal/record/SnapshotRecord.java      |  58 ++++
 .../pagemem/wal/record/TimeStampRecord.java     |  57 ++++
 .../internal/pagemem/wal/record/TxRecord.java   |  52 ++--
 .../pagemem/wal/record/UnwrapDataEntry.java     |  22 +-
 .../internal/pagemem/wal/record/WALRecord.java  |   6 +-
 .../processors/cache/GridCacheAdapter.java      |  16 +
 .../processors/cache/GridCacheMapEntry.java     |   2 +
 .../GridCacheDatabaseSharedManager.java         |   5 +
 .../persistence/pagemem/PageMemoryImpl.java     |  18 ++
 .../cache/persistence/wal/FileWALPointer.java   |   3 +
 .../wal/FileWriteAheadLogManager.java           |   7 +
 .../reader/StandaloneWalRecordsIterator.java    |  37 ++-
 .../wal/serializer/RecordDataV1Serializer.java  |   6 +-
 .../wal/serializer/RecordDataV2Serializer.java  |  49 ++-
 .../wal/serializer/RecordV2Serializer.java      |   2 +-
 .../wal/serializer/TxRecordSerializer.java      |   3 +-
 .../cache/query/GridCacheQueryManager.java      |  87 +++++-
 .../query/GridCacheQuerySqlMetadataJobV2.java   | 154 ++++++++++
 .../query/GridCacheQuerySqlMetadataV2.java      | 101 +++++++
 .../cache/query/GridCacheSqlMetadata.java       |   8 +
 .../cache/query/IgniteQueryErrorCode.java       |   2 +-
 .../cache/transactions/IgniteTxAdapter.java     |   3 +-
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |  10 +
 .../processors/odbc/jdbc/JdbcColumnMetaV2.java  |  74 +++++
 .../odbc/jdbc/JdbcConnectionContext.java        |   4 +-
 .../odbc/jdbc/JdbcMetaColumnsResult.java        |  28 +-
 .../odbc/jdbc/JdbcMetaColumnsResultV2.java      |  50 +++
 .../odbc/jdbc/JdbcRequestHandler.java           |  32 +-
 .../processors/odbc/jdbc/JdbcResult.java        |   8 +
 .../utils/PlatformConfigurationUtils.java       |   3 +
 .../processors/query/GridQueryIndexing.java     |  22 +-
 .../processors/query/GridQueryProcessor.java    |   8 +-
 .../query/QueryTypeDescriptorImpl.java          |  42 ++-
 .../query/property/QueryBinaryProperty.java     |   1 -
 .../handlers/cache/GridCacheCommandHandler.java |  12 +-
 .../internal/visor/query/VisorQueryTask.java    |  36 ++-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  45 ++-
 ...IgniteClientCacheInitializationFailTest.java |  18 +-
 .../db/wal/IgniteWalSerializerVersionTest.java  | 205 ++++++++++++-
 .../db/wal/reader/IgniteWalReaderTest.java      | 256 +++++++++++++++-
 .../query/h2/DmlStatementsProcessor.java        |  26 +-
 .../internal/processors/query/h2/H2Schema.java  |  17 +-
 .../internal/processors/query/h2/H2TypeKey.java |  64 ++++
 .../processors/query/h2/IgniteH2Indexing.java   |  33 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |  36 ++-
 .../query/h2/sql/GridSqlCreateTable.java        |  34 +++
 .../query/h2/sql/GridSqlQueryParser.java        |  58 +++-
 ...ynamicColumnsAbstractConcurrentSelfTest.java |  57 ++--
 ...umnsConcurrentAtomicPartitionedSelfTest.java |   2 +-
 ...lumnsConcurrentAtomicReplicatedSelfTest.java |   2 +-
 ...currentTransactionalPartitionedSelfTest.java |   2 +-
 ...ncurrentTransactionalReplicatedSelfTest.java |   5 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |  43 +++
 .../cache/index/H2DynamicTableSelfTest.java     | 301 +++++++++++++++++--
 .../query/IgniteSqlNotNullConstraintTest.java   |   2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  47 +--
 .../Cache/Query/CacheDmlQueriesTest.cs          |   4 +-
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs   |   5 +
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |   4 +
 modules/web-console/backend/app/agentSocket.js  |  21 +-
 .../web-console/backend/app/browsersHandler.js  |   9 +-
 modules/web-console/backend/package.json        |   4 +-
 .../app/modules/agent/AgentManager.service.js   |  25 +-
 .../app/modules/agent/decompress.worker.js      |  34 +++
 .../frontend/app/modules/sql/sql.controller.js  |  36 ++-
 .../frontend/app/utils/SimpleWorkerPool.js      | 119 ++++++++
 modules/web-console/frontend/package.json       |   8 +-
 .../web-console/frontend/views/sql/sql.tpl.pug  |  14 +-
 104 files changed, 3445 insertions(+), 687 deletions(-)
----------------------------------------------------------------------


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


[26/29] ignite git commit: ignite-5937 Added BPlusTree.iterate for more optimal mvcc search

Posted by sb...@apache.org.
ignite-5937 Added BPlusTree.iterate for more optimal mvcc search


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

Branch: refs/heads/ignite-5937
Commit: fd53c1a8f4e905a7aba469eb5decf38c50b7708e
Parents: 921404a
Author: sboikov <sb...@gridgain.com>
Authored: Mon Oct 9 14:42:43 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 9 14:42:43 2017 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheOffheapManager.java        |  11 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    | 244 +++++++----
 .../cache/mvcc/CacheCoordinatorsProcessor.java  |  12 +-
 .../cache/persistence/tree/BPlusTree.java       | 416 ++++++++++++++-----
 .../cache/tree/AbstractDataInnerIO.java         |   8 +-
 .../cache/tree/AbstractDataLeafIO.java          |   8 +-
 .../processors/cache/tree/CacheDataTree.java    |  10 +-
 .../cache/tree/CacheIdAwareDataInnerIO.java     |   4 +-
 .../cache/tree/CacheIdAwareDataLeafIO.java      |   4 +-
 .../processors/cache/tree/DataInnerIO.java      |   4 +-
 .../processors/cache/tree/DataLeafIO.java       |   4 +-
 .../processors/cache/tree/MvccDataInnerIO.java  |   4 +-
 .../processors/cache/tree/MvccDataLeafIO.java   |   4 +-
 .../cache/tree/MvccKeyMaxVersionBound.java      |  77 ++++
 .../cache/tree/MvccKeyMinVersionBound.java      |  49 +++
 .../processors/cache/tree/MvccUpdateRow.java    | 177 ++++++++
 .../cache/tree/MvccVersionBasedSearchRow.java   | 100 +++++
 .../processors/cache/tree/RowLinkIO.java        |  14 +-
 .../cache/mvcc/CacheMvccTransactionsTest.java   | 149 +++++++
 .../processors/database/BPlusTreeSelfTest.java  | 291 ++++++++++++-
 20 files changed, 1376 insertions(+), 214 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
index 9d03e4a..8967ce8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
@@ -492,7 +492,16 @@ public interface IgniteCacheOffheapManager {
             long expireTime,
             @Nullable CacheDataRow oldRow) throws IgniteCheckedException;
 
-        GridLongList mvccUpdate(
+        /**
+         * @param cctx Cache context.
+         * @param key Key.
+         * @param val Value.
+         * @param ver Version.
+         * @param mvccVer Mvcc version.
+         * @return List of transactions to wait for.
+         * @throws IgniteCheckedException If failed.
+         */
+        @Nullable GridLongList mvccUpdate(
             GridCacheContext cctx,
             KeyCacheObject key,
             CacheObject val,

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index d8c5eaa..25f36b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -55,7 +55,11 @@ import org.apache.ignite.internal.processors.cache.tree.CacheDataRowStore;
 import org.apache.ignite.internal.processors.cache.tree.CacheDataTree;
 import org.apache.ignite.internal.processors.cache.tree.DataRow;
 import org.apache.ignite.internal.processors.cache.tree.MvccDataRow;
+import org.apache.ignite.internal.processors.cache.tree.MvccKeyMaxVersionBound;
+import org.apache.ignite.internal.processors.cache.tree.MvccKeyMinVersionBound;
 import org.apache.ignite.internal.processors.cache.tree.MvccSearchRow;
+import org.apache.ignite.internal.processors.cache.tree.MvccUpdateRow;
+import org.apache.ignite.internal.processors.cache.tree.MvccVersionBasedSearchRow;
 import org.apache.ignite.internal.processors.cache.tree.PendingEntriesTree;
 import org.apache.ignite.internal.processors.cache.tree.PendingRow;
 import org.apache.ignite.internal.processors.cache.tree.SearchRow;
@@ -1360,83 +1364,141 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             try {
                 int cacheId = grp.storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID;
 
-                MvccDataRow dataRow = new MvccDataRow(
-                    key,
-                    val,
-                    ver,
-                    partId,
-                    cacheId,
-                    mvccVer.coordinatorVersion(),
-                    mvccVer.counter());
-
                 CacheObjectContext coCtx = cctx.cacheObjectContext();
 
                 // Make sure value bytes initialized.
                 key.valueBytes(coCtx);
                 val.valueBytes(coCtx);
 
-                rowStore.addRow(dataRow);
+                if (true) {
+                    MvccUpdateRow updateRow = new MvccUpdateRow(
+                        key,
+                        val,
+                        ver,
+                        mvccVer,
+                        partId,
+                        cacheId);
 
-                assert dataRow.link() != 0 : dataRow;
+                    rowStore.addRow(updateRow);
 
-                if (grp.sharedGroup() && dataRow.cacheId() == CU.UNDEFINED_CACHE_ID)
-                    dataRow.cacheId(cctx.cacheId());
+                    assert updateRow.link() != 0 : updateRow;
 
-                boolean old = dataTree.putx(dataRow);
+                    if (grp.sharedGroup() && updateRow.cacheId() == CU.UNDEFINED_CACHE_ID)
+                        updateRow.cacheId(cctx.cacheId());
 
-                assert !old;
+                    GridLongList waitTxs = null;
 
-                GridLongList waitTxs = null;
+                    if (mvccVer.initialLoad()) {
+                        boolean old = dataTree.putx(updateRow);
 
-                if (!mvccVer.initialLoad()) {
-                    MvccLongList activeTxs = mvccVer.activeTransactions();
+                        assert !old;
 
-                    // TODO IGNITE-3484: need special method.
-                    GridCursor<CacheDataRow> cur = dataTree.find(
-                        new MvccSearchRow(cacheId, key, mvccVer.coordinatorVersion(), mvccVer.counter() - 1),
-                        new MvccSearchRow(cacheId, key, 1, 1));
+                        incrementSize(cctx.cacheId());
+                    }
+                    else {
+                        dataTree.iterate(updateRow, new MvccKeyMinVersionBound(cacheId, key), updateRow);
 
-                    boolean first = true;
+                        boolean old = dataTree.putx(updateRow);
 
-                    boolean activeTx = false;
+                        assert !old;
 
-                    while (cur.next()) {
-                        CacheDataRow oldVal = cur.get();
+                        if (!updateRow.previousNotNull())
+                            incrementSize(cctx.cacheId());
 
-                        assert oldVal.link() != 0 : oldVal;
+                        waitTxs = updateRow.activeTransactions();
 
-                        if (activeTxs != null && oldVal.mvccCoordinatorVersion() == mvccVer.coordinatorVersion() &&
-                            activeTxs.contains(oldVal.mvccCounter())) {
-                            if (waitTxs == null)
-                                waitTxs = new GridLongList();
+                        List<CacheSearchRow> cleanupRows = updateRow.cleanupRows();
 
-                            assert oldVal.mvccCounter() != mvccVer.counter();
+                        if (cleanupRows != null) {
+                            for (int i = 0; i < cleanupRows.size(); i++) {
+                                CacheSearchRow oldRow = cleanupRows.get(i);
 
-                            waitTxs.add(oldVal.mvccCounter());
+                                assert oldRow.link() != 0L : oldRow;
 
-                            activeTx = true;
+                                boolean rmvd = dataTree.removex(oldRow);
+
+                                assert rmvd;
+
+                                rowStore.removeRow(oldRow.link());
+                            }
                         }
+                    }
 
-                        if (!activeTx) {
-                            // Should not delete oldest version which is less than cleanup version.
-                            int cmp = compare(oldVal, mvccVer.coordinatorVersion(), mvccVer.cleanupVersion());
+                    return waitTxs;
+                }
+                else {
+                    MvccDataRow dataRow = new MvccDataRow(
+                        key,
+                        val,
+                        ver,
+                        partId,
+                        cacheId,
+                        mvccVer.coordinatorVersion(),
+                        mvccVer.counter());
 
-                            if (cmp <= 0) {
-                                if (first)
-                                    first = false;
-                                else {
-                                    boolean rmvd = dataTree.removex(oldVal);
+                    rowStore.addRow(dataRow);
 
-                                    assert rmvd;
+                    assert dataRow.link() != 0 : dataRow;
+
+                    if (grp.sharedGroup() && dataRow.cacheId() == CU.UNDEFINED_CACHE_ID)
+                        dataRow.cacheId(cctx.cacheId());
+
+                    boolean old = dataTree.putx(dataRow);
+
+                    assert !old;
+
+                    GridLongList waitTxs = null;
+
+                    if (!mvccVer.initialLoad()) {
+                        MvccLongList activeTxs = mvccVer.activeTransactions();
+
+                        // TODO IGNITE-3484: need special method.
+                        GridCursor<CacheDataRow> cur = dataTree.find(
+                            new MvccSearchRow(cacheId, key, mvccVer.coordinatorVersion(), mvccVer.counter() - 1),
+                            new MvccSearchRow(cacheId, key, 1, 1));
+
+                        boolean first = true;
+
+                        boolean activeTx = false;
+
+                        while (cur.next()) {
+                            CacheDataRow oldVal = cur.get();
+
+                            assert oldVal.link() != 0 : oldVal;
+
+                            if (activeTxs != null && oldVal.mvccCoordinatorVersion() == mvccVer.coordinatorVersion() &&
+                                activeTxs.contains(oldVal.mvccCounter())) {
+                                if (waitTxs == null)
+                                    waitTxs = new GridLongList();
+
+                                assert oldVal.mvccCounter() != mvccVer.counter();
+
+                                waitTxs.add(oldVal.mvccCounter());
+
+                                activeTx = true;
+                            }
 
-                                    rowStore.removeRow(oldVal.link());
+                            if (!activeTx) {
+                                // Should not delete oldest version which is less than cleanup version.
+                                int cmp = compare(oldVal, mvccVer.coordinatorVersion(), mvccVer.cleanupVersion());
+
+                                if (cmp <= 0) {
+                                    if (first)
+                                        first = false;
+                                    else {
+                                        boolean rmvd = dataTree.removex(oldVal);
+
+                                        assert rmvd;
+
+                                        rowStore.removeRow(oldVal.link());
+                                    }
                                 }
                             }
                         }
                     }
-                }
 
-                return waitTxs;
+                    return waitTxs;
+                }
             }
             finally {
                 busyLock.leaveBusy();
@@ -1647,14 +1709,26 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             CacheDataRow row;
 
             if (grp.mvccEnabled()) {
-                // TODO IGNITE-3484: need special method.
-                GridCursor<CacheDataRow> cur = dataTree.find(new MvccSearchRow(cacheId, key, Long.MAX_VALUE, Long.MAX_VALUE),
-                    new MvccSearchRow(cacheId, key, 1, 1));
+                if (true) {
+                    MvccKeyMaxVersionBound searchRow = new MvccKeyMaxVersionBound(cacheId, key);
+
+                    dataTree.iterate(
+                        searchRow,
+                        new MvccKeyMinVersionBound(cacheId, key),
+                        searchRow // Use the same instance as closure to do not create extra object.
+                    );
 
-                if (cur.next())
-                    row = cur.get();
-                else
-                    row = null;
+                    row = searchRow.row();
+                }
+                else {
+                    GridCursor<CacheDataRow> cur = dataTree.find(new MvccSearchRow(cacheId, key, Long.MAX_VALUE, Long.MAX_VALUE),
+                        new MvccSearchRow(cacheId, key, 1, 1));
+
+                    if (cur.next())
+                        row = cur.get();
+                    else
+                        row = null;
+                }
             }
             else
                 row = dataTree.findOne(new SearchRow(cacheId, key), CacheDataRowAdapter.RowData.NO_KEY);
@@ -1672,6 +1746,8 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         {
             assert grp.mvccEnabled();
 
+            // Note: this method is intended for testing only.
+
             key.valueBytes(cctx.cacheObjectContext());
 
             int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID;
@@ -1705,41 +1781,55 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
 
             int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID;
 
-            // TODO IGNITE-3484: need special method.
-            GridCursor<CacheDataRow> cur = dataTree.find(
-                new MvccSearchRow(cacheId, key, ver.coordinatorVersion(), ver.counter()),
-                new MvccSearchRow(cacheId, key, 1, 1));
+            if (true) {
+                MvccVersionBasedSearchRow lower = new MvccVersionBasedSearchRow(cacheId, key, ver);
 
-            CacheDataRow row = null;
+                dataTree.iterate(
+                    lower,
+                    new MvccKeyMinVersionBound(cacheId, key),
+                    lower // Use the same instance as closure to do not create extra object.
+                );
 
-            MvccLongList txs = ver.activeTransactions();
+                CacheDataRow row = lower.row();
 
-            while (cur.next()) {
-                CacheDataRow row0 = cur.get();
+                afterRowFound(row, key);
 
-                assert row0.mvccCoordinatorVersion() > 0 : row0;
+                return row;
+            }
+            else {
+                GridCursor<CacheDataRow> cur = dataTree.find(
+                    new MvccSearchRow(cacheId, key, ver.coordinatorVersion(), ver.counter()),
+                    new MvccSearchRow(cacheId, key, 1, 1));
 
-                boolean visible;
+                CacheDataRow row = null;
 
-                if (txs != null) {
-                    visible = row0.mvccCoordinatorVersion() != ver.coordinatorVersion()
-                        || !txs.contains(row0.mvccCounter());
-                }
-                else
-                    visible = true;
+                MvccLongList txs = ver.activeTransactions();
 
-                if (visible) {
-                    row = row0;
+                while (cur.next()) {
+                    CacheDataRow row0 = cur.get();
 
-                    break;
-                }
-            }
+                    assert row0.mvccCoordinatorVersion() > 0 : row0;
 
-            assert row == null || key.equals(row.key());
+                    boolean visible;
 
-            //afterRowFound(row, key);
+                    if (txs != null) {
+                        visible = row0.mvccCoordinatorVersion() != ver.coordinatorVersion()
+                            || !txs.contains(row0.mvccCounter());
+                    }
+                    else
+                        visible = true;
 
-            return row;
+                    if (visible) {
+                        row = row0;
+
+                        break;
+                    }
+                }
+
+                assert row == null || key.equals(row.key());
+
+                return row;
+            }
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java
index 5080c83..b9b8ea1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java
@@ -614,8 +614,14 @@ public class CacheCoordinatorsProcessor extends GridProcessorAdapter {
         // TODO IGNITE-3478 sorted? + change GridLongList.writeTo?
         MvccCoordinatorVersionResponse res = new MvccCoordinatorVersionResponse();
 
-        for (Long txVer : activeTxs.keySet())
+        long minActive = Long.MAX_VALUE;
+
+        for (Long txVer : activeTxs.keySet()) {
+            if (txVer < minActive)
+                minActive = txVer;
+
             res.addTx(txVer);
+        }
 
         Object old = activeTxs.put(nextCtr, txId);
 
@@ -624,7 +630,9 @@ public class CacheCoordinatorsProcessor extends GridProcessorAdapter {
         long cleanupVer;
 
         if (prevCrdQueries.previousQueriesDone()) {
-            cleanupVer = committedCntr.get() - 1;
+            cleanupVer = Math.min(minActive, committedCntr.get());
+
+            cleanupVer--;
 
             Long qryVer = activeQueries.minimalQueryCounter();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index c73b4c7..b31a61f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -907,7 +907,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             long pageAddr = readLock(firstPageId, firstPage); // We always merge pages backwards, the first page is never removed.
 
             try {
-                cursor.init(pageAddr, io(pageAddr), 0);
+                cursor.init(pageAddr, io(pageAddr), -1);
             }
             finally {
                 readUnlock(firstPageId, firstPage, pageAddr);
@@ -972,6 +972,34 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
     }
 
+    /**
+     * @param lower Lower bound inclusive.
+     * @param upper Upper bound inclusive.
+     * @param c Closure applied for all found items, iteration is stopped if closure returns {@code false}.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void iterate(L lower, L upper, TreeRowClosure<L, T> c) throws IgniteCheckedException {
+        checkDestroyed();
+
+        try {
+            ClosureCursor cursor = new ClosureCursor(lower, upper, c);
+
+            cursor.iterate();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteCheckedException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
+        }
+        catch (RuntimeException e) {
+            throw new IgniteException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
+        }
+        catch (AssertionError e) {
+            throw new AssertionError("Assertion error on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
+        }
+        finally {
+            checkDestroyed();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public T findFirst() throws IgniteCheckedException {
         checkDestroyed();
@@ -2509,14 +2537,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private final class GetCursor extends Get {
         /** */
-        ForwardCursor cursor;
+        AbstractForwardCursor cursor;
 
         /**
          * @param lower Lower bound.
          * @param shift Shift.
          * @param cursor Cursor.
          */
-        GetCursor(L lower, int shift, ForwardCursor cursor) {
+        GetCursor(L lower, int shift, AbstractForwardCursor cursor) {
             super(lower, false);
 
             assert shift != 0; // Either handle range of equal rows or find a greater row after concurrent merge.
@@ -4385,51 +4413,57 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     protected abstract T getRow(BPlusIO<L> io, long pageAddr, int idx, Object x) throws IgniteCheckedException;
 
     /**
-     * Forward cursor.
+     *
      */
     @SuppressWarnings("unchecked")
-    private final class ForwardCursor implements GridCursor<T> {
-        /** */
-        private T[] rows = (T[])EMPTY;
-
-        /** */
-        private int row = -1;
-
+    private abstract class AbstractForwardCursor {
         /** */
-        private long nextPageId;
+        long nextPageId;
 
         /** */
-        private L lowerBound;
+        L lowerBound;
 
         /** */
         private int lowerShift = -1; // Initially it is -1 to handle multiple equal rows.
 
         /** */
-        private final L upperBound;
-
-        /** */
-        private final Object x;
+        final L upperBound;
 
         /**
          * @param lowerBound Lower bound.
          * @param upperBound Upper bound.
          */
-        ForwardCursor(L lowerBound, L upperBound) {
+        AbstractForwardCursor(L lowerBound, L upperBound) {
             this.lowerBound = lowerBound;
             this.upperBound = upperBound;
-            this.x = null;
         }
 
         /**
-         * @param lowerBound Lower bound.
-         * @param upperBound Upper bound.
-         * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row.
+         *
          */
-        ForwardCursor(L lowerBound, L upperBound, Object x) {
-            this.lowerBound = lowerBound;
-            this.upperBound = upperBound;
-            this.x = x;
-        }
+        abstract void init0();
+
+        /**
+         * @param pageAddr Page address.
+         * @param io IO.
+         * @param startIdx Start index.
+         * @param cnt Number of rows in the buffer.
+         * @return {@code true} If we were able to fetch rows from this page.
+         * @throws IgniteCheckedException If failed.
+         */
+        abstract boolean fillFromBuffer0(long pageAddr, BPlusIO<L> io, int startIdx, int cnt)
+            throws IgniteCheckedException;
+
+        /**
+         * @return {@code True} If we have rows to return after reading the next page.
+         * @throws IgniteCheckedException If failed.
+         */
+        abstract boolean reinitialize0() throws IgniteCheckedException;
+
+        /**
+         * @param readDone {@code True} if traversed all rows.
+         */
+        abstract void onNotFound(boolean readDone);
 
         /**
          * @param pageAddr Page address.
@@ -4437,9 +4471,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @param startIdx Start index.
          * @throws IgniteCheckedException If failed.
          */
-        private void init(long pageAddr, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
+        final void init(long pageAddr, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
             nextPageId = 0;
-            row = -1;
+
+            init0();
 
             int cnt = io.getCount(pageAddr);
 
@@ -4447,16 +4482,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (cnt == 0) {
                 assert io.getForward(pageAddr) == 0L;
 
-                rows = null;
-            }
-            else if (!fillFromBuffer(pageAddr, io, startIdx, cnt)) {
-                if (rows != EMPTY) {
-                    assert rows.length > 0; // Otherwise it makes no sense to create an array.
-
-                    // Fake clear.
-                    rows[0] = null;
-                }
+                onNotFound(true);
             }
+            else if (!fillFromBuffer(pageAddr, io, startIdx, cnt))
+                onNotFound(false);
         }
 
         /**
@@ -4466,7 +4495,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @return Adjusted to lower bound start index.
          * @throws IgniteCheckedException If failed.
          */
-        private int findLowerBound(long pageAddr, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
+        final int findLowerBound(long pageAddr, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
             assert io.isLeaf();
 
             // Compare with the first row on the page.
@@ -4491,7 +4520,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @return Corrected number of rows with respect to upper bound.
          * @throws IgniteCheckedException If failed.
          */
-        private int findUpperBound(long pageAddr, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
+        final int findUpperBound(long pageAddr, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
             assert io.isLeaf();
 
             // Compare with the last row on the page.
@@ -4523,75 +4552,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             throws IgniteCheckedException {
             assert io.isLeaf() : io;
             assert cnt != 0 : cnt; // We can not see empty pages (empty tree handled in init).
-            assert startIdx >= 0 : startIdx;
+            assert startIdx >= 0 || startIdx == -1: startIdx;
             assert cnt >= startIdx;
 
             checkDestroyed();
 
             nextPageId = io.getForward(pageAddr);
 
-            if (lowerBound != null && startIdx == 0)
-                startIdx = findLowerBound(pageAddr, io, cnt);
-
-            if (upperBound != null && cnt != startIdx)
-                cnt = findUpperBound(pageAddr, io, startIdx, cnt);
-
-            cnt -= startIdx;
-
-            if (cnt == 0)
-                return false;
-
-            if (rows == EMPTY)
-                rows = (T[])new Object[cnt];
-
-            for (int i = 0; i < cnt; i++) {
-                T r = getRow(io, pageAddr, startIdx + i, x);
-
-                rows = GridArrays.set(rows, i, r);
-            }
-
-            GridArrays.clearTail(rows, cnt);
-
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("SimplifiableIfStatement")
-        @Override public boolean next() throws IgniteCheckedException {
-            if (rows == null)
-                return false;
-
-            if (++row < rows.length && rows[row] != null) {
-                clearLastRow(); // Allow to GC the last returned row.
-
-                return true;
-            }
-
-            return nextPage();
-        }
-
-        /**
-         * @return Cleared last row.
-         */
-        private T clearLastRow() {
-            if (row == 0)
-                return null;
-
-            int last = row - 1;
-
-            T r = rows[last];
-
-            assert r != null;
-
-            rows[last] = null;
-
-            return r;
+            return fillFromBuffer0(pageAddr, io, startIdx, cnt);
         }
 
         /**
          * @throws IgniteCheckedException If failed.
          */
-        private void find() throws IgniteCheckedException {
+        final void find() throws IgniteCheckedException {
             assert lowerBound != null;
 
             doFind(new GetCursor(lowerBound, lowerShift, this));
@@ -4607,21 +4581,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             // to the previous lower bound.
             find();
 
-            return next();
+            return reinitialize0();
         }
 
         /**
+         * @param lastRow Last read row (to be used as new lower bound).
          * @return {@code true} If we have rows to return after reading the next page.
          * @throws IgniteCheckedException If failed.
          */
-        private boolean nextPage() throws IgniteCheckedException {
-            updateLowerBound(clearLastRow());
-
-            row = 0;
+        final boolean nextPage(L lastRow) throws IgniteCheckedException {
+            updateLowerBound(lastRow);
 
             for (;;) {
                 if (nextPageId == 0) {
-                    rows = null;
+                    onNotFound(true);
 
                     return false; // Done.
                 }
@@ -4638,7 +4611,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     try {
                         BPlusIO<L> io = io(pageAddr);
 
-                        if (fillFromBuffer(pageAddr, io, 0, io.getCount(pageAddr)))
+                        if (fillFromBuffer(pageAddr, io, -1, io.getCount(pageAddr)))
                             return true;
 
                         // Continue fetching forward.
@@ -4659,12 +4632,227 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param lower New exact lower bound.
          */
-        private void updateLowerBound(T lower) {
+        private void updateLowerBound(L lower) {
             if (lower != null) {
                 lowerShift = 1; // Now we have the full row an need to avoid duplicates.
                 lowerBound = lower; // Move the lower bound forward for further concurrent merge retries.
             }
         }
+    }
+
+    /**
+     * Closure cursor.
+     */
+    @SuppressWarnings("unchecked")
+    private final class ClosureCursor extends AbstractForwardCursor {
+        /** */
+        private final TreeRowClosure<L, T> p;
+
+        /** */
+        private L lastRow;
+
+        /**
+         * @param lowerBound Lower bound.
+         * @param upperBound Upper bound.
+         * @param p Row predicate.
+         */
+        ClosureCursor(L lowerBound, L upperBound, TreeRowClosure<L, T> p) {
+            super(lowerBound, upperBound);
+
+            assert lowerBound != null;
+            assert upperBound != null;
+            assert p != null;
+
+            this.p = p;
+        }
+
+        /** {@inheritDoc} */
+        @Override void init0() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override boolean fillFromBuffer0(long pageAddr, BPlusIO<L> io, int startIdx, int cnt)
+            throws IgniteCheckedException {
+            if (startIdx == -1) // TODO IGNITE-3478: startIdx == 0? can search twice for first item?
+                startIdx = findLowerBound(pageAddr, io, cnt);
+
+            if (cnt == startIdx)
+                return false;
+
+            for (int i = startIdx; i < cnt; i++) {
+                int cmp = compare(0, io, pageAddr, i, upperBound);
+
+                if (cmp > 0) {
+                    nextPageId = 0; // The End.
+
+                    return false;
+                }
+
+                boolean stop = !p.apply(BPlusTree.this, io, pageAddr, i);
+
+                if (stop) {
+                    nextPageId = 0; // The End.
+
+                    return true;
+                }
+            }
+
+            if (nextPageId != 0)
+                lastRow = io.getLookupRow(BPlusTree.this, pageAddr, cnt - 1); // Need save last row.
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override boolean reinitialize0() throws IgniteCheckedException {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override void onNotFound(boolean readDone) {
+            nextPageId = 0;
+        }
+
+        /**
+         * @throws IgniteCheckedException If failed.
+         */
+        private void iterate() throws IgniteCheckedException {
+            find();
+
+            if (nextPageId == 0) {
+                return;
+            }
+
+            for (;;) {
+                L lastRow0 = lastRow;
+
+                lastRow = null;
+
+                nextPage(lastRow0);
+
+                if (nextPageId == 0)
+                    return;
+            }
+        }
+    }
+
+    /**
+     * Forward cursor.
+     */
+    @SuppressWarnings("unchecked")
+    private final class ForwardCursor extends AbstractForwardCursor implements GridCursor<T> {
+        /** */
+        final Object x;
+
+        /** */
+        private T[] rows = (T[])EMPTY;
+
+        /** */
+        private int row = -1;
+
+        /**
+         * @param lowerBound Lower bound.
+         * @param upperBound Upper bound.
+         * @param x Implementation specific argument, {@code null} always means that we need to return full detached data row.
+         */
+        ForwardCursor(L lowerBound, L upperBound, Object x) {
+            super(lowerBound, upperBound);
+
+            this.x = x;
+        }
+
+        /** {@inheritDoc} */
+        @Override boolean fillFromBuffer0(long pageAddr, BPlusIO<L> io, int startIdx, int cnt) throws IgniteCheckedException {
+            if (startIdx == -1) {
+                if (lowerBound != null)
+                    startIdx = findLowerBound(pageAddr, io, cnt);
+                else
+                    startIdx = 0;
+            }
+
+            if (upperBound != null && cnt != startIdx)
+                cnt = findUpperBound(pageAddr, io, startIdx, cnt);
+
+            cnt -= startIdx;
+
+            if (cnt == 0)
+                return false;
+
+            if (rows == EMPTY)
+                rows = (T[])new Object[cnt];
+
+            for (int i = 0; i < cnt; i++) {
+                T r = getRow(io, pageAddr, startIdx + i, x);
+
+                rows = GridArrays.set(rows, i, r);
+            }
+
+            GridArrays.clearTail(rows, cnt);
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override boolean reinitialize0() throws IgniteCheckedException {
+            return next();
+        }
+
+        /** {@inheritDoc} */
+        @Override void onNotFound(boolean readDone) {
+            if (readDone)
+                rows = null;
+            else {
+                if (rows != EMPTY) {
+                    assert rows.length > 0; // Otherwise it makes no sense to create an array.
+
+                    // Fake clear.
+                    rows[0] = null;
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override void init0() {
+            row = -1;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("SimplifiableIfStatement")
+        @Override public boolean next() throws IgniteCheckedException {
+            if (rows == null)
+                return false;
+
+            if (++row < rows.length && rows[row] != null) {
+                clearLastRow(); // Allow to GC the last returned row.
+
+                return true;
+            }
+
+            T lastRow = clearLastRow();
+
+            row = 0;
+
+            return nextPage(lastRow);
+        }
+
+        /**
+         * @return Cleared last row.
+         */
+        private T clearLastRow() {
+            if (row == 0)
+                return null;
+
+            int last = row - 1;
+
+            T r = rows[last];
+
+            assert r != null;
+
+            rows[last] = null;
+
+            return r;
+        }
 
         /** {@inheritDoc} */
         @Override public T get() {
@@ -4805,4 +4993,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         DONE
     }
+
+    /**
+     *
+     */
+    public interface TreeRowClosure<L, T extends L> {
+        /**
+         * @param tree Tree.
+         * @param io Tree IO.
+         * @param pageAddr Page address.
+         * @param idx Item index.
+         * @return {@code True} if item pass predicate. TODO IGNITE-3478
+         * @throws IgniteCheckedException If failed.
+         */
+        public boolean apply(BPlusTree<L, T> tree, BPlusIO<L> io, long pageAddr, int idx)
+            throws IgniteCheckedException;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java
index 3fc0962..a07d012 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataInnerIO.java
@@ -76,8 +76,8 @@ public abstract class AbstractDataInnerIO extends BPlusInnerIO<CacheSearchRow> i
         long link = getLink(pageAddr, idx);
 
         if (storeMvccVersion()) {
-            long mvccTopVer = getMvccUpdateTopologyVersion(pageAddr, idx);
-            long mvccCntr = getMvccUpdateCounter(pageAddr, idx);
+            long mvccTopVer = getMvccCoordinatorVersion(pageAddr, idx);
+            long mvccCntr = getMvccCounter(pageAddr, idx);
 
             return ((CacheDataTree)tree).rowStore().mvccRow(cacheId,
                 hash,
@@ -119,8 +119,8 @@ public abstract class AbstractDataInnerIO extends BPlusInnerIO<CacheSearchRow> i
         }
 
         if (storeMvccVersion()) {
-            long mvccTopVer = rowIo.getMvccUpdateTopologyVersion(srcPageAddr, srcIdx);
-            long mvcCntr = rowIo.getMvccUpdateCounter(srcPageAddr, srcIdx);
+            long mvccTopVer = rowIo.getMvccCoordinatorVersion(srcPageAddr, srcIdx);
+            long mvcCntr = rowIo.getMvccCounter(srcPageAddr, srcIdx);
 
             assert mvccTopVer > 0 : mvccTopVer;
             assert mvcCntr != CacheCoordinatorsProcessor.COUNTER_NA;

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java
index a4eac3e..ef08bec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/AbstractDataLeafIO.java
@@ -94,8 +94,8 @@ public abstract class AbstractDataLeafIO extends BPlusLeafIO<CacheSearchRow> imp
         }
 
         if (storeMvccVersion()) {
-            long mvccUpdateTopVer = ((RowLinkIO)srcIo).getMvccUpdateTopologyVersion(srcPageAddr, srcIdx);
-            long mvccUpdateCntr = ((RowLinkIO)srcIo).getMvccUpdateCounter(srcPageAddr, srcIdx);
+            long mvccUpdateTopVer = ((RowLinkIO)srcIo).getMvccCoordinatorVersion(srcPageAddr, srcIdx);
+            long mvccUpdateCntr = ((RowLinkIO)srcIo).getMvccCounter(srcPageAddr, srcIdx);
 
             assert mvccUpdateTopVer >=0 : mvccUpdateCntr;
             assert mvccUpdateCntr != CacheCoordinatorsProcessor.COUNTER_NA;
@@ -114,8 +114,8 @@ public abstract class AbstractDataLeafIO extends BPlusLeafIO<CacheSearchRow> imp
         long link = getLink(pageAddr, idx);
 
         if (storeMvccVersion()) {
-            long mvccTopVer = getMvccUpdateTopologyVersion(pageAddr, idx);
-            long mvccCntr = getMvccUpdateCounter(pageAddr, idx);
+            long mvccTopVer = getMvccCoordinatorVersion(pageAddr, idx);
+            long mvccCntr = getMvccCounter(pageAddr, idx);
 
             return ((CacheDataTree)tree).rowStore().mvccRow(cacheId,
                 hash,

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
index 767c996..eaeefee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
@@ -114,7 +114,7 @@ public class CacheDataTree extends BPlusTree<CacheSearchRow, CacheDataRow> {
     /** {@inheritDoc} */
     @Override protected int compare(BPlusIO<CacheSearchRow> iox, long pageAddr, int idx, CacheSearchRow row)
         throws IgniteCheckedException {
-        assert !grp.mvccEnabled() || row.mvccCoordinatorVersion() != 0;// || row.getClass() == SearchRow.class;
+        assert !grp.mvccEnabled() || row.mvccCoordinatorVersion() != 0 : row;
 
         RowLinkIO io = (RowLinkIO)iox;
 
@@ -158,14 +158,14 @@ public class CacheDataTree extends BPlusTree<CacheSearchRow, CacheDataRow> {
         if (cmp != 0 || !grp.mvccEnabled())
             return 0;
 
-        long mvccCrdVer = io.getMvccUpdateTopologyVersion(pageAddr, idx);
+        long mvccCrdVer = io.getMvccCoordinatorVersion(pageAddr, idx);
 
         cmp = Long.compare(row.mvccCoordinatorVersion(), mvccCrdVer);
 
         if (cmp != 0)
             return cmp;
 
-        long mvccCntr = io.getMvccUpdateCounter(pageAddr, idx);
+        long mvccCntr = io.getMvccCounter(pageAddr, idx);
 
         assert row.mvccCounter() != CacheCoordinatorsProcessor.COUNTER_NA;
 
@@ -188,8 +188,8 @@ public class CacheDataTree extends BPlusTree<CacheSearchRow, CacheDataRow> {
             CacheDataRowAdapter.RowData.FULL;
 
         if (grp.mvccEnabled()) {
-            long mvccTopVer = rowIo.getMvccUpdateTopologyVersion(pageAddr, idx);
-            long mvccCntr = rowIo.getMvccUpdateCounter(pageAddr, idx);
+            long mvccTopVer = rowIo.getMvccCoordinatorVersion(pageAddr, idx);
+            long mvccCntr = rowIo.getMvccCounter(pageAddr, idx);
 
             return rowStore.mvccRow(cacheId, hash, link, x, mvccTopVer, mvccCntr);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java
index fc9d15d..3d02b27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataInnerIO.java
@@ -53,12 +53,12 @@ public final class CacheIdAwareDataInnerIO extends AbstractDataInnerIO {
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateTopologyVersion(long pageAddr, int idx) {
+    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
         return 0;
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateCounter(long pageAddr, int idx) {
+    @Override public long getMvccCounter(long pageAddr, int idx) {
         return CacheCoordinatorsProcessor.COUNTER_NA;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java
index b328924..58ae9ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheIdAwareDataLeafIO.java
@@ -53,12 +53,12 @@ public final class CacheIdAwareDataLeafIO extends AbstractDataLeafIO {
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateTopologyVersion(long pageAddr, int idx) {
+    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
         return 0;
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateCounter(long pageAddr, int idx) {
+    @Override public long getMvccCounter(long pageAddr, int idx) {
         return CacheCoordinatorsProcessor.COUNTER_NA;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java
index 0d424b7..19a5c47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataInnerIO.java
@@ -53,12 +53,12 @@ public final class DataInnerIO extends AbstractDataInnerIO {
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateTopologyVersion(long pageAddr, int idx) {
+    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
         return 0;
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateCounter(long pageAddr, int idx) {
+    @Override public long getMvccCounter(long pageAddr, int idx) {
         return CacheCoordinatorsProcessor.COUNTER_NA;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java
index ff51bc2..ab10b96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataLeafIO.java
@@ -53,12 +53,12 @@ public final class DataLeafIO extends AbstractDataLeafIO {
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateTopologyVersion(long pageAddr, int idx) {
+    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
         return 0;
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateCounter(long pageAddr, int idx) {
+    @Override public long getMvccCounter(long pageAddr, int idx) {
         return CacheCoordinatorsProcessor.COUNTER_NA;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataInnerIO.java
index 5f4f44c..51a911d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataInnerIO.java
@@ -53,12 +53,12 @@ public final class MvccDataInnerIO extends AbstractDataInnerIO {
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateTopologyVersion(long pageAddr, int idx) {
+    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx) + 12);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateCounter(long pageAddr, int idx) {
+    @Override public long getMvccCounter(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx) + 20);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataLeafIO.java
index e7cfca7..84c33a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccDataLeafIO.java
@@ -53,12 +53,12 @@ public final class MvccDataLeafIO extends AbstractDataLeafIO {
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateTopologyVersion(long pageAddr, int idx) {
+    @Override public long getMvccCoordinatorVersion(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx) + 12);
     }
 
     /** {@inheritDoc} */
-    @Override public long getMvccUpdateCounter(long pageAddr, int idx) {
+    @Override public long getMvccCounter(long pageAddr, int idx) {
         return PageUtils.getLong(pageAddr, offset(idx) + 20);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccKeyMaxVersionBound.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccKeyMaxVersionBound.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccKeyMaxVersionBound.java
new file mode 100644
index 0000000..aa9422d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccKeyMaxVersionBound.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.tree;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class MvccKeyMaxVersionBound extends SearchRow implements BPlusTree.TreeRowClosure<CacheSearchRow, CacheDataRow> {
+    /** */
+    private CacheDataRow resRow;
+
+    /**
+     * @param cacheId Cache ID.
+     * @param key Key.
+     */
+    public MvccKeyMaxVersionBound(int cacheId, KeyCacheObject key) {
+        super(cacheId, key);
+    }
+
+    /**
+     * @return Found row.
+     */
+    @Nullable public CacheDataRow row() {
+        return resRow;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(BPlusTree<CacheSearchRow, CacheDataRow> tree, BPlusIO<CacheSearchRow> io,
+        long pageAddr,
+        int idx)
+        throws IgniteCheckedException
+    {
+        resRow = ((CacheDataTree)tree).getRow(io, pageAddr, idx, CacheDataRowAdapter.RowData.NO_KEY);
+
+        return false;  // Stop search.
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCoordinatorVersion() {
+        return Long.MAX_VALUE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCounter() {
+        return Long.MAX_VALUE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MvccKeyMaxVersionBound.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccKeyMinVersionBound.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccKeyMinVersionBound.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccKeyMinVersionBound.java
new file mode 100644
index 0000000..f2ac308
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccKeyMinVersionBound.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.tree;
+
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class MvccKeyMinVersionBound extends SearchRow {
+    /**
+     * @param cacheId Cache ID.
+     * @param key Key.
+     */
+    public MvccKeyMinVersionBound(int cacheId, KeyCacheObject key) {
+        super(cacheId, key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCoordinatorVersion() {
+        return 1L;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCounter() {
+        return 1L;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MvccKeyMinVersionBound.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java
new file mode 100644
index 0000000..79544e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.tree;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinatorVersion;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.GridLongList;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class MvccUpdateRow extends DataRow implements BPlusTree.TreeRowClosure<CacheSearchRow, CacheDataRow> {
+    /** */
+    private Boolean hasPrev;
+
+    /** */
+    private boolean canCleanup;
+
+    /** */
+    private GridLongList activeTxs;
+
+    /** */
+    private List<CacheSearchRow> cleanupRows;
+
+    /** */
+    private final MvccCoordinatorVersion mvccVer;
+
+    /**
+     * @param key Key.
+     * @param val Value.
+     * @param ver Version.
+     * @param mvccVer Mvcc version.
+     * @param part Partition.
+     * @param cacheId Cache ID.
+     */
+    public MvccUpdateRow(
+        KeyCacheObject key,
+        CacheObject val,
+        GridCacheVersion ver,
+        MvccCoordinatorVersion mvccVer,
+        int part,
+        int cacheId) {
+        super(key, val, ver, part, 0L, cacheId);
+
+        this.mvccVer = mvccVer;
+    }
+
+    /**
+     * @return {@code True} if previous value was non-null.
+     */
+    public boolean previousNotNull() {
+        return hasPrev != null && hasPrev;
+    }
+
+    /**
+     * @return Active transactions to wait for.
+     */
+    @Nullable public GridLongList activeTransactions() {
+        return activeTxs;
+    }
+
+    /**
+     * @return Rows which are safe to cleanup.
+     */
+    public List<CacheSearchRow> cleanupRows() {
+        return cleanupRows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(BPlusTree<CacheSearchRow, CacheDataRow> tree,
+        BPlusIO<CacheSearchRow> io,
+        long pageAddr,
+        int idx)
+        throws IgniteCheckedException
+    {
+        RowLinkIO rowIo = (RowLinkIO)io;
+
+        // All previous version should be less then new one.
+        assert mvccVer.coordinatorVersion() >= rowIo.getMvccCoordinatorVersion(pageAddr, idx);
+        assert mvccVer.coordinatorVersion() > rowIo.getMvccCoordinatorVersion(pageAddr, idx) || mvccVer.counter() > rowIo.getMvccCounter(pageAddr, idx);
+
+        boolean checkActive = mvccVer.activeTransactions().size() > 0;
+
+        boolean txActive = false;
+
+        // Suppose transactions on previous coordinator versions are done.
+        if (checkActive && mvccVer.coordinatorVersion() == rowIo.getMvccCoordinatorVersion(pageAddr, idx)) {
+            long rowMvccCntr = rowIo.getMvccCounter(pageAddr, idx);
+
+            if (mvccVer.activeTransactions().contains(rowMvccCntr)) {
+                txActive = true;
+
+                if (activeTxs == null)
+                    activeTxs = new GridLongList();
+
+                activeTxs.add(rowMvccCntr);
+            }
+        }
+
+        if (hasPrev == null)
+            hasPrev = Boolean.TRUE; // TODO IGNITE-3478 support removes.
+
+        if (!txActive) {
+            assert Long.compare(mvccVer.coordinatorVersion(), rowIo.getMvccCoordinatorVersion(pageAddr, idx)) >= 0;
+
+            int cmp;
+
+            if (mvccVer.coordinatorVersion() == rowIo.getMvccCoordinatorVersion(pageAddr, idx))
+                cmp = Long.compare(mvccVer.cleanupVersion(), rowIo.getMvccCounter(pageAddr, idx));
+            else
+                cmp = 1;
+
+            if (cmp >= 0) {
+                // Do not cleanup oldest version.
+                if (canCleanup) {
+                    CacheSearchRow row = io.getLookupRow(tree, pageAddr, idx);
+
+                    assert row.link() != 0 && row.mvccCoordinatorVersion() > 0 : row;
+
+                    // Should not be possible to cleanup active tx.
+                    assert row.mvccCoordinatorVersion() != mvccVer.coordinatorVersion()
+                        || !mvccVer.activeTransactions().contains(row.mvccCounter());
+
+                    if (cleanupRows == null)
+                        cleanupRows = new ArrayList<>();
+
+                    cleanupRows.add(row);
+                }
+                else
+                    canCleanup = true;
+            }
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCoordinatorVersion() {
+        return mvccVer.coordinatorVersion();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCounter() {
+        return mvccVer.counter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MvccUpdateRow.class, this, "super", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccVersionBasedSearchRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccVersionBasedSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccVersionBasedSearchRow.java
new file mode 100644
index 0000000..c829afb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccVersionBasedSearchRow.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.tree;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.mvcc.MvccCoordinatorVersion;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+public class MvccVersionBasedSearchRow extends SearchRow implements BPlusTree.TreeRowClosure<CacheSearchRow, CacheDataRow> {
+    /** */
+    private final MvccCoordinatorVersion ver;
+
+    /** */
+    private CacheDataRow resRow;
+
+    /**
+     * @param cacheId Cache ID.
+     * @param key Key.
+     * @param ver Mvcc version.
+     */
+    public MvccVersionBasedSearchRow(int cacheId, KeyCacheObject key, MvccCoordinatorVersion ver) {
+        super(cacheId, key);
+
+        assert ver != null;
+
+        this.ver = ver;
+    }
+
+    /**
+     * @return Found row.
+     */
+    @Nullable public CacheDataRow row() {
+        return resRow;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(BPlusTree<CacheSearchRow, CacheDataRow> tree,
+        BPlusIO<CacheSearchRow> io,
+        long pageAddr,
+        int idx) throws IgniteCheckedException
+    {
+        boolean visible = true;
+
+        if (ver.activeTransactions().size() > 0) {
+            RowLinkIO rowIo = (RowLinkIO)io;
+
+            // TODO IGNITE-3478 sort active transactions?
+            if (rowIo.getMvccCoordinatorVersion(pageAddr, idx) == ver.coordinatorVersion())
+                visible = !ver.activeTransactions().contains(rowIo.getMvccCounter(pageAddr, idx));
+        }
+
+        if (visible) {
+            resRow = ((CacheDataTree) tree).getRow(io, pageAddr, idx, CacheDataRowAdapter.RowData.NO_KEY);
+
+            return false; // Stop search.
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCoordinatorVersion() {
+        return ver.coordinatorVersion();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long mvccCounter() {
+        return ver.counter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MvccVersionBasedSearchRow.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java
index 8b341cb..111968d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/RowLinkIO.java
@@ -42,7 +42,17 @@ public interface RowLinkIO {
      */
     public int getCacheId(long pageAddr, int idx);
 
-    public long getMvccUpdateTopologyVersion(long pageAddr, int idx);
+    /**
+     * @param pageAddr Page address.
+     * @param idx Index.
+     * @return Mvcc coordinator version.
+     */
+    public long getMvccCoordinatorVersion(long pageAddr, int idx);
 
-    public long getMvccUpdateCounter(long pageAddr, int idx);
+    /**
+     * @param pageAddr Page address.
+     * @param idx Index.
+     * @return Mvcc counter.
+     */
+    public long getMvccCounter(long pageAddr, int idx);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
index f28fe2d..115e8a2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccTransactionsTest.java
@@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteTransactions;
@@ -47,16 +48,20 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.TestCacheNodeExcludingFilter;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.lang.GridInClosure3;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -2534,6 +2539,7 @@ public class CacheMvccTransactionsTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param restartCrd If {@code true} dedicated coordinator node is restarted during test.
      * @param srvs Number of server nodes.
      * @param clients Number of client nodes.
      * @param cacheBackups Number of cache backups.
@@ -2680,6 +2686,149 @@ public class CacheMvccTransactionsTest extends GridCommonAbstractTest {
             stop.set(true);
         }
     }
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testSize() throws Exception {
+        Ignite node = startGrid(0);
+
+        IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 1));
+
+        assertEquals(cache.size(), 0);
+
+        final int KEYS = 10;
+
+        for (int i = 0; i < KEYS; i++) {
+            final Integer key = i;
+
+            try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(key, i);
+
+                tx.commit();
+            }
+
+            assertEquals(i + 1, cache.size());
+        }
+
+        for (int i = 0; i < KEYS; i++) {
+            final Integer key = i;
+
+            try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cache.put(key, i);
+
+                tx.commit();
+            }
+
+            assertEquals(KEYS, cache.size());
+        }
+
+        // TODO IGNITE-3478: test removes.
+    }
+
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInternalApi() throws Exception {
+        Ignite node = startGrid(0);
+
+        IgniteCache cache = node.createCache(cacheConfiguration(PARTITIONED, FULL_SYNC, 0, 1));
+
+        GridCacheContext cctx =
+            ((IgniteKernal)node).context().cache().context().cacheContext(CU.cacheId(cache.getName()));
+
+        CacheCoordinatorsProcessor crd = cctx.kernalContext().coordinators();
+
+        // Start query to prevent cleanup.
+        IgniteInternalFuture<MvccCoordinatorVersion> fut = crd.requestQueryCounter(crd.currentCoordinator());
+
+        fut.get();
+
+        final int KEYS = 1000;
+
+        for (int i = 0; i < 10; i++) {
+            for (int k = 0; k < KEYS; k++) {
+                final Integer key = k;
+
+                try (Transaction tx = node.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    cache.put(key, i);
+
+                    tx.commit();
+                }
+            }
+        }
+
+        for (int k = 0; k < KEYS; k++) {
+            final Integer key = k;
+
+            KeyCacheObject key0 = cctx.toCacheKeyObject(key);
+
+            List<T2<Object, MvccCounter>> vers = cctx.offheap().mvccAllVersions(cctx, key0);
+
+            assertEquals(10, vers.size());
+
+            CacheDataRow row = cctx.offheap().read(cctx, key0);
+
+            checkRow(cctx, row, key0, vers.get(0).get1());
+
+            for (T2<Object, MvccCounter> ver : vers) {
+                MvccCounter cntr = ver.get2();
+
+                MvccCoordinatorVersion readVer =
+                    new MvccCoordinatorVersionResponse(cntr.coordinatorVersion(), cntr.counter(), 0);
+
+                row = cctx.offheap().mvccRead(cctx, key0, readVer);
+
+                checkRow(cctx, row, key0, ver.get1());
+            }
+
+            checkRow(cctx,
+                cctx.offheap().mvccRead(cctx, key0, version(vers.get(0).get2().coordinatorVersion() + 1, 1)),
+                key0,
+                vers.get(0).get1());
+
+            checkRow(cctx,
+                cctx.offheap().mvccRead(cctx, key0, version(vers.get(0).get2().coordinatorVersion(), vers.get(0).get2().counter() + 1)),
+                key0,
+                vers.get(0).get1());
+
+            MvccCoordinatorVersionResponse ver = version(crd.currentCoordinator().coordinatorVersion(), 100000);
+
+            for (int v = 0; v < vers.size(); v++) {
+                MvccCounter cntr = vers.get(v).get2();
+
+                ver.addTx(cntr.counter());
+
+                row = cctx.offheap().mvccRead(cctx, key0, ver);
+
+                if (v == vers.size() - 1)
+                    assertNull(row);
+                else
+                    checkRow(cctx, row, key0, vers.get(v + 1).get1());
+            }
+        }
+    }
+
+    /**
+     * @param cctx Context.
+     * @param row Row.
+     * @param expKey Expected row key.
+     * @param expVal Expected row value.
+     */
+    private void checkRow(GridCacheContext cctx, CacheDataRow row, KeyCacheObject expKey, Object expVal) {
+        assertNotNull(row);
+        assertEquals(expKey, row.key());
+        assertEquals(expVal, row.value().value(cctx.cacheObjectContext(), false));
+    }
+
+    /**
+     * @param crdVer Coordinator version.
+     * @param cntr Counter.
+     * @return Version.
+     */
+    private MvccCoordinatorVersionResponse version(long crdVer, long cntr) {
+        return new MvccCoordinatorVersionResponse(crdVer, cntr, 0);
+    }
 
     /**
      * @return Cache configurations.

http://git-wip-us.apache.org/repos/asf/ignite/blob/fd53c1a8/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index 9c0d791..e2f6b2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -25,6 +25,7 @@ import java.util.Random;
 import java.util.TreeMap;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -570,6 +571,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
             assertNoLocks();
 
             assertEquals(x, tree.findOne(x).longValue());
+            checkIterate(tree, x, x, x, true);
 
             assertNoLocks();
 
@@ -584,12 +586,15 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
         assertNull(tree.findOne(-1L));
 
-        for (long x = 0; x < cnt; x++)
+        for (long x = 0; x < cnt; x++) {
             assertEquals(x, tree.findOne(x).longValue());
+            checkIterate(tree, x, x, x, true);
+        }
 
         assertNoLocks();
 
         assertNull(tree.findOne(cnt));
+        checkIterate(tree, cnt, cnt, null, false);
 
         for (long x = RMV_INC > 0 ? 0 : cnt - 1; x >= 0 && x < cnt; x += RMV_INC) {
             X.println(" -- " + x);
@@ -603,6 +608,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
             assertNoLocks();
 
             assertNull(tree.findOne(x));
+            checkIterate(tree, x, x, null, false);
 
             assertNoLocks();
 
@@ -619,6 +625,32 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param tree
+     * @param lower
+     * @param upper
+     * @param exp
+     * @param expFound
+     * @throws IgniteCheckedException
+     */
+    private void checkIterate(TestTree tree, long lower, long upper, Long exp, boolean expFound)
+        throws IgniteCheckedException {
+        TestTreeRowClosure c = new TestTreeRowClosure(exp);
+
+        tree.iterate(lower, upper, c);
+
+        assertEquals(expFound, c.found);
+    }
+
+    private void checkIterateC(TestTree tree, long lower, long upper, TestTreeRowClosure c, boolean expFound)
+        throws IgniteCheckedException {
+        c.found = false;
+
+        tree.iterate(lower, upper, c);
+
+        assertEquals(expFound, c.found);
+    }
+
+    /**
      * @throws IgniteCheckedException If failed.
      */
     public void testRandomInvoke_1_30_1() throws IgniteCheckedException {
@@ -1242,6 +1274,206 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testIterate() throws Exception {
+        MAX_PER_PAGE = 5;
+
+        TestTree tree = createTestTree(true);
+
+        checkIterate(tree, 0L, 100L, null, false);
+
+        for (long idx = 1L; idx <= 10L; ++idx)
+            tree.put(idx);
+
+        for (long idx = 1L; idx <= 10L; ++idx)
+            checkIterate(tree, idx, 100L, idx, true);
+
+        checkIterate(tree, 0L, 100L, 1L, true);
+
+        for (long idx = 1L; idx <= 10L; ++idx)
+            checkIterate(tree, idx, 100L, 10L, true);
+
+        checkIterate(tree, 0L, 100L, 100L, false);
+
+        for (long idx = 1L; idx <= 10L; ++idx)
+            checkIterate(tree, 0L, 100L, idx, true);
+
+        for (long idx = 0L; idx <= 10L; ++idx)
+            checkIterate(tree, idx, 11L, -1L, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIterateConcurrentPutRemove() throws Exception {
+        findOneBoundedConcurrentPutRemove();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIterateConcurrentPutRemove_1() throws Exception {
+        MAX_PER_PAGE = 1;
+
+        findOneBoundedConcurrentPutRemove();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIterateConcurrentPutRemove_5() throws Exception {
+        MAX_PER_PAGE = 5;
+
+        findOneBoundedConcurrentPutRemove();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIteratePutRemove_10() throws Exception {
+        MAX_PER_PAGE = 10;
+
+        findOneBoundedConcurrentPutRemove();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void findOneBoundedConcurrentPutRemove() throws Exception {
+        final TestTree tree = createTestTree(true);
+
+        final int KEYS = 10_000;
+
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 10; i++) {
+            for (long idx = 0L; idx < KEYS; ++idx)
+                tree.put(idx);
+
+            final Long findKey;
+
+            if (MAX_PER_PAGE > 0) {
+                switch (i) {
+                    case 0:
+                        findKey = 1L;
+
+                        break;
+
+                    case 1:
+                        findKey = (long)MAX_PER_PAGE;
+
+                        break;
+
+                    case 2:
+                        findKey = (long)MAX_PER_PAGE - 1;
+
+                        break;
+
+                    case 3:
+                        findKey = (long)MAX_PER_PAGE + 1;
+
+                        break;
+
+                    case 4:
+                        findKey = (long)(KEYS / MAX_PER_PAGE / 2) * MAX_PER_PAGE;
+
+                        break;
+
+                    case 5:
+                        findKey = (long)(KEYS / MAX_PER_PAGE / 2) * MAX_PER_PAGE - 1;
+
+                        break;
+
+                    case 6:
+                        findKey = (long)(KEYS / MAX_PER_PAGE / 2) * MAX_PER_PAGE + 1;
+
+                        break;
+
+                    case 7:
+                        findKey = (long)KEYS - 1;
+
+                        break;
+
+                    default:
+                        findKey = rnd.nextLong(KEYS);
+                }
+            }
+            else
+                findKey = rnd.nextLong(KEYS);
+
+            info("Iteration [iter=" + i + ", key=" + findKey + ']');
+
+            assertEquals(findKey, tree.findOne(findKey));
+            checkIterate(tree, findKey, findKey, findKey, true);
+
+            IgniteInternalFuture getFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    TestTreeRowClosure p = new TestTreeRowClosure(findKey);
+
+                    TestTreeRowClosure falseP = new TestTreeRowClosure(-1L);
+
+                    int cnt = 0;
+
+                    while (!stop.get()) {
+                        int shift = MAX_PER_PAGE > 0 ? rnd.nextInt(MAX_PER_PAGE * 2) : rnd.nextInt(100);
+
+                        checkIterateC(tree, findKey, findKey, p, true);
+
+                        checkIterateC(tree, findKey - shift, findKey, p, true);
+
+                        checkIterateC(tree, findKey - shift, findKey + shift, p, true);
+
+                        checkIterateC(tree, findKey, findKey + shift, p, true);
+
+                        checkIterateC(tree, -100L, KEYS + 100L, falseP, false);
+
+                        cnt++;
+                    }
+
+                    info("Done, read count: " + cnt);
+
+                    return null;
+                }
+            }, 10, "find");
+
+            asyncRunFut = new GridCompoundFuture<>();
+
+            asyncRunFut.add(getFut);
+
+            asyncRunFut.markInitialized();
+
+            try {
+                U.sleep(100);
+
+                for (int j = 0; j < 20; j++) {
+                    for (long idx = 0L; idx < KEYS / 2; ++idx) {
+                        long toRmv = rnd.nextLong(KEYS);
+
+                        if (toRmv != findKey)
+                            tree.remove(toRmv);
+                    }
+
+                    for (long idx = 0L; idx < KEYS / 2; ++idx) {
+                        long put = rnd.nextLong(KEYS);
+
+                        tree.put(put);
+                    }
+                }
+            }
+            finally {
+                stop.set(true);
+            }
+
+            asyncRunFut.get();
+
+            stop.set(false);
+        }
+    }
+
+    /**
      *
      */
     public void testConcurrentGrowDegenerateTreeAndConcurrentRemove() throws Exception {
@@ -1449,6 +1681,17 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
                         last = c.get();
                     }
+
+                    TestTreeFindFirstClosure cl = new TestTreeFindFirstClosure();
+
+                    tree.iterate((long)low, (long)high, cl);
+
+                    last = cl.val;
+
+                    if (last != null) {
+                        assertTrue(low + " <= " + last + " <= " + high, last >= low);
+                        assertTrue(low + " <= " + last + " <= " + high, last <= high);
+                    }
                 }
 
                 return null;
@@ -1853,4 +2096,50 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
             return PageUtils.getLong(pageAddr, offset(idx));
         }
     }
+
+    /**
+     *
+     */
+    static class TestTreeRowClosure implements BPlusTree.TreeRowClosure<Long, Long> {
+        /** */
+        private final Long expVal;
+
+        /** */
+        private boolean found;
+
+        /**
+         * @param expVal Value to find or {@code null} to find first.
+         */
+        TestTreeRowClosure(Long expVal) {
+            this.expVal = expVal;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(BPlusTree<Long, Long> tree, BPlusIO<Long> io, long pageAddr, int idx)
+            throws IgniteCheckedException {
+            assert !found;
+
+            found = expVal == null || io.getLookupRow(tree, pageAddr, idx).equals(expVal);
+
+            return !found;
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestTreeFindFirstClosure implements BPlusTree.TreeRowClosure<Long, Long> {
+        /** */
+        private Long val;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(BPlusTree<Long, Long> tree, BPlusIO<Long> io, long pageAddr, int idx)
+            throws IgniteCheckedException {
+            assert val == null;
+
+            val = io.getLookupRow(tree, pageAddr, idx);
+
+            return false;
+        }
+    }
 }


[20/29] ignite git commit: IGNITE-6463 Web Console: Fixed output of big numbers in SQL query results.

Posted by sb...@apache.org.
IGNITE-6463 Web Console: Fixed output of big numbers in SQL query results.


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

Branch: refs/heads/ignite-5937
Commit: 35589a776a33285c212ddb7e0d51982cb61ce338
Parents: 79d94cf
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Sat Oct 7 00:11:37 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Sat Oct 7 00:11:37 2017 +0700

----------------------------------------------------------------------
 modules/web-console/backend/package.json                      | 4 +++-
 .../frontend/app/modules/agent/decompress.worker.js           | 3 ++-
 modules/web-console/frontend/package.json                     | 7 +++++--
 3 files changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/35589a77/modules/web-console/backend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/package.json b/modules/web-console/backend/package.json
index 837f41c..07af45f 100644
--- a/modules/web-console/backend/package.json
+++ b/modules/web-console/backend/package.json
@@ -17,7 +17,9 @@
     }
   ],
   "license": "Apache-2.0",
-  "keywords": "grid",
+  "keywords": [
+    "Apache Ignite Web console"
+  ],
   "homepage": "https://ignite.apache.org/",
   "engines": {
     "npm": "^3.x.x",

http://git-wip-us.apache.org/repos/asf/ignite/blob/35589a77/modules/web-console/frontend/app/modules/agent/decompress.worker.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/decompress.worker.js b/modules/web-console/frontend/app/modules/agent/decompress.worker.js
index d8e176d..2fd294d 100644
--- a/modules/web-console/frontend/app/modules/agent/decompress.worker.js
+++ b/modules/web-console/frontend/app/modules/agent/decompress.worker.js
@@ -17,6 +17,7 @@
 
 import _ from 'lodash';
 import pako from 'pako';
+import bigIntJSON from 'json-bigint';
 
 /** This worker decode & decompress BASE64/Zipped data and parse to JSON. */
 // eslint-disable-next-line no-undef
@@ -27,7 +28,7 @@ onmessage = function(e) {
 
     const unzipped = pako.inflate(binaryString, {to: 'string'});
 
-    const res = JSON.parse(unzipped);
+    const res = bigIntJSON({storeAsString: true}).parse(unzipped);
 
     postMessage(_.get(res, 'result', res));
 };

http://git-wip-us.apache.org/repos/asf/ignite/blob/35589a77/modules/web-console/frontend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json
index 2083640..82c3eea 100644
--- a/modules/web-console/frontend/package.json
+++ b/modules/web-console/frontend/package.json
@@ -18,7 +18,9 @@
     }
   ],
   "license": "Apache-2.0",
-  "keywords": "grid",
+  "keywords": [
+    "Apache Ignite Web console"
+  ],
   "homepage": "https://ignite.apache.org/",
   "engines": {
     "npm": "3.x.x",
@@ -47,7 +49,7 @@
     "angular-touch": "1.5.11",
     "angular-translate": "2.15.2",
     "angular-tree-control": "0.2.28",
-    "angular-ui-grid": "4.0.6",
+    "angular-ui-grid": "4.0.7",
     "@uirouter/angularjs": "1.0.5",
     "babel-core": "6.25.0",
     "babel-eslint": "7.2.3",
@@ -76,6 +78,7 @@
     "html-webpack-plugin": "2.29.0",
     "jquery": "3.2.1",
     "json-loader": "0.5.7",
+    "json-bigint": "0.2.3",
     "jszip": "3.1.4",
     "lodash": "4.17.4",
     "node-sass": "4.5.3",


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

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


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

Branch: refs/heads/ignite-5937
Commit: c1b2c03dc1ee9de222997cba4efcb2e5fb1a5885
Parents: c553638
Author: sboikov <sb...@gridgain.com>
Authored: Mon Oct 9 17:05:02 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 9 17:50:53 2017 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheOffheapManagerImpl.java    | 285 +++++++------------
 .../GridDhtPartitionsExchangeFuture.java        |   2 +-
 .../cache/mvcc/CacheCoordinatorsProcessor.java  |  32 ++-
 .../cache/mvcc/CacheMvccClusterRestartTest.java | 173 +++++++++++
 4 files changed, 308 insertions(+), 184 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b2c03d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index dd4d7e0..80d36c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -1356,15 +1356,6 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             return dataRow;
         }
 
-        private int compare(CacheDataRow row, long crdVer, long mvccCntr) {
-            int cmp = Long.compare(row.mvccCoordinatorVersion(), crdVer);
-
-            if (cmp != 0)
-                return cmp;
-
-            return Long.compare(row.mvccCounter(), mvccCntr);
-        }
-
         /** {@inheritDoc} */
         @Override public GridLongList mvccRemove(GridCacheContext cctx,
             boolean primary,
@@ -1376,9 +1367,67 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
                 throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
 
             try {
+                int cacheId = grp.storeCacheIdInDataPage() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID;
 
+                CacheObjectContext coCtx = cctx.cacheObjectContext();
 
-                return null;
+                // Make sure value bytes initialized.
+                key.valueBytes(coCtx);
+
+                MvccUpdateRow updateRow = new MvccUpdateRow(
+                    key,
+                    null,
+                    null,
+                    mvccVer,
+                    partId,
+                    cacheId);
+
+                rowStore.addRow(updateRow);
+
+                assert updateRow.link() != 0 : updateRow;
+
+                if (grp.sharedGroup() && updateRow.cacheId() == CU.UNDEFINED_CACHE_ID)
+                    updateRow.cacheId(cctx.cacheId());
+
+                GridLongList waitTxs = null;
+
+                if (mvccVer.initialLoad()) {
+                    boolean old = dataTree.putx(updateRow);
+
+                    assert !old;
+
+                    incrementSize(cctx.cacheId());
+                }
+                else {
+                    dataTree.iterate(updateRow, new MvccKeyMinVersionBound(cacheId, key), updateRow);
+
+                    boolean old = dataTree.putx(updateRow);
+
+                    assert !old;
+
+                    if (!updateRow.previousNotNull())
+                        incrementSize(cctx.cacheId());
+
+                    waitTxs = updateRow.activeTransactions();
+
+                    List<CacheSearchRow> cleanupRows = updateRow.cleanupRows();
+
+                    if (cleanupRows != null) {
+                        for (int i = 0; i < cleanupRows.size(); i++) {
+                            CacheSearchRow oldRow = cleanupRows.get(i);
+
+                            assert oldRow.link() != 0L : oldRow;
+
+                            boolean rmvd = dataTree.removex(oldRow);
+
+                            assert rmvd;
+
+                            rowStore.removeRow(oldRow.link());
+                        }
+                    }
+                }
+
+                return waitTxs;
             }
             finally {
                 busyLock.leaveBusy();
@@ -1407,135 +1456,60 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
                 key.valueBytes(coCtx);
                 val.valueBytes(coCtx);
 
-                if (true) {
-                    MvccUpdateRow updateRow = new MvccUpdateRow(
-                        key,
-                        val,
-                        ver,
-                        mvccVer,
-                        partId,
-                        cacheId);
-
-                    rowStore.addRow(updateRow);
-
-                    assert updateRow.link() != 0 : updateRow;
-
-                    if (grp.sharedGroup() && updateRow.cacheId() == CU.UNDEFINED_CACHE_ID)
-                        updateRow.cacheId(cctx.cacheId());
-
-                    GridLongList waitTxs = null;
-
-                    if (mvccVer.initialLoad()) {
-                        boolean old = dataTree.putx(updateRow);
-
-                        assert !old;
-
-                        incrementSize(cctx.cacheId());
-                    }
-                    else {
-                        dataTree.iterate(updateRow, new MvccKeyMinVersionBound(cacheId, key), updateRow);
-
-                        boolean old = dataTree.putx(updateRow);
-
-                        assert !old;
-
-                        if (!updateRow.previousNotNull())
-                            incrementSize(cctx.cacheId());
-
-                        waitTxs = updateRow.activeTransactions();
+                MvccUpdateRow updateRow = new MvccUpdateRow(
+                    key,
+                    val,
+                    ver,
+                    mvccVer,
+                    partId,
+                    cacheId);
 
-                        List<CacheSearchRow> cleanupRows = updateRow.cleanupRows();
+                rowStore.addRow(updateRow);
 
-                        if (cleanupRows != null) {
-                            for (int i = 0; i < cleanupRows.size(); i++) {
-                                CacheSearchRow oldRow = cleanupRows.get(i);
+                assert updateRow.link() != 0 : updateRow;
 
-                                assert oldRow.link() != 0L : oldRow;
+                if (grp.sharedGroup() && updateRow.cacheId() == CU.UNDEFINED_CACHE_ID)
+                    updateRow.cacheId(cctx.cacheId());
 
-                                boolean rmvd = dataTree.removex(oldRow);
+                GridLongList waitTxs = null;
 
-                                assert rmvd;
+                if (mvccVer.initialLoad()) {
+                    boolean old = dataTree.putx(updateRow);
 
-                                rowStore.removeRow(oldRow.link());
-                            }
-                        }
-                    }
+                    assert !old;
 
-                    return waitTxs;
+                    incrementSize(cctx.cacheId());
                 }
                 else {
-                    MvccDataRow dataRow = new MvccDataRow(
-                        key,
-                        val,
-                        ver,
-                        partId,
-                        cacheId,
-                        mvccVer.coordinatorVersion(),
-                        mvccVer.counter());
-
-                    rowStore.addRow(dataRow);
+                    dataTree.iterate(updateRow, new MvccKeyMinVersionBound(cacheId, key), updateRow);
 
-                    assert dataRow.link() != 0 : dataRow;
-
-                    if (grp.sharedGroup() && dataRow.cacheId() == CU.UNDEFINED_CACHE_ID)
-                        dataRow.cacheId(cctx.cacheId());
-
-                    boolean old = dataTree.putx(dataRow);
+                    boolean old = dataTree.putx(updateRow);
 
                     assert !old;
 
-                    GridLongList waitTxs = null;
-
-                    if (!mvccVer.initialLoad()) {
-                        MvccLongList activeTxs = mvccVer.activeTransactions();
-
-                        // TODO IGNITE-3484: need special method.
-                        GridCursor<CacheDataRow> cur = dataTree.find(
-                            new MvccSearchRow(cacheId, key, mvccVer.coordinatorVersion(), mvccVer.counter() - 1),
-                            new MvccSearchRow(cacheId, key, 1, 1));
-
-                        boolean first = true;
-
-                        boolean activeTx = false;
-
-                        while (cur.next()) {
-                            CacheDataRow oldVal = cur.get();
-
-                            assert oldVal.link() != 0 : oldVal;
-
-                            if (activeTxs != null && oldVal.mvccCoordinatorVersion() == mvccVer.coordinatorVersion() &&
-                                activeTxs.contains(oldVal.mvccCounter())) {
-                                if (waitTxs == null)
-                                    waitTxs = new GridLongList();
+                    if (!updateRow.previousNotNull())
+                        incrementSize(cctx.cacheId());
 
-                                assert oldVal.mvccCounter() != mvccVer.counter();
+                    waitTxs = updateRow.activeTransactions();
 
-                                waitTxs.add(oldVal.mvccCounter());
+                    List<CacheSearchRow> cleanupRows = updateRow.cleanupRows();
 
-                                activeTx = true;
-                            }
+                    if (cleanupRows != null) {
+                        for (int i = 0; i < cleanupRows.size(); i++) {
+                            CacheSearchRow oldRow = cleanupRows.get(i);
 
-                            if (!activeTx) {
-                                // Should not delete oldest version which is less than cleanup version.
-                                int cmp = compare(oldVal, mvccVer.coordinatorVersion(), mvccVer.cleanupVersion());
+                            assert oldRow.link() != 0L : oldRow;
 
-                                if (cmp <= 0) {
-                                    if (first)
-                                        first = false;
-                                    else {
-                                        boolean rmvd = dataTree.removex(oldVal);
+                            boolean rmvd = dataTree.removex(oldRow);
 
-                                        assert rmvd;
+                            assert rmvd;
 
-                                        rowStore.removeRow(oldVal.link());
-                                    }
-                                }
-                            }
+                            rowStore.removeRow(oldRow.link());
                         }
                     }
-
-                    return waitTxs;
                 }
+
+                return waitTxs;
             }
             finally {
                 busyLock.leaveBusy();
@@ -1746,26 +1720,15 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
             CacheDataRow row;
 
             if (grp.mvccEnabled()) {
-                if (true) {
-                    MvccKeyMaxVersionBound searchRow = new MvccKeyMaxVersionBound(cacheId, key);
-
-                    dataTree.iterate(
-                        searchRow,
-                        new MvccKeyMinVersionBound(cacheId, key),
-                        searchRow // Use the same instance as closure to do not create extra object.
-                    );
+                MvccKeyMaxVersionBound searchRow = new MvccKeyMaxVersionBound(cacheId, key);
 
-                    row = searchRow.row();
-                }
-                else {
-                    GridCursor<CacheDataRow> cur = dataTree.find(new MvccSearchRow(cacheId, key, Long.MAX_VALUE, Long.MAX_VALUE),
-                        new MvccSearchRow(cacheId, key, 1, 1));
+                dataTree.iterate(
+                    searchRow,
+                    new MvccKeyMinVersionBound(cacheId, key),
+                    searchRow // Use the same instance as closure to do not create extra object.
+                );
 
-                    if (cur.next())
-                        row = cur.get();
-                    else
-                        row = null;
-                }
+                row = searchRow.row();
             }
             else
                 row = dataTree.findOne(new SearchRow(cacheId, key), CacheDataRowAdapter.RowData.NO_KEY);
@@ -1818,55 +1781,19 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
 
             int cacheId = grp.sharedGroup() ? cctx.cacheId() : CU.UNDEFINED_CACHE_ID;
 
-            if (true) {
-                MvccVersionBasedSearchRow lower = new MvccVersionBasedSearchRow(cacheId, key, ver);
-
-                dataTree.iterate(
-                    lower,
-                    new MvccKeyMinVersionBound(cacheId, key),
-                    lower // Use the same instance as closure to do not create extra object.
-                );
-
-                CacheDataRow row = lower.row();
-
-                afterRowFound(row, key);
-
-                return row;
-            }
-            else {
-                GridCursor<CacheDataRow> cur = dataTree.find(
-                    new MvccSearchRow(cacheId, key, ver.coordinatorVersion(), ver.counter()),
-                    new MvccSearchRow(cacheId, key, 1, 1));
-
-                CacheDataRow row = null;
+            MvccVersionBasedSearchRow lower = new MvccVersionBasedSearchRow(cacheId, key, ver);
 
-                MvccLongList txs = ver.activeTransactions();
+            dataTree.iterate(
+                lower,
+                new MvccKeyMinVersionBound(cacheId, key),
+                lower // Use the same instance as closure to do not create extra object.
+            );
 
-                while (cur.next()) {
-                    CacheDataRow row0 = cur.get();
+            CacheDataRow row = lower.row();
 
-                    assert row0.mvccCoordinatorVersion() > 0 : row0;
-
-                    boolean visible;
-
-                    if (txs != null) {
-                        visible = row0.mvccCoordinatorVersion() != ver.coordinatorVersion()
-                            || !txs.contains(row0.mvccCounter());
-                    }
-                    else
-                        visible = true;
-
-                    if (visible) {
-                        row = row0;
-
-                        break;
-                    }
-                }
-
-                assert row == null || key.equals(row.key());
+            afterRowFound(row, key);
 
-                return row;
-            }
+            return row;
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b2c03d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 830d50b..88095ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -560,7 +560,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
             MvccCoordinator mvccCrd = firstEvtDiscoCache.mvccCoordinator();
 
             boolean mvccCrdChange = mvccCrd != null &&
-                initialVersion().equals(mvccCrd.topologyVersion());
+                (initialVersion().equals(mvccCrd.topologyVersion()) || activateCluster());
 
             cctx.kernalContext().coordinators().currentCoordinator(mvccCrd);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b2c03d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java
index b9b8ea1..54fb3c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/CacheCoordinatorsProcessor.java
@@ -66,6 +66,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED;
 import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE_COORDINATOR;
+import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 
 /**
@@ -86,7 +87,13 @@ public class CacheCoordinatorsProcessor extends GridProcessorAdapter {
 
     /** */
     private static final byte MSG_POLICY = SYSTEM_POOL;
-    
+
+    /** */
+    private static final long CRD_VER_MASK = 0x3F_FF_FF_FF_FF_FF_FF_FFL;
+
+    /** */
+    private static final long RMVD_VAL_VER_MASK = 0x80_00_00_00_00_00_00_00L;
+
     /** */
     private volatile MvccCoordinator curCrd;
 
@@ -139,6 +146,21 @@ public class CacheCoordinatorsProcessor extends GridProcessorAdapter {
         super(ctx);
     }
 
+    public static int compareCoordinatorVersions(long crdVer1, long crdVer2) {
+        crdVer1 = CRD_VER_MASK & crdVer1;
+        crdVer2 = CRD_VER_MASK & crdVer2;
+
+        return Long.compare(crdVer1, crdVer2);
+    }
+
+    public long createVersionForRemovedValue(long crdVer) {
+        return crdVer | RMVD_VAL_VER_MASK;
+    }
+
+    public boolean versionForRemovedValue(long crdVer) {
+        return (crdVer & RMVD_VAL_VER_MASK) != 0;
+    }
+
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         statCntrs = new StatCounter[7];
@@ -199,7 +221,7 @@ public class CacheCoordinatorsProcessor extends GridProcessorAdapter {
      * @param topVer Topology version.
      */
     public void onDiscoveryEvent(int evtType, Collection<ClusterNode> nodes, long topVer) {
-        if (evtType == EVT_NODE_METRICS_UPDATED)
+        if (evtType == EVT_NODE_METRICS_UPDATED || evtType == EVT_DISCOVERY_CUSTOM_EVT)
             return;
 
         MvccCoordinator crd;
@@ -778,7 +800,9 @@ public class CacheCoordinatorsProcessor extends GridProcessorAdapter {
     private MvccCoordinatorVersionResponse assignQueryCounter(UUID qryNodeId, long futId) {
         assert crdVer != 0;
 
-        return activeQueries.assignQueryCounter(qryNodeId, futId);
+        MvccCoordinatorVersionResponse res = activeQueries.assignQueryCounter(qryNodeId, futId);
+
+        return res;
 
 //        MvccCoordinatorVersionResponse res = new MvccCoordinatorVersionResponse();
 //
@@ -989,7 +1013,7 @@ public class CacheCoordinatorsProcessor extends GridProcessorAdapter {
         log.info("Initialize local node as mvcc coordinator [node=" + ctx.localNodeId() +
             ", topVer=" + topVer + ']');
 
-        crdVer = topVer.topologyVersion();
+        crdVer = topVer.topologyVersion() + ctx.discovery().gridStartTime();
 
         prevCrdQueries.init(activeQueries, discoCache, ctx.discovery());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1b2c03d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java
new file mode 100644
index 0000000..ed7b62d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/mvcc/CacheMvccClusterRestartTest.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.mvcc;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ *
+ */
+public class CacheMvccClusterRestartTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setConsistentId(gridName);
+
+        cfg.setMvccEnabled(true);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        MemoryConfiguration memCfg = new MemoryConfiguration();
+        memCfg.setPageSize(1024);
+        memCfg.setDefaultMemoryPolicySize(100 * 1024 * 1024);
+
+        cfg.setMemoryConfiguration(memCfg);
+
+        cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration().setWalMode(WALMode.LOG_ONLY));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        GridTestUtils.deleteDbFiles();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        GridTestUtils.deleteDbFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        GridTestUtils.deleteDbFiles();
+
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestart1() throws Exception {
+       restart1(3, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestart2() throws Exception {
+        restart1(1, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRestart3() throws Exception {
+        restart1(3, 1);
+    }
+
+    /**
+     * @param srvBefore Number of servers before restart.
+     * @param srvAfter Number of servers after restart.
+     * @throws Exception If failed.
+     */
+    private void restart1(int srvBefore, int srvAfter) throws Exception {
+        Ignite srv0 = startGridsMultiThreaded(srvBefore);
+
+        srv0.active(true);
+
+        IgniteCache<Object, Object> cache = srv0.createCache(cacheConfiguration());
+
+        Set<Integer> keys = new HashSet<>(primaryKeys(cache, 1, 0));
+
+        try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            for (Integer k : keys)
+                cache.put(k, k);
+
+            tx.commit();
+        }
+
+        stopAllGrids();
+
+        srv0 = startGridsMultiThreaded(srvAfter);
+
+        srv0.active(true);
+
+        cache = srv0.cache(DEFAULT_CACHE_NAME);
+
+        Map<Object, Object> res = cache.getAll(keys);
+
+        assertEquals(keys.size(), res.size());
+
+        for (Integer k : keys)
+            assertEquals(k, cache.get(k));
+
+        try (Transaction tx = srv0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            for (Integer k : keys)
+                cache.put(k, k + 1);
+
+            tx.commit();
+        }
+
+        for (Integer k : keys)
+            assertEquals(k + 1, cache.get(k));
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Object, Object> cacheConfiguration() {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>(DEFAULT_CACHE_NAME);
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setBackups(2);
+
+        return ccfg;
+    }
+}


[27/29] ignite git commit: ignite-5937

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


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

Branch: refs/heads/ignite-5937
Commit: 0bbbbe2909718a171f2c4e31d39851220f4b646a
Parents: 5c195c3
Author: sboikov <sb...@gridgain.com>
Authored: Mon Oct 9 16:24:30 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 9 16:24:30 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     | 20 ++++++++--
 .../cache/IgniteCacheOffheapManager.java        | 20 +++++++++-
 .../cache/IgniteCacheOffheapManagerImpl.java    | 39 +++++++++++++++++++-
 .../persistence/GridCacheOffheapManager.java    | 17 ++++++++-
 .../processors/cache/tree/MvccUpdateRow.java    | 28 ++++++++++++--
 5 files changed, 112 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0bbbbe29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 00307a5..4b71a7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -1012,7 +1012,11 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (cctx.mvccEnabled() && !((IgniteCacheOffheapManagerImpl)cctx.offheap()).IGNITE_FAKE_MVCC_STORAGE) {
                 assert mvccVer != null;
 
-                mvccWaitTxs = cctx.offheap().mvccUpdate(this, val, newVer, mvccVer);
+                mvccWaitTxs = cctx.offheap().mvccUpdate(tx.local(),
+                    this,
+                    val,
+                    newVer,
+                    mvccVer);
             }
             else
                 storeValue(val, expireTime, newVer, null);
@@ -1141,6 +1145,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
         boolean marked = false;
 
+        GridLongList mvccWaitTxs = null;
+
         synchronized (this) {
             checkObsolete();
 
@@ -1181,7 +1187,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 }
             }
 
-            removeValue();
+            if (cctx.mvccEnabled() && !((IgniteCacheOffheapManagerImpl)cctx.offheap()).IGNITE_FAKE_MVCC_STORAGE) {
+                assert mvccVer != null;
+
+                mvccWaitTxs = cctx.offheap().mvccRemove(tx.local(), this, mvccVer);
+            }
+            else
+                removeValue();
 
             update(null, 0, 0, newVer, true);
 
@@ -1292,7 +1304,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             cctx.config().getInterceptor().onAfterRemove(entry0);
 
         if (valid)
-            return new GridCacheUpdateTxResult(true, updateCntr0, null);
+            return new GridCacheUpdateTxResult(true, updateCntr0, mvccWaitTxs);
         else
             return new GridCacheUpdateTxResult(false);
     }
@@ -2593,7 +2605,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 if (val != null) {
                     if (cctx.mvccEnabled())
-                        cctx.offheap().mvccUpdate(this, val, ver, mvccVer);
+                        cctx.offheap().mvccUpdate(false, this, val, ver, mvccVer);
                     else
                         storeValue(val, expTime, ver, null);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0bbbbe29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
index 8967ce8..bee2108 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
@@ -196,10 +196,19 @@ public interface IgniteCacheOffheapManager {
      * @return Transactions to wait for before finishing current transaction.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public GridLongList mvccUpdate(GridCacheMapEntry entry,
+    @Nullable public GridLongList mvccUpdate(
+        boolean primary,
+        GridCacheMapEntry entry,
         CacheObject val,
         GridCacheVersion ver,
-        MvccCoordinatorVersion mvccVer) throws IgniteCheckedException;
+        MvccCoordinatorVersion mvccVer
+    ) throws IgniteCheckedException;
+
+    @Nullable public GridLongList mvccRemove(
+        boolean primary,
+        GridCacheMapEntry entry,
+        MvccCoordinatorVersion mvccVer
+    ) throws IgniteCheckedException;
 
     /**
      * @param cctx Cache context.
@@ -503,11 +512,18 @@ public interface IgniteCacheOffheapManager {
          */
         @Nullable GridLongList mvccUpdate(
             GridCacheContext cctx,
+            boolean primary,
             KeyCacheObject key,
             CacheObject val,
             GridCacheVersion ver,
             MvccCoordinatorVersion mvccVer) throws IgniteCheckedException;
 
+        @Nullable GridLongList mvccRemove(
+            GridCacheContext cctx,
+            boolean primary,
+            KeyCacheObject key,
+            MvccCoordinatorVersion mvccVer) throws IgniteCheckedException;
+
         /**
          * @param cctx Cache context.
          * @param key Key.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0bbbbe29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 25f36b2..dd4d7e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -380,11 +380,14 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
     }
 
     /** {@inheritDoc} */
-    @Override public GridLongList mvccUpdate(GridCacheMapEntry entry,
+    @Override public GridLongList mvccUpdate(
+        boolean primary,
+        GridCacheMapEntry entry,
         CacheObject val,
         GridCacheVersion ver,
         MvccCoordinatorVersion mvccVer) throws IgniteCheckedException {
         return dataStore(entry.localPartition()).mvccUpdate(entry.context(),
+            primary,
             entry.key(),
             val,
             ver,
@@ -392,6 +395,19 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
     }
 
     /** {@inheritDoc} */
+    @Override public GridLongList mvccRemove(
+        boolean primary,
+        GridCacheMapEntry entry,
+        MvccCoordinatorVersion mvccVer
+    )
+        throws IgniteCheckedException {
+        return dataStore(entry.localPartition()).mvccRemove(entry.context(),
+            primary,
+            entry.key(),
+            mvccVer);
+    }
+
+    /** {@inheritDoc} */
     @Override public void updateIndexes(GridCacheContext cctx, KeyCacheObject key, GridDhtLocalPartition part)
         throws IgniteCheckedException {
         dataStore(part).updateIndexes(cctx, key);
@@ -1350,8 +1366,29 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         }
 
         /** {@inheritDoc} */
+        @Override public GridLongList mvccRemove(GridCacheContext cctx,
+            boolean primary,
+            KeyCacheObject key,
+            MvccCoordinatorVersion mvccVer) throws IgniteCheckedException {
+            assert mvccVer != null;
+
+            if (!busyLock.enterBusy())
+                throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
+
+            try {
+
+
+                return null;
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+
+        /** {@inheritDoc} */
         @Override public GridLongList mvccUpdate(
             GridCacheContext cctx,
+            boolean primary,
             KeyCacheObject key,
             CacheObject val,
             GridCacheVersion ver,

http://git-wip-us.apache.org/repos/asf/ignite/blob/0bbbbe29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index cb01b6c..1f52309 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -1251,14 +1251,27 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         }
 
         /** {@inheritDoc} */
-        @Override public GridLongList mvccUpdate(GridCacheContext cctx,
+        @Override public GridLongList mvccUpdate(
+            GridCacheContext cctx,
+            boolean primary,
             KeyCacheObject key,
             CacheObject val,
             GridCacheVersion ver,
             MvccCoordinatorVersion mvccVer) throws IgniteCheckedException {
             CacheDataStore delegate = init0(false);
 
-            return delegate.mvccUpdate(cctx, key, val, ver, mvccVer);
+            return delegate.mvccUpdate(cctx, primary, key, val, ver, mvccVer);
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridLongList mvccRemove(
+            GridCacheContext cctx,
+            boolean primary,
+            KeyCacheObject key,
+            MvccCoordinatorVersion mvccVer) throws IgniteCheckedException {
+            CacheDataStore delegate = init0(false);
+
+            return delegate.mvccRemove(cctx, primary, key, mvccVer);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0bbbbe29/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java
index 79544e6..d3303e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/MvccUpdateRow.java
@@ -92,6 +92,29 @@ public class MvccUpdateRow extends DataRow implements BPlusTree.TreeRowClosure<C
         return cleanupRows;
     }
 
+    /**
+     * @param io IO.
+     * @param pageAddr Page address.
+     * @param idx Item index.
+     * @return Always {@code true}.
+     */
+    private boolean assertVersionGreater(RowLinkIO io, long pageAddr, int idx) {
+        long rowCrdVer = io.getMvccCoordinatorVersion(pageAddr, idx);
+        long rowCntr = io.getMvccCounter(pageAddr, idx);
+
+        int cmp = Long.compare(mvccCoordinatorVersion(), rowCrdVer);
+
+        if (cmp == 0)
+            cmp = Long.compare(mvccCounter(), rowCntr);
+
+        assert cmp > 0 : "[updCrd=" + mvccCoordinatorVersion() +
+            ", updCntr=" + mvccCounter() +
+            ", rowCrd=" + rowCrdVer +
+            ", rowCntr=" + rowCntr + ']';
+
+        return true;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean apply(BPlusTree<CacheSearchRow, CacheDataRow> tree,
         BPlusIO<CacheSearchRow> io,
@@ -101,9 +124,8 @@ public class MvccUpdateRow extends DataRow implements BPlusTree.TreeRowClosure<C
     {
         RowLinkIO rowIo = (RowLinkIO)io;
 
-        // All previous version should be less then new one.
-        assert mvccVer.coordinatorVersion() >= rowIo.getMvccCoordinatorVersion(pageAddr, idx);
-        assert mvccVer.coordinatorVersion() > rowIo.getMvccCoordinatorVersion(pageAddr, idx) || mvccVer.counter() > rowIo.getMvccCounter(pageAddr, idx);
+        // All previous versions should be less then new one.
+        assert assertVersionGreater(rowIo, pageAddr, idx);
 
         boolean checkActive = mvccVer.activeTransactions().size() > 0;
 


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

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


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

Branch: refs/heads/ignite-5937
Commit: 410c84fcf0d6f7bbe8373f996b6633c4d5b15b61
Parents: a1d9ddd e98e392
Author: sboikov <sb...@gridgain.com>
Authored: Thu Oct 5 11:40:04 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Oct 5 11:40:04 2017 +0300

----------------------------------------------------------------------
 assembly/release-fabric-base.xml                |   1 -
 .../apache/ignite/configuration/WALMode.java    |   9 +-
 .../ignite/internal/pagemem/wal/WALPointer.java |   4 +-
 .../internal/pagemem/wal/record/DataRecord.java |  20 +-
 .../pagemem/wal/record/SnapshotRecord.java      |  58 ++++++
 .../pagemem/wal/record/TimeStampRecord.java     |  57 ++++++
 .../internal/pagemem/wal/record/TxRecord.java   |  52 +++--
 .../internal/pagemem/wal/record/WALRecord.java  |   6 +-
 .../processors/cache/GridCacheAdapter.java      |  16 ++
 .../processors/cache/GridCacheMapEntry.java     |   2 +
 .../GridCacheDatabaseSharedManager.java         |   3 +
 .../cache/persistence/wal/FileWALPointer.java   |   3 +
 .../wal/FileWriteAheadLogManager.java           |   7 +
 .../reader/StandaloneWalRecordsIterator.java    |   2 +-
 .../wal/serializer/RecordDataV1Serializer.java  |   6 +-
 .../wal/serializer/RecordDataV2Serializer.java  |  49 ++++-
 .../wal/serializer/TxRecordSerializer.java      |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     |   3 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  45 +++-
 .../db/wal/IgniteWalSerializerVersionTest.java  | 205 ++++++++++++++++++-
 .../db/wal/reader/IgniteWalReaderTest.java      | 159 +++++++++++++-
 21 files changed, 665 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/410c84fc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------


[15/29] ignite git commit: IGNITE-6287 Web Console: Improved DDL support.

Posted by sb...@apache.org.
IGNITE-6287 Web Console: Improved DDL support.


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

Branch: refs/heads/ignite-5937
Commit: 2410f0792fec33725f1b7f74b5b576b353b8fe55
Parents: 49b8358
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Fri Oct 6 14:25:42 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Oct 6 14:25:42 2017 +0700

----------------------------------------------------------------------
 .../handlers/cache/GridCacheCommandHandler.java | 12 +++++--
 .../internal/visor/query/VisorQueryTask.java    | 36 ++++++++++++++++++--
 .../app/modules/agent/AgentManager.service.js   |  7 ++--
 .../frontend/app/modules/sql/sql.controller.js  | 36 ++++++++++++++------
 .../web-console/frontend/views/sql/sql.tpl.pug  | 14 ++++----
 5 files changed, 79 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2410f079/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index 53342c9..d627b20 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -56,6 +56,7 @@ import org.apache.ignite.internal.processors.cache.CacheInvokeEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlMetadata;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
@@ -1078,7 +1079,11 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         /** {@inheritDoc} */
         @Override public Collection<GridCacheSqlMetadata> execute() {
             String cacheName = null;
-            IgniteInternalCache<?, ?> cache;
+
+            if (!ignite.active())
+                return Collections.emptyList();
+
+            IgniteInternalCache<?, ?> cache = null;
 
             if (!F.isEmpty(arguments())) {
                 cacheName = argument(0);
@@ -1088,7 +1093,10 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                 assert cache != null;
             }
             else {
-                cache = F.first(ignite.context().cache().publicCaches()).internalProxy();
+                IgniteCacheProxy<?, ?> pubCache = F.first(ignite.context().cache().publicCaches());
+
+                if (pubCache != null)
+                    cache = pubCache.internalProxy();
 
                 if (cache == null)
                     return Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/ignite/blob/2410f079/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
index c85ceea..a3668c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
@@ -22,10 +22,15 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.UUID;
+import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.visor.VisorEither;
@@ -70,7 +75,6 @@ public class VisorQueryTask extends VisorOneNodeTask<VisorQueryTaskArg, VisorEit
         /** {@inheritDoc} */
         @Override protected VisorEither<VisorQueryResult> run(final VisorQueryTaskArg arg) {
             try {
-                IgniteCache<Object, Object> c = ignite.cache(arg.getCacheName());
                 UUID nid = ignite.localNode().id();
 
                 SqlFieldsQuery qry = new SqlFieldsQuery(arg.getQueryText());
@@ -83,7 +87,35 @@ public class VisorQueryTask extends VisorOneNodeTask<VisorQueryTaskArg, VisorEit
 
                 long start = U.currentTimeMillis();
 
-                VisorQueryCursor<List<?>> cur = new VisorQueryCursor<>(c.withKeepBinary().query(qry));
+                FieldsQueryCursor<List<?>> qryCursor;
+
+                String cacheName = arg.getCacheName();
+
+                if (F.isEmpty(cacheName))
+                    qryCursor = ignite.context().query().querySqlFieldsNoCache(qry, true);
+                else {
+                    IgniteCache<Object, Object> c = ignite.cache(cacheName);
+
+                    if (c == null)
+                        throw new SQLException("Fail to execute query. Cache not found: " + cacheName);
+
+                    try {
+                        qryCursor = c.withKeepBinary().query(qry);
+                    }
+                    catch (CacheException e) {
+                        // Work around for DDL without explicit schema name.
+                        if (X.hasCause(e, IgniteSQLException.class)
+                            && e.getMessage().contains("can only be executed on PUBLIC schema")) {
+                            qry.setSchema("PUBLIC");
+
+                            qryCursor = c.withKeepBinary().query(qry);
+                        }
+                        else
+                            throw e;
+                    }
+                }
+
+                VisorQueryCursor<List<?>> cur = new VisorQueryCursor<>(qryCursor);
 
                 Collection<GridQueryFieldMetadata> meta = cur.fieldsMeta();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/2410f079/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
index 20d2976..288ec94 100644
--- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
+++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
@@ -377,11 +377,10 @@ export default class IgniteAgentManager {
     }
 
     /**
-     * @param {String} [cacheName] Cache name.
      * @returns {Promise}
      */
-    metadata(cacheName) {
-        return this._rest('node:rest', {cmd: 'metadata', cacheName: maskNull(cacheName)})
+    metadata() {
+        return this._rest('node:rest', {cmd: 'metadata'})
             .then((caches) => {
                 let types = [];
 
@@ -590,7 +589,7 @@ export default class IgniteAgentManager {
                 nid + '=' + queryId);
         }
 
-        return this.visorTask('queryClose', nid, queryId);
+        return this.visorTask('queryClose', nid, nid, queryId);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/2410f079/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index 5f06c1e..8011b0f 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -863,9 +863,6 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                         return cachesAcc;
                     }, []), 'label');
 
-                    if (_.isEmpty($scope.caches))
-                        return;
-
                     // Reset to first cache in case of stopped selected.
                     const cacheNames = _.map($scope.caches, (cache) => cache.value);
 
@@ -1313,6 +1310,9 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
          * @return {String} Nid
          */
         const _chooseNode = (name, local) => {
+            if (_.isEmpty(name))
+                return Promise.resolve(null);
+
             const nodes = cacheNodes(name);
 
             if (local) {
@@ -1386,7 +1386,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             const enforceJoinOrder = !!paragraph.enforceJoinOrder;
             const lazy = !!paragraph.lazy;
 
-            $scope.actionAvailable(paragraph, true) && _chooseNode(paragraph.cacheName, local)
+            $scope.queryAvailable(paragraph) && _chooseNode(paragraph.cacheName, local)
                 .then((nid) => {
                     Notebook.save($scope.notebook)
                         .catch(Messages.showError);
@@ -1444,7 +1444,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         };
 
         $scope.explain = (paragraph) => {
-            if (!$scope.actionAvailable(paragraph, true))
+            if (!$scope.queryAvailable(paragraph))
                 return;
 
             Notebook.save($scope.notebook)
@@ -1483,7 +1483,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             const filter = paragraph.filter;
             const pageSize = paragraph.pageSize;
 
-            $scope.actionAvailable(paragraph, false) && _chooseNode(cacheName, local)
+            $scope.scanAvailable(paragraph) && _chooseNode(cacheName, local)
                 .then((nid) => {
                     Notebook.save($scope.notebook)
                         .catch(Messages.showError);
@@ -1689,18 +1689,32 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             _chartApplySettings(paragraph, true);
         };
 
-        $scope.actionAvailable = function(paragraph, needQuery) {
-            return $scope.caches.length > 0 && (!needQuery || paragraph.query) && !paragraph.loading;
+        $scope.queryAvailable = function(paragraph) {
+            return paragraph.query && !paragraph.loading;
+        };
+
+        $scope.queryTooltip = function(paragraph, action) {
+            if ($scope.queryAvailable(paragraph))
+                return;
+
+            if (paragraph.loading)
+                return 'Waiting for server response';
+
+            return 'Input text to ' + action;
+        };
+
+        $scope.scanAvailable = function(paragraph) {
+            return $scope.caches.length && !paragraph.loading;
         };
 
-        $scope.actionTooltip = function(paragraph, action, needQuery) {
-            if ($scope.actionAvailable(paragraph, needQuery))
+        $scope.scanTooltip = function(paragraph) {
+            if ($scope.scanAvailable(paragraph))
                 return;
 
             if (paragraph.loading)
                 return 'Waiting for server response';
 
-            return 'To ' + action + ' query select cache' + (needQuery ? ' and input query' : '');
+            return 'Select cache to export scan results';
         };
 
         $scope.clickableMetadata = function(node) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/2410f079/modules/web-console/frontend/views/sql/sql.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/sql/sql.tpl.pug b/modules/web-console/frontend/views/sql/sql.tpl.pug
index 1ef2a4c..724c53c 100644
--- a/modules/web-console/frontend/views/sql/sql.tpl.pug
+++ b/modules/web-console/frontend/views/sql/sql.tpl.pug
@@ -123,10 +123,10 @@ mixin query-settings
                     span Lazy result set
 
 mixin query-actions
-    button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute
-    button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph, true)') Execute on selected node
+    button.btn.btn-primary(ng-disabled='!queryAvailable(paragraph)' ng-click='execute(paragraph)') Execute
+    button.btn.btn-primary(ng-disabled='!queryAvailable(paragraph)' ng-click='execute(paragraph, true)') Execute on selected node
 
-    a.btn.btn-default(ng-disabled='!actionAvailable(paragraph, true)' ng-click='explain(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "explain", true)}}') Explain
+    a.btn.btn-default(ng-disabled='!queryAvailable(paragraph)' ng-click='explain(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{queryTooltip(paragraph, "explain query")}}') Explain
 
 mixin table-result-heading-query
     .total.row
@@ -142,7 +142,7 @@ mixin table-result-heading-query
         .col-xs-4
             .pull-right
                 -var options = [{ text: "Export", click: 'exportCsv(paragraph)' }, { text: 'Export all', click: 'exportCsvAll(paragraph)' }]
-                +btn-group('paragraph.loading', options, '{{ actionTooltip(paragraph, "export", false) }}')
+                +btn-group('paragraph.loading', options, '{{ queryTooltip(paragraph, "export query results") }}')
 
 mixin table-result-heading-scan
     .total.row
@@ -158,7 +158,7 @@ mixin table-result-heading-scan
         .col-xs-4
             .pull-right
                 -var options = [{ text: "Export", click: 'exportCsv(paragraph)' }, { text: 'Export all', click: 'exportCsvAll(paragraph)' }]
-                +btn-group('paragraph.loading', options, '{{ actionTooltip(paragraph, "export", false) }}')
+                +btn-group('paragraph.loading', options, '{{ scanTooltip(paragraph) }}')
 
 mixin table-result-body
     .grid(ui-grid='paragraph.gridOptions' ui-grid-resize-columns ui-grid-exporter)
@@ -196,9 +196,9 @@ mixin paragraph-scan
                 button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes')
 
         .col-sm-12.sql-controls
-            button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)')
+            button.btn.btn-primary(ng-disabled='!scanAvailable(paragraph)' ng-click='scan(paragraph)')
                 | Scan
-            button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph, true)')
+            button.btn.btn-primary(ng-disabled='!scanAvailable(paragraph)' ng-click='scan(paragraph, true)')
                 | Scan on selected node
 
         .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')


[17/29] ignite git commit: .NET: Add PlatformConfigurationUtils modification warning

Posted by sb...@apache.org.
.NET: Add PlatformConfigurationUtils modification warning


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

Branch: refs/heads/ignite-5937
Commit: a38fdec7212e0e21dc3ff8e6673fb56e2e00e1d1
Parents: e5c25b4
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Oct 6 12:54:06 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Oct 6 12:54:06 2017 +0300

----------------------------------------------------------------------
 .../processors/platform/utils/PlatformConfigurationUtils.java     | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a38fdec7/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 66160fb..24f4438 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -98,6 +98,9 @@ import org.apache.ignite.transactions.TransactionIsolation;
 
 /**
  * Configuration utils.
+ *
+ * WARNING: DO NOT MODIFY THIS FILE without updating corresponding platform code!
+ * Each read/write method has a counterpart in .NET platform (see IgniteConfiguration.cs, CacheConfiguration.cs, etc).
  */
 @SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
 public class PlatformConfigurationUtils {


[18/29] ignite git commit: IGNITE-6570 Web Console: Move parsing of JSON to pool of workers.

Posted by sb...@apache.org.
IGNITE-6570 Web Console: Move parsing of JSON to pool of workers.


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

Branch: refs/heads/ignite-5937
Commit: 74f04001a985211c499ee4bbd73de686288684a8
Parents: a38fdec
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Oct 6 17:00:39 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Oct 6 17:00:39 2017 +0700

----------------------------------------------------------------------
 modules/web-console/backend/app/agentSocket.js  |  21 +---
 .../web-console/backend/app/browsersHandler.js  |   9 +-
 .../app/modules/agent/AgentManager.service.js   |  18 ++-
 .../app/modules/agent/decompress.worker.js      |  33 +++++
 .../frontend/app/utils/SimpleWorkerPool.js      | 119 +++++++++++++++++++
 modules/web-console/frontend/package.json       |   1 +
 6 files changed, 176 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/74f04001/modules/web-console/backend/app/agentSocket.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/agentSocket.js b/modules/web-console/backend/app/agentSocket.js
index 489d145..75dcd53 100644
--- a/modules/web-console/backend/app/agentSocket.js
+++ b/modules/web-console/backend/app/agentSocket.js
@@ -24,7 +24,7 @@
  */
 module.exports = {
     implements: 'agent-socket',
-    inject: ['require(lodash)', 'require(zlib)']
+    inject: ['require(lodash)']
 };
 
 /**
@@ -79,10 +79,9 @@ class Command {
 
 /**
  * @param _
- * @param zlib
  * @returns {AgentSocket}
  */
-module.exports.factory = function(_, zlib) {
+module.exports.factory = function(_) {
     /**
      * Connected agent descriptor.
      */
@@ -136,21 +135,7 @@ module.exports.factory = function(_, zlib) {
                     if (resErr)
                         return reject(resErr);
 
-                    if (res.zipped) {
-                        // TODO IGNITE-6127 Temporary solution until GZip support for socket.io-client-java.
-                        // See: https://github.com/socketio/socket.io-client-java/issues/312
-                        // We can GZip manually for now.
-                        zlib.gunzip(new Buffer(res.data, 'base64'), (unzipErr, unzipped) => {
-                            if (unzipErr)
-                                return reject(unzipErr);
-
-                            res.data = unzipped.toString();
-
-                            resolve(res);
-                        });
-                    }
-                    else
-                        resolve(res);
+                    resolve(res);
                 })
             );
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f04001/modules/web-console/backend/app/browsersHandler.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/browsersHandler.js b/modules/web-console/backend/app/browsersHandler.js
index 4fb5088..f4ff23c 100644
--- a/modules/web-console/backend/app/browsersHandler.js
+++ b/modules/web-console/backend/app/browsersHandler.js
@@ -181,8 +181,12 @@ module.exports.factory = (_, socketio, configure, errors, mongo) => {
             return agent
                 .then((agentSock) => agentSock.emitEvent('node:rest', {uri: 'ignite', demo, params}))
                 .then((res) => {
-                    if (res.status === 0)
+                    if (res.status === 0) {
+                        if (res.zipped)
+                            return res;
+
                         return JSON.parse(res.data);
+                    }
 
                     throw new Error(res.error);
                 });
@@ -250,6 +254,9 @@ module.exports.factory = (_, socketio, configure, errors, mongo) => {
 
                 this.executeOnNode(agent, demo, params)
                     .then((data) => {
+                        if (data.zipped)
+                            return cb(null, data);
+
                         if (data.finished)
                             return cb(null, data.result);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f04001/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
index 288ec94..752b4f0 100644
--- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
+++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
@@ -17,6 +17,8 @@
 
 import { BehaviorSubject } from 'rxjs/BehaviorSubject';
 
+import Worker from 'worker!./decompress.worker';
+import SimpleWorkerPool from '../../utils/SimpleWorkerPool';
 import maskNull from 'app/core/utils/maskNull';
 
 const State = {
@@ -82,11 +84,9 @@ export default class IgniteAgentManager {
 
         this.promises = new Set();
 
-        /**
-         * Connection to backend.
-         * @type {Socket}
-         */
-        this.socket = null;
+        this.pool = new SimpleWorkerPool('decompressor', Worker, 4);
+
+        this.socket = null; // Connection to backend.
 
         let cluster;
 
@@ -364,7 +364,13 @@ export default class IgniteAgentManager {
      * @private
      */
     _rest(event, ...args) {
-        return this._emit(event, _.get(this.connectionSbj.getValue(), 'cluster.id'), ...args);
+        return this._emit(event, _.get(this.connectionSbj.getValue(), 'cluster.id'), ...args)
+            .then((data) => {
+                if (data.zipped)
+                    return this.pool.postMessage(data.data);
+
+                return data;
+            });
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f04001/modules/web-console/frontend/app/modules/agent/decompress.worker.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/decompress.worker.js b/modules/web-console/frontend/app/modules/agent/decompress.worker.js
new file mode 100644
index 0000000..d8e176d
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/agent/decompress.worker.js
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import _ from 'lodash';
+import pako from 'pako';
+
+/** This worker decode & decompress BASE64/Zipped data and parse to JSON. */
+// eslint-disable-next-line no-undef
+onmessage = function(e) {
+    const data = e.data;
+
+    const binaryString = atob(data); // Decode from BASE64
+
+    const unzipped = pako.inflate(binaryString, {to: 'string'});
+
+    const res = JSON.parse(unzipped);
+
+    postMessage(_.get(res, 'result', res));
+};

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f04001/modules/web-console/frontend/app/utils/SimpleWorkerPool.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/utils/SimpleWorkerPool.js b/modules/web-console/frontend/app/utils/SimpleWorkerPool.js
new file mode 100644
index 0000000..d8ed28b
--- /dev/null
+++ b/modules/web-console/frontend/app/utils/SimpleWorkerPool.js
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import {Observable} from 'rxjs/Observable';
+import {Subject} from 'rxjs/Subject';
+import 'rxjs/add/observable/race';
+import 'rxjs/add/operator/filter';
+import 'rxjs/add/operator/pluck';
+import 'rxjs/add/operator/take';
+import 'rxjs/add/operator/toPromise';
+
+/**
+ * Simple implementation of workers pool.
+ */
+export default class SimpleWorkerPool {
+    constructor(name, WorkerClass, poolSize = (navigator.hardwareConcurrency || 4), dbg = false) {
+        this._name = name;
+        this._WorkerClass = WorkerClass;
+        this._tasks = [];
+        this._msgId = 0;
+        this.messages$ = new Subject();
+        this.errors$ = new Subject();
+        this.__dbg = dbg;
+
+        this._workers = _.range(poolSize).map(() => {
+            const worker = new this._WorkerClass();
+
+            worker.onmessage = (m) => {
+                this.messages$.next({tid: worker.tid, m});
+
+                worker.tid = null;
+
+                this._run();
+            };
+
+            worker.onerror = (e) => {
+                this.errors$.next({tid: worker.tid, e});
+
+                worker.tid = null;
+
+                this._run();
+            };
+
+            return worker;
+        });
+    }
+
+    _makeTaskID() {
+        return this._msgId++;
+    }
+
+    _getNextWorker() {
+        return this._workers.find((w) => !w.tid);
+    }
+
+    _getNextTask() {
+        return this._tasks.shift();
+    }
+
+    _run() {
+        const worker = this._getNextWorker();
+
+        if (!worker || !this._tasks.length)
+            return;
+
+        const task = this._getNextTask();
+
+        worker.tid = task.tid;
+
+        if (this.__dbg)
+            console.time(`Post message[pool=${this._name}]`);
+
+        worker.postMessage(task.data);
+
+        if (this.__dbg)
+            console.timeEnd('Post message');
+    }
+
+    terminate() {
+        this._workers.forEach((w) => w.terminate());
+
+        this.messages$.complete();
+        this.errors$.complete();
+
+        this._workers = null;
+    }
+
+    postMessage(data) {
+        const tid = this._makeTaskID();
+
+        this._tasks.push({tid, data});
+
+        if (this.__dbg)
+            console.log(`Pool: [name=${this._name}, queue=${this._tasks.length}]`);
+
+        this._run();
+
+        return Observable.race(
+            this.messages$.filter((e) => e.tid === tid).take(1).pluck('m', 'data'),
+            this.errors$.filter((e) => e.tid === tid).take(1).map((e) => {
+                throw e.e;
+            }))
+            .take(1).toPromise();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/74f04001/modules/web-console/frontend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json
index d828e17..2083640 100644
--- a/modules/web-console/frontend/package.json
+++ b/modules/web-console/frontend/package.json
@@ -80,6 +80,7 @@
     "lodash": "4.17.4",
     "node-sass": "4.5.3",
     "nvd3": "1.8.4",
+    "pako": "1.0.6",
     "progress-bar-webpack-plugin": "1.10.0",
     "pug-html-loader": "1.1.0",
     "pug-loader": "2.3.0",


[21/29] ignite git commit: IGNITE-6574 Remove pending requests in case STATUS_AUTH_FAILURE && credentials == null.

Posted by sb...@apache.org.
IGNITE-6574 Remove pending requests in case STATUS_AUTH_FAILURE && credentials == null.


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

Branch: refs/heads/ignite-5937
Commit: 85261a341936547693aaed2105dc504fda2a9bf7
Parents: 35589a7
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Sat Oct 7 01:10:08 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Sat Oct 7 01:10:08 2017 +0700

----------------------------------------------------------------------
 .../client/impl/connection/GridClientNioTcpConnection.java         | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/85261a34/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index f72a009..3bedd5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -575,6 +575,8 @@ public class GridClientNioTcpConnection extends GridClientConnection {
                             "(client has no credentials) [clientId=" + clientId +
                             ", srvAddr=" + serverAddress() + ", errMsg=" + resp.errorMessage() +']'));
 
+                        removePending(resp.requestId());
+
                         return;
                     }
 


[19/29] ignite git commit: IGNITE-6054: SQL: implemented "WRAP_KEY" and "WRAP_VALUE" modes for CREATE TABLE. This closes #2784.

Posted by sb...@apache.org.
IGNITE-6054: SQL: implemented "WRAP_KEY" and "WRAP_VALUE" modes for CREATE TABLE. This closes #2784.


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

Branch: refs/heads/ignite-5937
Commit: 79d94cf4bbc906449a4c6995801fa61daaf80faa
Parents: 74f0400
Author: Alexander Paschenko <al...@gmail.com>
Authored: Fri Oct 6 18:04:44 2017 +0300
Committer: devozerov <pp...@gmail.com>
Committed: Fri Oct 6 18:04:44 2017 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |   2 +-
 .../cache/query/IgniteQueryErrorCode.java       |   2 +-
 .../processors/query/GridQueryIndexing.java     |  22 +-
 .../processors/query/GridQueryProcessor.java    |   8 +-
 .../query/QueryTypeDescriptorImpl.java          |  42 ++-
 .../query/property/QueryBinaryProperty.java     |   1 -
 ...IgniteClientCacheInitializationFailTest.java |  18 +-
 .../query/h2/DmlStatementsProcessor.java        |  26 +-
 .../internal/processors/query/h2/H2Schema.java  |  17 +-
 .../internal/processors/query/h2/H2TypeKey.java |  64 ++++
 .../processors/query/h2/IgniteH2Indexing.java   |  33 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |  36 ++-
 .../query/h2/sql/GridSqlCreateTable.java        |  34 +++
 .../query/h2/sql/GridSqlQueryParser.java        |  58 +++-
 ...ynamicColumnsAbstractConcurrentSelfTest.java |  57 ++--
 ...umnsConcurrentAtomicPartitionedSelfTest.java |   2 +-
 ...lumnsConcurrentAtomicReplicatedSelfTest.java |   2 +-
 ...currentTransactionalPartitionedSelfTest.java |   2 +-
 ...ncurrentTransactionalReplicatedSelfTest.java |   5 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |  44 +++
 .../cache/index/H2DynamicTableSelfTest.java     | 301 +++++++++++++++++--
 .../query/IgniteSqlNotNullConstraintTest.java   |   2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  47 +--
 .../Cache/Query/CacheDmlQueriesTest.cs          |   4 +-
 24 files changed, 673 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index 6c20de0..4e1ae4d 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -139,7 +139,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
             Statement stmt = conn.createStatement();
 
             stmt.execute("CREATE TABLE TEST (ID INT, NAME VARCHAR(50), VAL VARCHAR(50), PRIMARY KEY (ID, NAME))");
-            stmt.execute("CREATE TABLE \"Quoted\" (\"Id\" INT primary key, \"Name\" VARCHAR(50))");
+            stmt.execute("CREATE TABLE \"Quoted\" (\"Id\" INT primary key, \"Name\" VARCHAR(50)) WITH WRAP_KEY");
             stmt.execute("CREATE INDEX \"MyTestIndex quoted\" on \"Quoted\" (\"Id\" DESC)");
             stmt.execute("CREATE INDEX IDX ON TEST (ID ASC)");
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
index 8e5af31..e0ff9a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/IgniteQueryErrorCode.java
@@ -91,7 +91,7 @@ public final class IgniteQueryErrorCode {
     /** Attempt to INSERT or MERGE {@code null} key. */
     public final static int NULL_KEY = 4003;
 
-    /** Attempt to INSERT or MERGE {@code null} value. */
+    /** Attempt to INSERT or MERGE {@code null} value, or to to set {@code null} to a {@code NOT NULL} column. */
     public final static int NULL_VALUE = 4004;
 
     /** {@link EntryProcessor} has thrown an exception during {@link IgniteCache#invokeAll}. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index b8445ca..93d541d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -66,13 +66,13 @@ public interface GridQueryIndexing {
      * Parses SQL query into two step query and executes it.
      *
      * @param schemaName Schema name.
+     * @param cacheName Cache name.
      * @param qry Query.
      * @param keepBinary Keep binary flag.
-     * @param mainCacheId Main cache ID.
-     * @return Cursor.
+     * @param mainCacheId Main cache ID.    @return Cursor.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, SqlQuery qry,
+    public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, String cacheName, SqlQuery qry,
         boolean keepBinary, int mainCacheId) throws IgniteCheckedException;
 
     /**
@@ -109,12 +109,12 @@ public interface GridQueryIndexing {
      * Executes regular query.
      *
      * @param schemaName Schema name.
-     * @param qry Query.
+     * @param cacheName Cache name.
+     *@param qry Query.
      * @param filter Cache name and key filter.
-     * @param keepBinary Keep binary flag.
-     * @return Cursor.
+     * @param keepBinary Keep binary flag.    @return Cursor.
      */
-    public <K, V> QueryCursor<Cache.Entry<K,V>> queryLocalSql(String schemaName, SqlQuery qry,
+    public <K, V> QueryCursor<Cache.Entry<K,V>> queryLocalSql(String schemaName, String cacheName, SqlQuery qry,
         IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException;
 
     /**
@@ -134,14 +134,14 @@ public interface GridQueryIndexing {
      * Executes text query.
      *
      * @param schemaName Schema name.
+     * @param cacheName Cache name.
      * @param qry Text query.
      * @param typeName Type name.
-     * @param filter Cache name and key filter.
-     * @return Queried rows.
+     * @param filter Cache name and key filter.    @return Queried rows.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String schemaName, String qry,
-        String typeName, IndexingQueryFilter filter) throws IgniteCheckedException;
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String schemaName, String cacheName,
+        String qry, String typeName, IndexingQueryFilter filter) throws IgniteCheckedException;
 
     /**
      * Create new index locally.

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/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 56e8a42..f044c1d 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
@@ -2044,7 +2044,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return executeQuery(GridCacheQueryType.SQL, qry.getSql(), cctx,
                 new IgniteOutClosureX<QueryCursor<Cache.Entry<K, V>>>() {
                     @Override public QueryCursor<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
-                        return idx.queryDistributedSql(schemaName, qry, keepBinary, mainCacheId);
+                        return idx.queryDistributedSql(schemaName, cctx.name(), qry, keepBinary, mainCacheId);
                     }
                 }, true);
         }
@@ -2088,10 +2088,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         if (cctx.config().getQueryParallelism() > 1) {
                             qry.setDistributedJoins(true);
 
-                            return idx.queryDistributedSql(schemaName, qry, keepBinary, mainCacheId);
+                            return idx.queryDistributedSql(schemaName, cctx.name(), qry, keepBinary, mainCacheId);
                         }
                         else
-                            return idx.queryLocalSql(schemaName, qry, idx.backupFilter(requestTopVer.get(),
+                            return idx.queryLocalSql(schemaName, cctx.name(), qry, idx.backupFilter(requestTopVer.get(),
                                 qry.getPartitions()), keepBinary);
                     }
                 }, true);
@@ -2344,7 +2344,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         String typeName = typeName(cacheName, resType);
                         String schemaName = idx.schema(cacheName);
 
-                        return idx.queryLocalText(schemaName, clause, typeName, filters);
+                        return idx.queryLocalText(schemaName, cacheName, clause, typeName, filters);
                     }
                 }, true);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index e12476a..72adefd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -24,12 +24,12 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
@@ -453,7 +453,7 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
      * Sets key field name.
      * @param keyFieldName Key field name.
      */
-    public void keyFieldName(String keyFieldName) {
+    void keyFieldName(String keyFieldName) {
         this.keyFieldName = keyFieldName;
     }
 
@@ -464,10 +464,10 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
 
     /**
      * Sets value field name.
-     * @param valueFieldName value field name.
+     * @param valFieldName value field name.
      */
-    public void valueFieldName(String valueFieldName) {
-        this.valFieldName = valueFieldName;
+    void valueFieldName(String valFieldName) {
+        this.valFieldName = valFieldName;
     }
 
     /** {@inheritDoc} */
@@ -480,6 +480,7 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
         return keyFieldName != null ? aliases.get(keyFieldName) : null;
     }
 
+    /** {@inheritDoc} */
     @Nullable @Override public String valueFieldAlias() {
         return valFieldName != null ? aliases.get(valFieldName) : null;
     }
@@ -487,17 +488,34 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
     /** {@inheritDoc} */
     @SuppressWarnings("ForLoopReplaceableByForEach")
     @Override public void validateKeyAndValue(Object key, Object val) throws IgniteCheckedException {
-        if (validateProps == null)
+        if (F.isEmpty(validateProps))
             return;
 
-        final int size = validateProps.size();
+        for (int i = 0; i < validateProps.size(); ++i) {
+            GridQueryProperty prop = validateProps.get(i);
+
+            Object propVal;
+
+            int errCode;
+
+            if (F.eq(prop.name(), keyFieldName)) {
+                propVal = key;
+
+                errCode = IgniteQueryErrorCode.NULL_KEY;
+            }
+            else if (F.eq(prop.name(), valFieldName)) {
+                propVal = val;
+
+                errCode = IgniteQueryErrorCode.NULL_VALUE;
+            }
+            else {
+                propVal = prop.value(key, val);
 
-        for (int idx = 0; idx < size; ++idx) {
-            GridQueryProperty prop = validateProps.get(idx);
+                errCode = IgniteQueryErrorCode.NULL_VALUE;
+            }
 
-            if (prop.value(key, val) == null)
-                throw new IgniteSQLException("Null value is not allowed for field '" + prop.name() + "'",
-                    IgniteQueryErrorCode.NULL_VALUE);
+            if (propVal == null)
+                throw new IgniteSQLException("Null value is not allowed for column '" + prop.name() + "'", errCode);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
index 5d90a43..18508a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
@@ -81,7 +81,6 @@ public class QueryBinaryProperty implements GridQueryProperty {
      */
     public QueryBinaryProperty(GridKernalContext ctx, String propName, QueryBinaryProperty parent,
         Class<?> type, @Nullable Boolean key, String alias, boolean notNull) {
-
         this.ctx = ctx;
 
         log = ctx.log(QueryBinaryProperty.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
index 1ebf556..83dd9c9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
@@ -193,7 +193,7 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
     /**
      * @param client Client.
      */
-    @SuppressWarnings("ThrowableNotThrown")
+    @SuppressWarnings({"ThrowableNotThrown", "ThrowableResultOfMethodCallIgnored"})
     private void checkFailedCache(final Ignite client, final String cacheName) {
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -236,14 +236,14 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, SqlQuery qry,
-            boolean keepBinary, int mainCacheId) throws IgniteCheckedException {
+        @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, String cacheName,
+            SqlQuery qry, boolean keepBinary, int mainCacheId) throws IgniteCheckedException {
             return null;
         }
 
         /** {@inheritDoc} */
-        @Override public List<FieldsQueryCursor<List<?>>> queryDistributedSqlFields(String schemaName, SqlFieldsQuery qry,
-            boolean keepBinary, GridQueryCancel cancel,
+        @Override public List<FieldsQueryCursor<List<?>>> queryDistributedSqlFields(String schemaName,
+            SqlFieldsQuery qry, boolean keepBinary, GridQueryCancel cancel,
             @Nullable Integer mainCacheId, boolean failOnMultipleStmts) throws IgniteCheckedException {
             return null;
         }
@@ -255,8 +255,8 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(String schemaName, SqlQuery qry,
-            IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException {
+        @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryLocalSql(String schemaName, String cacheName,
+            SqlQuery qry, IndexingQueryFilter filter, boolean keepBinary) throws IgniteCheckedException {
             return null;
         }
 
@@ -267,8 +267,8 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String spaceName, String qry,
-            String typeName, IndexingQueryFilter filter) throws IgniteCheckedException {
+        @Override public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String spaceName,
+            String cacheName, String qry, String typeName, IndexingQueryFilter filter) throws IgniteCheckedException {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index ee1875f..98117b2 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -49,13 +49,13 @@ import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.odbc.SqlStateCode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheOperationContext;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.odbc.SqlStateCode;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
@@ -255,13 +255,12 @@ public class DmlStatementsProcessor {
     }
 
     /**
-     * Perform given statement against given data streamer. Only rows based INSERT and MERGE are supported
-     * as well as key bound UPDATE and DELETE (ones with filter {@code WHERE _key = ?}).
+     * Perform given statement against given data streamer. Only rows based INSERT is supported.
      *
      * @param streamer Streamer to feed data to.
      * @param stmt Statement.
      * @param args Statement arguments.
-     * @return Number of rows in given statement for INSERT and MERGE, {@code 1} otherwise.
+     * @return Number of rows in given INSERT statement.
      * @throws IgniteCheckedException if failed.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
@@ -916,11 +915,22 @@ public class DmlStatementsProcessor {
             val = convert(val, rowDesc, desc.valueClass(), plan.colTypes[plan.valColIdx]);
         }
 
-        if (key == null)
-            throw new IgniteSQLException("Key for INSERT or MERGE must not be null",  IgniteQueryErrorCode.NULL_KEY);
+        if (key == null) {
+            if (F.isEmpty(desc.keyFieldName()))
+                throw new IgniteSQLException("Key for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_KEY);
+            else
+                throw new IgniteSQLException("Null value is not allowed for column '" + desc.keyFieldName() + "'",
+                    IgniteQueryErrorCode.NULL_KEY);
+        }
 
-        if (val == null)
-            throw new IgniteSQLException("Value for INSERT or MERGE must not be null", IgniteQueryErrorCode.NULL_VALUE);
+        if (val == null) {
+            if (F.isEmpty(desc.valueFieldName()))
+                throw new IgniteSQLException("Value for INSERT, MERGE, or UPDATE must not be null",
+                    IgniteQueryErrorCode.NULL_VALUE);
+            else
+                throw new IgniteSQLException("Null value is not allowed for column '" + desc.valueFieldName() + "'",
+                    IgniteQueryErrorCode.NULL_VALUE);
+        }
 
         Map<String, Object> newColVals = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java
index 3f39e6a..f5cf0f2 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java
@@ -17,9 +17,8 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
-import org.jsr166.ConcurrentHashMap8;
-
 import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 /**
@@ -30,10 +29,10 @@ public class H2Schema {
     private final String schemaName;
 
     /** */
-    private final ConcurrentMap<String, H2TableDescriptor> tbls = new ConcurrentHashMap8<>();
+    private final ConcurrentMap<String, H2TableDescriptor> tbls = new ConcurrentHashMap<>();
 
     /** */
-    private final ConcurrentMap<String, H2TableDescriptor> typeToTbl = new ConcurrentHashMap8<>();
+    private final ConcurrentMap<H2TypeKey, H2TableDescriptor> typeToTbl = new ConcurrentHashMap<>();
 
     /**
      * Constructor.
@@ -70,8 +69,8 @@ public class H2Schema {
      * @param typeName Type name.
      * @return Table.
      */
-    public H2TableDescriptor tableByTypeName(String typeName) {
-        return typeToTbl.get(typeName);
+    public H2TableDescriptor tableByTypeName(String cacheName, String typeName) {
+        return typeToTbl.get(new H2TypeKey(cacheName, typeName));
     }
 
     /**
@@ -81,7 +80,7 @@ public class H2Schema {
         if (tbls.putIfAbsent(tbl.tableName(), tbl) != null)
             throw new IllegalStateException("Table already registered: " + tbl.fullTableName());
 
-        if (typeToTbl.putIfAbsent(tbl.typeName(), tbl) != null)
+        if (typeToTbl.putIfAbsent(new H2TypeKey(tbl.cache().name(), tbl.typeName()), tbl) != null)
             throw new IllegalStateException("Table already registered: " + tbl.fullTableName());
     }
 
@@ -91,7 +90,7 @@ public class H2Schema {
     public void remove(H2TableDescriptor tbl) {
         tbls.remove(tbl.tableName());
 
-        typeToTbl.remove(tbl.typeName());
+        typeToTbl.remove(new H2TypeKey(tbl.cache().name(), tbl.typeName()));
     }
 
     /**
@@ -104,7 +103,7 @@ public class H2Schema {
 
         tbls.remove(tbl.tableName());
 
-        typeToTbl.remove(tbl.typeName());
+        typeToTbl.remove(new H2TypeKey(tbl.cache().name(), tbl.typeName()));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TypeKey.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TypeKey.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TypeKey.java
new file mode 100644
index 0000000..d39918c
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TypeKey.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.h2;
+
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Key for types lookup.
+ */
+public class H2TypeKey {
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Type name. */
+    private final String typeName;
+
+    /**
+     * Constructor.
+     *
+     * @param cacheName Cache name.
+     * @param typeName Type name.
+     */
+    H2TypeKey(String cacheName, String typeName) {
+        this.cacheName = cacheName;
+        this.typeName = typeName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        H2TypeKey other = (H2TypeKey)o;
+
+        return F.eq(typeName, other.typeName) && F.eq(cacheName, other.cacheName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = cacheName.hashCode();
+
+        res = 31 * res + typeName.hashCode();
+
+        return res;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index fd7b9a8..22ed592 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -548,7 +548,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         GridCacheVersion ver,
         long expirationTime,
         long link) throws IgniteCheckedException {
-        H2TableDescriptor tbl = tableDescriptor(schema(cacheName), type.name());
+        H2TableDescriptor tbl = tableDescriptor(schema(cacheName), cacheName, type.name());
 
         if (tbl == null)
             return; // Type was rejected.
@@ -572,7 +572,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Removing key from cache query index [locId=" + nodeId + ", key=" + key + ", val=" + val + ']');
 
-        H2TableDescriptor tbl = tableDescriptor(schema(cacheName), type.name());
+        H2TableDescriptor tbl = tableDescriptor(schema(cacheName), cacheName, type.name());
 
         if (tbl == null)
             return;
@@ -777,10 +777,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
     }
 
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String schemaName, String qry,
-        String typeName, IndexingQueryFilter filters) throws IgniteCheckedException {
-        H2TableDescriptor tbl = tableDescriptor(schemaName, typeName);
+    @Override public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String schemaName,
+        String cacheName, String qry, String typeName, IndexingQueryFilter filters) throws IgniteCheckedException {
+        H2TableDescriptor tbl = tableDescriptor(schemaName, cacheName, typeName);
 
         if (tbl != null && tbl.luceneIndex() != null) {
             GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, schemaName,
@@ -1098,7 +1099,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <K, V> QueryCursor<Cache.Entry<K,V>> queryLocalSql(String schemaName,
+    @Override public <K, V> QueryCursor<Cache.Entry<K,V>> queryLocalSql(String schemaName, String cacheName,
         final SqlQuery qry, final IndexingQueryFilter filter, final boolean keepBinary) throws IgniteCheckedException {
         String type = qry.getType();
         String sqlQry = qry.getSql();
@@ -1107,7 +1108,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         GridQueryCancel cancel = new GridQueryCancel();
 
-        final GridCloseableIterator<IgniteBiTuple<K, V>> i = queryLocalSql(schemaName, sqlQry, alias,
+        final GridCloseableIterator<IgniteBiTuple<K, V>> i = queryLocalSql(schemaName, cacheName, sqlQry, alias,
             F.asList(params), type, filter, cancel);
 
         return new QueryCursorImpl<>(new Iterable<Cache.Entry<K, V>>() {
@@ -1142,19 +1143,19 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * Executes regular query.
      *
      * @param schemaName Schema name.
+     * @param cacheName Cache name.
      * @param qry Query.
      * @param alias Table alias.
      * @param params Query parameters.
      * @param type Query return type.
-     * @param filter Cache name and key filter.
-     * @return Queried rows.
+     * @param filter Cache name and key filter.      @return Queried rows.
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalSql(String schemaName,
+    public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalSql(String schemaName, String cacheName,
         final String qry, String alias, @Nullable final Collection<Object> params, String type,
         final IndexingQueryFilter filter, GridQueryCancel cancel) throws IgniteCheckedException {
-        final H2TableDescriptor tbl = tableDescriptor(schemaName, type);
+        final H2TableDescriptor tbl = tableDescriptor(schemaName, cacheName, type);
 
         if (tbl == null)
             throw new IgniteSQLException("Failed to find SQL table for type: " + type,
@@ -1216,11 +1217,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, SqlQuery qry,
-        boolean keepBinary, int mainCacheId) {
+    @Override public <K, V> QueryCursor<Cache.Entry<K, V>> queryDistributedSql(String schemaName, String cacheName,
+        SqlQuery qry, boolean keepBinary, int mainCacheId) {
         String type = qry.getType();
 
-        H2TableDescriptor tblDesc = tableDescriptor(schemaName, type);
+        H2TableDescriptor tblDesc = tableDescriptor(schemaName, cacheName, type);
 
         if (tblDesc == null)
             throw new IgniteSQLException("Failed to find SQL table for type: " + type,
@@ -1829,13 +1830,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param type Type name.
      * @return Descriptor.
      */
-    @Nullable private H2TableDescriptor tableDescriptor(String schemaName, String type) {
+    @Nullable private H2TableDescriptor tableDescriptor(String schemaName, String cacheName, String type) {
         H2Schema schema = schemas.get(schemaName);
 
         if (schema == null)
             return null;
 
-        return schema.tableByTypeName(type);
+        return schema.tableByTypeName(cacheName, type);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/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 affd903..f39e587 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
@@ -62,6 +62,7 @@ import org.h2.table.Column;
 import org.h2.value.DataType;
 
 import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META;
+import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser.PARAM_WRAP_VALUE;
 
 /**
  * DDL statements processor.<p>
@@ -227,6 +228,10 @@ public class DdlStatementsProcessor {
                             cmd.tableName());
                 }
                 else {
+                    if (QueryUtils.isSqlType(tbl.rowDescriptor().type().valueClass()))
+                        throw new SchemaOperationException("Cannot add column(s) because table was created " +
+                            "with " + PARAM_WRAP_VALUE + "=false option.");
+
                     List<QueryField> cols = new ArrayList<>(cmd.columns().length);
 
                     boolean allFieldsNullable = true;
@@ -373,11 +378,38 @@ public class DdlStatementsProcessor {
         if (!F.isEmpty(createTbl.valueTypeName()))
             valTypeName = createTbl.valueTypeName();
 
+        assert createTbl.wrapKey() != null;
+        assert createTbl.wrapValue() != null;
+
+        if (!createTbl.wrapKey()) {
+            GridSqlColumn pkCol = createTbl.columns().get(createTbl.primaryKeyColumns().iterator().next());
+
+            keyTypeName = DataType.getTypeClassName(pkCol.column().getType());
+
+            res.setKeyFieldName(pkCol.columnName());
+        }
+        else
+            res.setKeyFields(createTbl.primaryKeyColumns());
+
+        if (!createTbl.wrapValue()) {
+            GridSqlColumn valCol = null;
+
+            for (Map.Entry<String, GridSqlColumn> e : createTbl.columns().entrySet()) {
+                if (!createTbl.primaryKeyColumns().contains(e.getKey())) {
+                    valCol = e.getValue();
+
+                    break;
+                }
+            }
+
+            valTypeName = DataType.getTypeClassName(valCol.column().getType());
+
+            res.setValueFieldName(valCol.columnName());
+        }
+
         res.setValueType(valTypeName);
         res.setKeyType(keyTypeName);
 
-        res.setKeyFields(createTbl.primaryKeyColumns());
-
         if (!F.isEmpty(notNullFields)) {
             QueryEntityEx res0 = new QueryEntityEx(res);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
index b73214f..de10826 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
@@ -71,6 +71,12 @@ public class GridSqlCreateTable extends GridSqlStatement {
     /** Name of the column that represents affinity key. */
     private String affinityKey;
 
+    /** Forcefully turn single column PK into an Object. */
+    private Boolean wrapKey;
+
+    /** Forcefully turn single column value into an Object. */
+    private Boolean wrapVal;
+
     /** Extra WITH-params. */
     private List<String> params;
 
@@ -271,6 +277,34 @@ public class GridSqlCreateTable extends GridSqlStatement {
     }
 
     /**
+     * @return Forcefully turn single column PK into an Object.
+     */
+    public Boolean wrapKey() {
+        return wrapKey;
+    }
+
+    /**
+     * @param wrapKey Forcefully turn single column PK into an Object.
+     */
+    public void wrapKey(boolean wrapKey) {
+        this.wrapKey = wrapKey;
+    }
+
+    /**
+     * @return Forcefully turn single column value into an Object.
+     */
+    public Boolean wrapValue() {
+        return wrapVal;
+    }
+
+    /**
+     * @param wrapVal Forcefully turn single column value into an Object..
+     */
+    public void wrapValue(boolean wrapVal) {
+        this.wrapVal = wrapVal;
+    }
+
+    /**
      * @return Extra WITH-params.
      */
     public List<String> params() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index 3d7a1a0..bf72200 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -475,6 +475,12 @@ public class GridSqlQueryParser {
     private static final String PARAM_VAL_TYPE = "VALUE_TYPE";
 
     /** */
+    private static final String PARAM_WRAP_KEY = "WRAP_KEY";
+
+    /** */
+    public static final String PARAM_WRAP_VALUE = "WRAP_VALUE";
+
+    /** */
     private final IdentityHashMap<Object, Object> h2ObjToGridObj = new IdentityHashMap<>();
 
     /** */
@@ -1007,7 +1013,8 @@ public class GridSqlQueryParser {
             pkCols.add(gridCol.columnName());
         }
 
-        int valColsNum = cols.size() - pkCols.size();
+        int keyColsNum = pkCols.size();
+        int valColsNum = cols.size() - keyColsNum;
 
         if (valColsNum == 0)
             throw new IgniteSQLException("Table must have at least one non PRIMARY KEY column.",
@@ -1052,6 +1059,44 @@ public class GridSqlQueryParser {
                 processExtraParam(e.getKey(), e.getValue(), res);
         }
 
+        // Process key wrapping.
+        Boolean wrapKey = res.wrapKey();
+
+        if (wrapKey != null && !wrapKey) {
+            if (keyColsNum > 1) {
+                throw new IgniteSQLException(PARAM_WRAP_KEY + " cannot be false when composite primary key exists.",
+                    IgniteQueryErrorCode.PARSING);
+            }
+
+            if (!F.isEmpty(res.keyTypeName())) {
+                throw new IgniteSQLException(PARAM_WRAP_KEY + " cannot be false when " + PARAM_KEY_TYPE + " is set.",
+                    IgniteQueryErrorCode.PARSING);
+            }
+        }
+
+        boolean wrapKey0 = (res.wrapKey() != null && res.wrapKey()) || !F.isEmpty(res.keyTypeName()) || keyColsNum > 1;
+
+        res.wrapKey(wrapKey0);
+
+        // Process value wrapping.
+        Boolean wrapVal = res.wrapValue();
+
+        if (wrapVal != null && !wrapVal) {
+            if (valColsNum > 1) {
+                throw new IgniteSQLException(PARAM_WRAP_VALUE + " cannot be false when multiple non-primary key " +
+                    "columns exist.", IgniteQueryErrorCode.PARSING);
+            }
+
+            if (!F.isEmpty(res.valueTypeName())) {
+                throw new IgniteSQLException(PARAM_WRAP_VALUE + " cannot be false when " + PARAM_VAL_TYPE + " is set.",
+                    IgniteQueryErrorCode.PARSING);
+            }
+
+            res.wrapValue(false);
+        }
+        else
+            res.wrapValue(true); // By default value is always wrapped to allow for ALTER TABLE ADD COLUMN commands.
+
         if (!F.isEmpty(res.valueTypeName()) && F.eq(res.keyTypeName(), res.valueTypeName()))
             throw new IgniteSQLException("Key and value type names " +
                 "should be different for CREATE TABLE: " + res.valueTypeName(), IgniteQueryErrorCode.PARSING);
@@ -1336,6 +1381,17 @@ public class GridSqlQueryParser {
 
                 break;
 
+            case PARAM_WRAP_KEY: {
+                res.wrapKey(F.isEmpty(val) || Boolean.parseBoolean(val));
+
+                break;
+            }
+
+            case PARAM_WRAP_VALUE:
+                res.wrapValue(F.isEmpty(val) || Boolean.parseBoolean(val));
+
+                break;
+
             default:
                 throw new IgniteSQLException("Unsupported parameter: " + name, IgniteQueryErrorCode.PARSING);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
index 969c985..3d4b2a3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
@@ -82,7 +82,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     private static final String ATTR_FILTERED = "FILTERED";
 
     /** SQL statement to create test table accompanied by template specification. */
-    private static final String CREATE_SQL_WITH_TEMPLATE = CREATE_SQL + " WITH \"template=TPL\"";
+    private final String createSql;
 
     /** Latches to block certain index operations. */
     private static final ConcurrentHashMap<UUID, T3<CountDownLatch, AtomicBoolean, CountDownLatch>> BLOCKS =
@@ -103,6 +103,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     DynamicColumnsAbstractConcurrentSelfTest(CacheMode cacheMode, CacheAtomicityMode atomicityMode) {
         this.cacheMode = cacheMode;
         this.atomicityMode = atomicityMode;
+        createSql =  CREATE_SQL + " WITH \"template=TPL\"";
     }
 
     /** {@inheritDoc} */
@@ -158,7 +159,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(cli);
 
-        run(cli, CREATE_SQL_WITH_TEMPLATE);
+        run(cli, createSql);
 
         // Test migration between normal servers.
         CountDownLatch idxLatch = blockIndexing(srv1Id);
@@ -210,7 +211,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(srv1);
 
-        run(srv1, CREATE_SQL_WITH_TEMPLATE);
+        run(srv1, createSql);
 
         CountDownLatch idxLatch = blockIndexing(srv1);
 
@@ -253,7 +254,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(srv1);
 
-        run(srv1, CREATE_SQL_WITH_TEMPLATE);
+        run(srv1, createSql);
 
         CountDownLatch idxLatch = blockIndexing(srv1);
 
@@ -296,7 +297,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(srv1);
 
-        run(srv1, CREATE_SQL_WITH_TEMPLATE);
+        run(srv1, createSql);
 
         // Start data change operations from several threads.
         final AtomicBoolean stopped = new AtomicBoolean();
@@ -309,7 +310,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
                     int key = ThreadLocalRandom.current().nextInt(0, LARGE_CACHE_SIZE);
                     int val = ThreadLocalRandom.current().nextInt();
 
-                    IgniteCache<BinaryObject, BinaryObject> cache = node.cache(CACHE_NAME);
+                    IgniteCache<Object, BinaryObject> cache = node.cache(CACHE_NAME);
 
                     if (ThreadLocalRandom.current().nextBoolean())
                         cache.put(key(node, key), val(node, val));
@@ -342,15 +343,15 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         // Get expected values.
         Set<Integer> expKeys = new HashSet<>();
 
-        IgniteCache<BinaryObject, BinaryObject> cache = srv1.cache(CACHE_NAME).withKeepBinary();
+        IgniteCache<Object, BinaryObject> cache = srv1.cache(CACHE_NAME).withKeepBinary();
 
         for (int i = 0; i < LARGE_CACHE_SIZE; i++) {
-            BinaryObject key = key(srv1, i);
+            Object key = key(srv1, i);
 
             BinaryObject val = cache.get(key);
 
             if (val != null) {
-                int id = key.field("ID");
+                int id = (Integer)key;
 
                 assertEquals(i, id);
 
@@ -363,17 +364,17 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         // Validate query result.
         for (Ignite node : Ignition.allGrids()) {
-            IgniteCache<BinaryObject, BinaryObject> nodeCache = node.cache(CACHE_NAME).withKeepBinary();
+            IgniteCache<Object, BinaryObject> nodeCache = node.cache(CACHE_NAME).withKeepBinary();
 
             SqlQuery qry = new SqlQuery(valTypeName, "from " + TBL_NAME + " where mod(id, 2) <> 0");
 
-            List<Cache.Entry<BinaryObject, BinaryObject>> res = nodeCache.query(qry).getAll();
+            List<Cache.Entry<Object, BinaryObject>> res = nodeCache.query(qry).getAll();
 
             assertEquals("Cache size mismatch [exp=" + expKeys.size() + ", actual=" + res.size() + ']',
                 expKeys.size(), res.size());
 
-            for (Cache.Entry<BinaryObject, BinaryObject> entry : res) {
-                int key = entry.getKey().field("ID");
+            for (Cache.Entry<Object, BinaryObject> entry : res) {
+                int key = (Integer)entry.getKey();
                 int v = entry.getValue().field("v");
 
                 String name = entry.getValue().field("NAME");
@@ -399,13 +400,11 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
     /**
      * @param node Node.
-     * @param key Value for ID field.
-     * @return PERSON cache key.
+     * @param id Key.
+     * @return PERSON cache key (int or {@link BinaryObject}).
      */
-    private BinaryObject key(Ignite node, int key) {
-        String keyTypeName = ((IgniteEx)node).context().query().types(CACHE_NAME).iterator().next().keyTypeName();
-
-        return node.binary().builder(keyTypeName).setField("ID", key).build();
+    private Object key(Ignite node, int id) {
+        return id;
     }
 
     /**
@@ -420,7 +419,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(srv1);
 
-        run(srv1, CREATE_SQL_WITH_TEMPLATE);
+        run(srv1, createSql);
 
         awaitPartitionMapExchange();
 
@@ -481,7 +480,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         // Start cache and populate it with data.
         createSqlCache(cli);
 
-        run(cli, CREATE_SQL_WITH_TEMPLATE);
+        run(cli, createSql);
 
         put(cli, 0, LARGE_CACHE_SIZE);
 
@@ -526,7 +525,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(cli);
 
-        run(cli, CREATE_SQL_WITH_TEMPLATE);
+        run(cli, createSql);
 
         put(cli, 0, 5000);
 
@@ -539,8 +538,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
                 while (!stopped.get()) {
                     Ignite node = grid(ThreadLocalRandom.current().nextInt(1, 5));
 
-                    IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + dynColCnt.getAndIncrement(),
-                        Integer.class.getName()));
+                    IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" +
+                        dynColCnt.getAndIncrement(), Integer.class.getName()));
 
                     try {
                         fut.get();
@@ -641,7 +640,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         if (dynamicCache) {
             createSqlCache(cli);
 
-            run(cli, CREATE_SQL_WITH_TEMPLATE);
+            run(cli, createSql);
         }
 
         final String schemaName = dynamicCache ? QueryUtils.DFLT_SCHEMA : "idx";
@@ -677,7 +676,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
                     if (dynamicCache) {
                         DynamicColumnsAbstractConcurrentSelfTest.this.run(srvNode, DROP_SQL);
 
-                        DynamicColumnsAbstractConcurrentSelfTest.this.run(srvNode, CREATE_SQL_WITH_TEMPLATE);
+                        DynamicColumnsAbstractConcurrentSelfTest.this.run(srvNode, createSql);
                     }
                     else {
                         srvNode.destroyCache("idx");
@@ -724,7 +723,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(cli);
 
-        run(cli, CREATE_SQL_WITH_TEMPLATE);
+        run(cli, createSql);
 
         final AtomicBoolean stopped = new AtomicBoolean();
 
@@ -783,8 +782,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
                 while (!stopped.get()) {
                     Ignite node = grid(ThreadLocalRandom.current().nextInt(1, 5));
 
-                    IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + dynColCnt.getAndIncrement(),
-                        Integer.class.getName()));
+                    IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" +
+                        dynColCnt.getAndIncrement(), Integer.class.getName()));
 
                     try {
                         fut.get();

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java
index ca68903..3f65fca 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicPartitionedSelfTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 
 /**
- * Test to check concurrent operations on dynamic columns on ATOMIC PARTITIONED cache.
+ * Test to check concurrent operations on dynamic columns on ATOMIC PARTITIONED cache with flat key.
  */
 public class DynamicColumnsConcurrentAtomicPartitionedSelfTest extends DynamicColumnsAbstractConcurrentSelfTest {
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java
index 9a3a32c..289a01e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentAtomicReplicatedSelfTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 
 /**
- * Test to check concurrent operations on dynamic columns on ATOMIC REPLICATED cache.
+ * Test to check concurrent operations on dynamic columns on ATOMIC REPLICATED cache with flat key.
  */
 public class DynamicColumnsConcurrentAtomicReplicatedSelfTest extends DynamicColumnsAbstractConcurrentSelfTest {
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java
index f42a447..371313e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalPartitionedSelfTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 
 /**
- * Test to check concurrent operations on dynamic columns on TRANSACTIONAL PARTITIONED cache.
+ * Test to check concurrent operations on dynamic columns on TRANSACTIONAL PARTITIONED cache with flat key.
  */
 public class DynamicColumnsConcurrentTransactionalPartitionedSelfTest extends DynamicColumnsAbstractConcurrentSelfTest {
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java
index 2b53e42..f004b89 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsConcurrentTransactionalReplicatedSelfTest.java
@@ -21,9 +21,10 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 
 /**
- * Test to check concurrent operations on dynamic columns on TRANSACTIONAL REPLICATED cache.
+ * Test to check concurrent operations on dynamic columns on TRANSACTIONAL REPLICATED cache with flat key.
  */
-public class DynamicColumnsConcurrentTransactionalReplicatedSelfTest extends DynamicColumnsAbstractConcurrentSelfTest {
+public class DynamicColumnsConcurrentTransactionalReplicatedSelfTest
+    extends DynamicColumnsAbstractConcurrentSelfTest {
     /**
      * Constructor.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
index 5e64959..34be34d 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
@@ -302,6 +302,50 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
     }
 
     /**
+     * Test that {@code ADD COLUMN} fails for non dynamic table that has flat value.
+     */
+    @SuppressWarnings({"unchecked", "ThrowFromFinallyBlock"})
+    public void testTestAlterTableOnFlatValueNonDynamicTable() {
+        CacheConfiguration c =
+            new CacheConfiguration("ints").setIndexedTypes(Integer.class, Integer.class)
+                .setSqlSchema(QueryUtils.DFLT_SCHEMA);
+
+        try {
+            grid(nodeIndex()).getOrCreateCache(c);
+
+            doTestAlterTableOnFlatValue("INTEGER");
+        }
+        finally {
+            grid(nodeIndex()).destroyCache("ints");
+        }
+    }
+
+    /**
+     * Test that {@code ADD COLUMN} fails for dynamic table that has flat value.
+     */
+    @SuppressWarnings({"unchecked", "ThrowFromFinallyBlock"})
+    public void testTestAlterTableOnFlatValueDynamicTable() {
+        try {
+            run("CREATE TABLE TEST (id int primary key, x varchar) with \"wrap_value=false\"");
+
+            doTestAlterTableOnFlatValue("TEST");
+        }
+        finally {
+            run("DROP TABLE TEST");
+        }
+    }
+
+    /**
+     * Test that {@code ADD COLUMN} fails for tables that have flat value.
+     * @param tblName table name.
+     */
+    private void doTestAlterTableOnFlatValue(String tblName) {
+        assertThrows("ALTER TABLE " + tblName + " ADD COLUMN y varchar",
+            "ADD COLUMN is not supported for tables created with wrap_value=false param. " +
+                "(To enable ADD COLUMN, create table with wrap_value=true param).");
+    }
+
+    /**
      * @return Node index to run queries on.
      */
     protected abstract int nodeIndex();

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
index 773e7e0..e0ab6c5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
@@ -17,6 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache.index;
 
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -59,6 +64,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.h2.jdbc.JdbcSQLException;
+import org.h2.value.DataType;
 
 /**
  * Tests for CREATE/DROP TABLE.
@@ -285,6 +291,14 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /**
+     * Test that {@code CREATE TABLE} with given write sync mode actually creates new cache as needed.
+     * @throws Exception if failed.
+     */
+    public void testPlainKey() throws Exception {
+        doTestCreateTable(null, null, null, CacheWriteSynchronizationMode.FULL_SYNC);
+    }
+
+    /**
      * Test that appending supplied arguments to {@code CREATE TABLE} results in creating new cache that has settings
      * as expected
      * @param cacheName Cache name, or {@code null} if the name generated by default should be used.
@@ -292,18 +306,19 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      * @param valTypeName Value type name, or {@code null} if the name generated by default should be used.
      */
     private void doTestCustomNames(String cacheName, String keyTypeName, String valTypeName) {
-        GridStringBuilder b = new GridStringBuilder("CREATE TABLE \"NameTest\" (id int primary key, x varchar) WITH ");
+        GridStringBuilder b = new GridStringBuilder("CREATE TABLE \"NameTest\" (id int primary key, x varchar) WITH " +
+            "wrap_key,wrap_value");
 
         assert !F.isEmpty(cacheName) || !F.isEmpty(keyTypeName) || !F.isEmpty(valTypeName);
 
         if (!F.isEmpty(cacheName))
-            b.a("\"cache_name=").a(cacheName).a('"').a(',');
+            b.a(",\"cache_name=").a(cacheName).a('"');
 
         if (!F.isEmpty(keyTypeName))
-            b.a("\"key_type=").a(keyTypeName).a('"').a(',');
+            b.a(",\"key_type=").a(keyTypeName).a('"');
 
         if (!F.isEmpty(valTypeName))
-            b.a("\"value_type=").a(valTypeName).a('"');
+            b.a(",\"value_type=").a(valTypeName).a('"');
 
         String res = b.toString();
 
@@ -312,8 +327,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
 
         execute(client(), res);
 
-        String resCacheName = U.firstNotNull(cacheName, QueryUtils.createTableCacheName(QueryUtils.DFLT_SCHEMA,
-            "NameTest"));
+        String resCacheName = U.firstNotNull(cacheName, cacheName("NameTest"));
 
         IgniteInternalCache<BinaryObject, BinaryObject> cache = client().cachex(resCacheName);
 
@@ -770,7 +784,8 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      * @throws Exception if failed.
      */
     public void testAffinityKey() throws Exception {
-        execute("CREATE TABLE \"City\" (\"name\" varchar primary key, \"code\" int) WITH \"affinityKey='name'\"");
+        execute("CREATE TABLE \"City\" (\"name\" varchar primary key, \"code\" int) WITH wrap_key,wrap_value," +
+            "\"affinityKey='name'\"");
 
         assertAffinityCacheConfiguration("City", "name");
 
@@ -783,7 +798,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
         // We need unique name for this table to avoid conflicts with existing binary metadata.
         execute("CREATE TABLE \"Person2\" (\"id\" int, \"city\" varchar," +
             " \"name\" varchar, \"surname\" varchar, \"age\" int, PRIMARY KEY (\"id\", \"city\")) WITH " +
-            "\"template=cache,affinityKey='city'\"");
+            "wrap_key,wrap_value,\"template=cache,affinityKey='city'\"");
 
         assertAffinityCacheConfiguration("Person2", "city");
 
@@ -828,26 +843,31 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
     public void testAffinityKeyCaseSensitivity() {
-        execute("CREATE TABLE \"A\" (\"name\" varchar primary key, \"code\" int) WITH \"affinityKey='name'\"");
+        execute("CREATE TABLE \"A\" (\"name\" varchar primary key, \"code\" int) WITH wrap_key,wrap_value," +
+            "\"affinityKey='name'\"");
 
         assertAffinityCacheConfiguration("A", "name");
 
-        execute("CREATE TABLE \"B\" (name varchar primary key, \"code\" int) WITH \"affinityKey=name\"");
+        execute("CREATE TABLE \"B\" (name varchar primary key, \"code\" int) WITH wrap_key,wrap_value," +
+            "\"affinityKey=name\"");
 
         assertAffinityCacheConfiguration("B", "NAME");
 
-        execute("CREATE TABLE \"C\" (name varchar primary key, \"code\" int) WITH \"affinityKey=NamE\"");
+        execute("CREATE TABLE \"C\" (name varchar primary key, \"code\" int) WITH wrap_key,wrap_value," +
+            "\"affinityKey=NamE\"");
 
         assertAffinityCacheConfiguration("C", "NAME");
 
-        execute("CREATE TABLE \"D\" (\"name\" varchar primary key, \"code\" int) WITH \"affinityKey=NAME\"");
+        execute("CREATE TABLE \"D\" (\"name\" varchar primary key, \"code\" int) WITH wrap_key,wrap_value," +
+            "\"affinityKey=NAME\"");
 
         assertAffinityCacheConfiguration("D", "name");
 
         // Error arises because user has specified case sensitive affinity column name
         GridTestUtils.assertThrows(null, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                execute("CREATE TABLE \"E\" (name varchar primary key, \"code\" int) WITH \"affinityKey='Name'\"");
+                execute("CREATE TABLE \"E\" (name varchar primary key, \"code\" int) WITH wrap_key,wrap_value," +
+                    "\"affinityKey='Name'\"");
 
                 return null;
             }
@@ -865,9 +885,19 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
         }, IgniteSQLException.class, "Ambiguous affinity column name, use single quotes for case sensitivity: name");
 
         execute("CREATE TABLE \"E\" (\"name\" varchar, \"Name\" int, val int, primary key(\"name\", " +
-            "\"Name\")) WITH \"affinityKey='Name'\"");
+            "\"Name\")) WITH wrap_key,wrap_value,\"affinityKey='Name'\"");
 
         assertAffinityCacheConfiguration("E", "Name");
+
+        execute("drop table a");
+
+        execute("drop table b");
+
+        execute("drop table c");
+
+        execute("drop table d");
+
+        execute("drop table e");
     }
 
     /**
@@ -914,7 +944,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
         execute("create index on \"PUBLIC\".t (b desc)");
         execute("drop table \"PUBLIC\".t");
 
-        assertNull(client().cache("t"));
+        assertNull(client().cache(cacheName("t")));
 
         execute("create table \"PUBLIC\".t (a int primary key, b varchar(30))");
 
@@ -944,6 +974,224 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /**
+     * Test that it's impossible to create tables with same name regardless of key/value wrapping settings.
+     */
+    public void testWrappedAndUnwrappedKeyTablesInteroperability() {
+        {
+            execute("create table a (id int primary key, x varchar)");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key",
+                "Table already exists: A");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_value",
+                "Table already exists: A");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key,wrap_value",
+                "Table already exists: A");
+
+            execute("drop table a");
+        }
+
+        {
+            execute("create table a (id int primary key, x varchar) with wrap_key");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar)",
+                "Table already exists: A");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_value",
+                "Table already exists: A");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key,wrap_value",
+                "Table already exists: A");
+
+            execute("drop table a");
+        }
+
+        {
+            execute("create table a (id int primary key, x varchar) with wrap_value");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar)",
+                "Table already exists: A");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key",
+                "Table already exists: A");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key,wrap_value",
+                "Table already exists: A");
+
+            execute("drop table a");
+        }
+
+        {
+            execute("create table a (id int primary key, x varchar) with wrap_key,wrap_value");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar)",
+                "Table already exists: A");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_value",
+                "Table already exists: A");
+
+            assertDdlCommandThrows("create table a (id int primary key, x varchar) with wrap_key",
+                "Table already exists: A");
+
+            execute("drop table a");
+        }
+    }
+
+    /**
+     * Test that it's possible to create tables with matching key and/or value primitive types.
+     */
+    public void testDynamicTablesInteroperability() {
+        execute("create table a (id int primary key, x varchar) with \"wrap_value=false\"");
+
+        execute("create table b (id long primary key, y varchar) with \"wrap_value=false\"");
+
+        execute("create table c (id int primary key, z long) with \"wrap_value=false\"");
+
+        execute("create table d (id int primary key, w varchar) with \"wrap_value=false\"");
+
+        execute("drop table a");
+
+        execute("drop table b");
+
+        execute("drop table c");
+
+        execute("drop table d");
+    }
+
+    /**
+     * Test that when key or value has more than one column, wrap=false is forbidden.
+     */
+    public void testWrappingAlwaysOnWithComplexObjects() {
+        assertDdlCommandThrows("create table a (id int, x varchar, c long, primary key(id, c)) with \"wrap_key=false\"",
+            "WRAP_KEY cannot be false when composite primary key exists.");
+
+        assertDdlCommandThrows("create table a (id int, x varchar, c long, primary key(id)) with \"wrap_value=false\"",
+            "WRAP_VALUE cannot be false when multiple non-primary key columns exist.");
+    }
+
+    /**
+     * Test behavior when neither key nor value should be wrapped.
+     * @throws SQLException if failed.
+     */
+    public void testNoWrap() throws SQLException {
+        doTestKeyValueWrap(false, false);
+    }
+
+    /**
+     * Test behavior when only key is wrapped.
+     * @throws SQLException if failed.
+     */
+    public void testKeyWrap() throws SQLException {
+        doTestKeyValueWrap(true, false);
+    }
+
+    /**
+     * Test behavior when only value is wrapped.
+     * @throws SQLException if failed.
+     */
+    public void testValueWrap() throws SQLException {
+        doTestKeyValueWrap(false, true);
+    }
+
+    /**
+     * Test behavior when both key and value is wrapped.
+     * @throws SQLException if failed.
+     */
+    public void testKeyAndValueWrap() throws SQLException {
+        doTestKeyValueWrap(true, true);
+    }
+
+    /**
+     * Test behavior for given combination of wrap flags.
+     * @param wrapKey Whether key wrap should be enforced.
+     * @param wrapVal Whether value wrap should be enforced.
+     * @throws SQLException if failed.
+     */
+    private void doTestKeyValueWrap(boolean wrapKey, boolean wrapVal) throws SQLException {
+        try {
+            String sql = String.format("CREATE TABLE T (\"id\" int primary key, \"x\" varchar) WITH " +
+                "\"wrap_key=%b,wrap_value=%b\"", wrapKey, wrapVal);
+
+            if (wrapKey)
+                sql += ",\"key_type=tkey\"";
+
+            if (wrapVal)
+                sql += ",\"value_type=tval\"";
+
+            execute(sql);
+
+            execute("INSERT INTO T(\"id\", \"x\") values(1, 'a')");
+
+            LinkedHashMap<String, String> resCols = new LinkedHashMap<>();
+
+            List<Object> resData = new ArrayList<>();
+
+            try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1")) {
+                try (ResultSet colsRs = conn.getMetaData().getColumns(null, QueryUtils.DFLT_SCHEMA, "T", ".*")) {
+                    while (colsRs.next())
+                        resCols.put(colsRs.getString("COLUMN_NAME"),
+                            DataType.getTypeClassName(DataType.convertSQLTypeToValueType(colsRs
+                                .getShort("DATA_TYPE"))));
+                }
+
+                try (PreparedStatement ps = conn.prepareStatement("SELECT * FROM T")) {
+                    try (ResultSet dataRs = ps.executeQuery()) {
+                        assertTrue(dataRs.next());
+
+                        for (int i = 0; i < dataRs.getMetaData().getColumnCount(); i++)
+                            resData.add(dataRs.getObject(i + 1));
+                    }
+                }
+            }
+
+            LinkedHashMap<String, String> expCols = new LinkedHashMap<>();
+
+            expCols.put("id", Integer.class.getName());
+            expCols.put("x", String.class.getName());
+
+            assertEquals(expCols, resCols);
+
+            assertEqualsCollections(Arrays.asList(1, "a"), resData);
+
+            Object key = createKeyForWrapTest(1, wrapKey);
+
+            Object val = client().cache(cacheName("T")).withKeepBinary().get(key);
+
+            assertNotNull(val);
+
+            assertEquals(createValueForWrapTest("a", wrapVal), val);
+        }
+        finally {
+            execute("DROP TABLE IF EXISTS T");
+        }
+    }
+
+    /**
+     * @param key Key to wrap.
+     * @param wrap Whether key should be wrapped.
+     * @return (optionally wrapped) key.
+     */
+    private Object createKeyForWrapTest(int key, boolean wrap) {
+        if (!wrap)
+            return key;
+
+        return client().binary().builder("tkey").setField("id", key).build();
+    }
+
+    /**
+     * @param val Value to wrap.
+     * @param wrap Whether value should be wrapped.
+     * @return (optionally wrapped) value.
+     */
+    private Object createValueForWrapTest(String val, boolean wrap) {
+        if (!wrap)
+            return val;
+
+        return client().binary().builder("tval").setField("x", val).build();
+    }
+
+    /**
      * Fill re-created table with data.
      */
     private void fillRecreatedTable() {
@@ -1025,19 +1273,30 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /**
-     * Test that {@code DROP TABLE} on non-public schema causes an exception.
-     *
-     * @throws Exception if failed.
+     * Test that arbitrary command yields specific error.
+     * @param cmd Command.
+     * @param expErrMsg Expected error message.
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public void testDropTableNotPublicSchema() throws Exception {
+    private void assertDdlCommandThrows(final String cmd, String expErrMsg) {
         GridTestUtils.assertThrows(null, new Callable<Object>() {
             @Override public Object call() throws Exception {
-                execute("DROP TABLE \"cache_idx\".\"Person\"");
+                execute(cmd);
 
                 return null;
             }
-        }, IgniteSQLException.class, "DROP TABLE can only be executed on PUBLIC schema.");
+        }, IgniteSQLException.class, expErrMsg);
+    }
+
+    /**
+     * Test that {@code DROP TABLE} on non-public schema causes an exception.
+     *
+     * @throws Exception if failed.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public void testDropTableNotPublicSchema() throws Exception {
+       assertDdlCommandThrows("DROP TABLE \"cache_idx\".\"Person\"",
+           "DROP TABLE can only be executed on PUBLIC schema.");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/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 b372eb3..8deb61f 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
@@ -86,7 +86,7 @@ public class IgniteSqlNotNullConstraintTest extends GridCommonAbstractTest {
     private static String CACHE_INTERCEPTOR = "cacheInterceptor";
 
     /** Expected error message. */
-    private static String ERR_MSG = "Null value is not allowed for field 'NAME'";
+    private static String ERR_MSG = "Null value is not allowed for column 'NAME'";
 
     /** Expected error message for read-through restriction. */
     private static String READ_THROUGH_ERR_MSG = "NOT NULL constraint is not supported when " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 1ee8a84..62860c0 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -251,20 +251,21 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         IgniteCache<Integer, BinaryObject> cacheB = ignite0.createCache(cacheBCfg());
 
-        assertFalse(spi.queryLocalSql(spi.schema(typeAA.cacheName()), "select * from A.A", null, Collections.emptySet(),
-            typeAA.name(), null, null).hasNext());
+        assertFalse(spi.queryLocalSql(spi.schema(typeAA.cacheName()), typeAA.cacheName(), "select * from A.A", null,
+            Collections.emptySet(), typeAA.name(), null, null).hasNext());
 
-        assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), "select * from A.B", null, Collections.emptySet(),
-            typeAB.name(), null, null).hasNext());
-
-        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), "select * from B.A", null, Collections.emptySet(),
-            typeBA.name(), null, null).hasNext());
+        assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), "select * from A.B", null,
+            Collections.emptySet(), typeAB.name(), null, null).hasNext());
 
-        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), "select * from B.A, A.B, A.A", null,
+        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(), "select * from B.A", null,
             Collections.emptySet(), typeBA.name(), null, null).hasNext());
 
+        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(),
+            "select * from B.A, A.B, A.A", null, Collections.emptySet(), typeBA.name(), null, null).hasNext());
+
         try {
-            spi.queryLocalSql(spi.schema(typeBA.cacheName()), "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba",
+            spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(),
+                "select aa.*, ab.*, ba.* from A.A aa, A.B ab, B.A ba",
                 null, Collections.emptySet(), typeBA.name(), null, null).hasNext();
 
             fail("Enumerations of aliases in select block must be prohibited");
@@ -273,11 +274,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
             // all fine
         }
 
-        assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), "select ab.* from A.B ab", null,
-            Collections.emptySet(), typeAB.name(), null, null).hasNext());
+        assertFalse(spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), "select ab.* from A.B ab",
+            null, Collections.emptySet(), typeAB.name(), null, null).hasNext());
 
-        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), "select   ba.*   from B.A  as ba", null,
-            Collections.emptySet(), typeBA.name(), null, null).hasNext());
+        assertFalse(spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(),
+            "select   ba.*   from B.A  as ba", null, Collections.emptySet(), typeBA.name(), null, null).hasNext());
 
         cacheA.put(1, aa("A", 1, "Vasya", 10).build());
         cacheA.put(1, ab(1, "Vasya", 20, "Some text about Vasya goes here.").build());
@@ -289,7 +290,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         // Query data.
         Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> res = spi.queryLocalSql(spi.schema(typeAA.cacheName()),
-            "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null);
+            typeAA.cacheName(), "from a order by age", null, Collections.emptySet(), typeAA.name(), null, null);
 
         assertTrue(res.hasNext());
         assertEquals(aa("A", 3, "Borya", 18).build(), value(res.next()));
@@ -297,8 +298,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next()));
         assertFalse(res.hasNext());
 
-        res = spi.queryLocalSql(spi.schema(typeAA.cacheName()), "select aa.* from a aa order by aa.age", null,
-            Collections.emptySet(), typeAA.name(), null, null);
+        res = spi.queryLocalSql(spi.schema(typeAA.cacheName()), typeAA.cacheName(),
+            "select aa.* from a aa order by aa.age", null, Collections.emptySet(), typeAA.name(), null, null);
 
         assertTrue(res.hasNext());
         assertEquals(aa("A", 3, "Borya", 18).build(), value(res.next()));
@@ -306,8 +307,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(aa("A", 2, "Valera", 19).build(), value(res.next()));
         assertFalse(res.hasNext());
 
-        res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), "from b order by name", null, Collections.emptySet(),
-            typeAB.name(), null, null);
+        res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(), "from b order by name", null,
+            Collections.emptySet(), typeAB.name(), null, null);
 
         assertTrue(res.hasNext());
         assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").build(), value(res.next()));
@@ -315,8 +316,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(res.next()));
         assertFalse(res.hasNext());
 
-        res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), "select bb.* from b as bb order by bb.name", null,
-            Collections.emptySet(), typeAB.name(), null, null);
+        res = spi.queryLocalSql(spi.schema(typeAB.cacheName()), typeAB.cacheName(),
+            "select bb.* from b as bb order by bb.name", null, Collections.emptySet(), typeAB.name(), null, null);
 
         assertTrue(res.hasNext());
         assertEquals(ab(1, "Vasya", 20, "Some text about Vasya goes here.").build(), value(res.next()));
@@ -324,8 +325,8 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(res.next()));
         assertFalse(res.hasNext());
 
-        res = spi.queryLocalSql(spi.schema(typeBA.cacheName()), "from a", null, Collections.emptySet(), typeBA.name(),
-            null, null);
+        res = spi.queryLocalSql(spi.schema(typeBA.cacheName()), typeBA.cacheName(), "from a", null,
+            Collections.emptySet(), typeBA.name(), null, null);
 
         assertTrue(res.hasNext());
         assertEquals(ba(2, "Kolya", 25, true).build(), value(res.next()));
@@ -333,7 +334,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         // Text queries
         Iterator<IgniteBiTuple<Integer, BinaryObjectImpl>> txtRes = spi.queryLocalText(spi.schema(typeAB.cacheName()),
-            "good", typeAB.name(), null);
+            typeAB.cacheName(), "good", typeAB.name(), null);
 
         assertTrue(txtRes.hasNext());
         assertEquals(ab(4, "Vitalya", 20, "Very Good guy").build(), value(txtRes.next()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/79d94cf4/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
index a6ddc8c..172cb90 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
@@ -122,7 +122,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             var ex = Assert.Throws<IgniteException>(() => cache.QueryFields(new SqlFieldsQuery(
                 "insert into foo(_key, name) values (?, ?)", 1, "bar")).GetAll());
 
-            Assert.AreEqual("Null value is not allowed for field 'ID'", ex.Message);
+            Assert.AreEqual("Null value is not allowed for column 'ID'", ex.Message);
         }
 
         /// <summary>
@@ -137,7 +137,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             var ex = Assert.Throws<IgniteException>(() => cache.QueryFields(new SqlFieldsQuery(
                 "insert into foo(_key, id) values (?, ?)", 1, 2)).GetAll());
 
-            Assert.AreEqual("Null value is not allowed for field 'NAME'", ex.Message);
+            Assert.AreEqual("Null value is not allowed for column 'NAME'", ex.Message);
         }
 
         /// <summary>


[13/29] ignite git commit: IGNITE-6539 WAL parser fails if empty log files exist in directory - Fixes #2794.

Posted by sb...@apache.org.
IGNITE-6539 WAL parser fails if empty log files exist in directory - Fixes #2794.

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


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

Branch: refs/heads/ignite-5937
Commit: 474479c3f624c3a3c67b6ae549a566caac1f6b1c
Parents: 3b1cad2
Author: dpavlov <dp...@gridgain.com>
Authored: Thu Oct 5 18:06:27 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Oct 5 18:06:27 2017 +0300

----------------------------------------------------------------------
 .../reader/StandaloneWalRecordsIterator.java    | 24 ++++--
 .../db/wal/reader/IgniteWalReaderTest.java      | 89 ++++++++++++++++++++
 2 files changed, 104 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/474479c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
index 24b2148..42bb410 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneWalRecordsIterator.java
@@ -45,10 +45,10 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.ByteBufferExp
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileInput;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWALPointer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager;
-import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentEofException;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -185,12 +185,11 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
      * Header record and its position is checked. WAL position is used to determine real index.
      * File index from file name is ignored.
      *
-     * @param allFiles files to scan
-     * @return list of file descriptors with checked header records, file index is set
-     * @throws IgniteCheckedException if IO error occurs
+     * @param allFiles files to scan.
+     * @return list of file descriptors with checked header records, having correct file index is set
      */
     private List<FileWriteAheadLogManager.FileDescriptor> scanIndexesFromFileHeaders(
-        @Nullable final File[] allFiles) throws IgniteCheckedException {
+        @Nullable final File[] allFiles) {
         if (allFiles == null || allFiles.length == 0)
             return Collections.emptyList();
 
@@ -198,7 +197,7 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
 
         for (File file : allFiles) {
             if (file.length() < HEADER_RECORD_SIZE)
-                continue;
+                continue;  //filter out this segment as it is too short
 
             FileWALPointer ptr;
 
@@ -211,17 +210,24 @@ class StandaloneWalRecordsIterator extends AbstractWalRecordsIterator {
                 // Header record must be agnostic to the serializer version.
                 final int type = in.readUnsignedByte();
 
-                if (type == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE)
-                    throw new SegmentEofException("Reached logical end of the segment", null);
+                if (type == WALRecord.RecordType.STOP_ITERATION_RECORD_TYPE) {
+                    if (log.isInfoEnabled())
+                        log.info("Reached logical end of the segment for file " + file);
+
+                    continue; //filter out this segment
+                }
                 ptr = RecordV1Serializer.readPosition(in);
             }
             catch (IOException e) {
-                throw new IgniteCheckedException("Failed to scan index from file [" + file + "]", e);
+                U.warn(log, "Failed to scan index from file [" + file + "]. Skipping this file during iteration", e);
+
+                continue; //filter out this segment
             }
 
             resultingDescs.add(new FileWriteAheadLogManager.FileDescriptor(file, ptr.index()));
         }
         Collections.sort(resultingDescs);
+
         return resultingDescs;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/474479c3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
index 6db2784..93df8b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/reader/IgniteWalReaderTest.java
@@ -75,6 +75,7 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
+import org.junit.Assert;
 
 import static org.apache.ignite.events.EventType.EVT_WAL_SEGMENT_ARCHIVED;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE;
@@ -340,6 +341,22 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Puts provided number of records to fill WAL
+     *
+     * @param ignite ignite instance
+     * @param recordsToWrite count
+     */
+    private void putAllDummyRecords(Ignite ignite, int recordsToWrite) {
+        IgniteCache<Object, Object> cache0 = ignite.cache(CACHE_NAME);
+
+        Map<Object, Object> values = new HashMap<>();
+
+        for (int i = 0; i < recordsToWrite; i++)
+            values.put(i, new IndexedObject(i));
+
+        cache0.putAll(values);
+    }
+    /**
      * Puts provided number of records to fill WAL under transactions
      *
      * @param ignite ignite instance
@@ -715,6 +732,78 @@ public class IgniteWalReaderTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Tests archive completed event is fired
+     *
+     * @throws Exception if failed
+     */
+    public void testFillWalForExactSegmentsCount() throws Exception {
+        customWalMode = WALMode.DEFAULT;
+
+        final CountDownLatch reqSegments = new CountDownLatch(15);
+        final Ignite ignite = startGrid("node0");
+
+        ignite.active(true);
+
+        final IgniteEvents evts = ignite.events();
+
+        if (!evts.isEnabled(EVT_WAL_SEGMENT_ARCHIVED))
+            assertTrue("nothing to test", false);
+
+        evts.localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event e) {
+                WalSegmentArchivedEvent archComplEvt = (WalSegmentArchivedEvent)e;
+                long idx = archComplEvt.getAbsWalSegmentIdx();
+                log.info("Finished archive for segment [" + idx + ", " +
+                    archComplEvt.getArchiveFile() + "]: [" + e + "]");
+
+                reqSegments.countDown();
+                return true;
+            }
+        }, EVT_WAL_SEGMENT_ARCHIVED);
+
+
+        int totalEntries = 0;
+        while (reqSegments.getCount() > 0) {
+            final int write = 500;
+            putAllDummyRecords(ignite, write);
+            totalEntries += write;
+            Assert.assertTrue("Too much entries generated, but segments was not become available",
+                totalEntries < 10000);
+        }
+        final String subfolderName = genDbSubfolderName(ignite, 0);
+
+        stopGrid("node0");
+
+        final String workDir = U.defaultWorkDirectory();
+        final IgniteWalIteratorFactory factory = createWalIteratorFactory(subfolderName, workDir);
+
+        scanIterateAndCount(factory, workDir, subfolderName, totalEntries, 0, null, null);
+    }
+
+    /**
+     * Tests reading of empty WAL from non filled cluster
+     *
+     * @throws Exception if failed.
+     */
+    public void testReadEmptyWal() throws Exception {
+        customWalMode = WALMode.DEFAULT;
+
+        final Ignite ignite = startGrid("node0");
+
+        ignite.active(true);
+        ignite.active(false);
+
+        final String subfolderName = genDbSubfolderName(ignite, 0);
+
+        stopGrid("node0");
+
+        final String workDir = U.defaultWorkDirectory();
+        final IgniteWalIteratorFactory factory = createWalIteratorFactory(subfolderName, workDir);
+
+        scanIterateAndCount(factory, workDir, subfolderName, 0, 0, null, null);
+    }
+
+    /**
      * Creates and fills cache with data.
      *
      * @param ig Ignite instance.


[24/29] ignite git commit: IGNITE-6054: Fixed tests.

Posted by sb...@apache.org.
IGNITE-6054: Fixed tests.


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

Branch: refs/heads/ignite-5937
Commit: 90624c64f76ae46cb98d30b0f7f0697ec6701fbb
Parents: 87e7b32
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 9 11:57:06 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 9 11:57:06 2017 +0300

----------------------------------------------------------------------
 .../cache/index/H2DynamicColumnsAbstractBasicSelfTest.java        | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/90624c64/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
index 34be34d..b9f8c61 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
@@ -341,8 +341,7 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
      */
     private void doTestAlterTableOnFlatValue(String tblName) {
         assertThrows("ALTER TABLE " + tblName + " ADD COLUMN y varchar",
-            "ADD COLUMN is not supported for tables created with wrap_value=false param. " +
-                "(To enable ADD COLUMN, create table with wrap_value=true param).");
+            "Cannot add column(s) because table was created with WRAP_VALUE=false option.");
     }
 
     /**


[04/29] ignite git commit: IGNITE-6550 Added new snapshot WAL record type - Fixes #2795.

Posted by sb...@apache.org.
IGNITE-6550 Added new snapshot WAL record type - Fixes #2795.

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


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

Branch: refs/heads/ignite-5937
Commit: b2a0295894dbdeb638ac8f764549a724578322ac
Parents: e6e1ca9
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Wed Oct 4 18:59:40 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Oct 4 18:59:40 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/pagemem/wal/WALPointer.java |  4 +-
 .../pagemem/wal/record/SnapshotRecord.java      | 58 ++++++++++++++++++++
 .../internal/pagemem/wal/record/WALRecord.java  |  6 +-
 .../GridCacheDatabaseSharedManager.java         |  3 +
 .../cache/persistence/wal/FileWALPointer.java   |  3 +
 5 files changed, 71 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b2a02958/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java
index 73321f1..7ca4aee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/WALPointer.java
@@ -17,10 +17,12 @@
 
 package org.apache.ignite.internal.pagemem.wal;
 
+import java.io.Serializable;
+
 /**
  *
  */
-public interface WALPointer {
+public interface WALPointer extends Serializable {
     /**
      * Pointer to the next record. Can be used only for original pointers obtained from WAL manager.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2a02958/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/SnapshotRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/SnapshotRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/SnapshotRecord.java
new file mode 100644
index 0000000..3c3a77b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/SnapshotRecord.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagemem.wal.record;
+
+/**
+ * Wal snapshot record.
+ */
+public class SnapshotRecord extends WALRecord {
+    /** Snapshot id. */
+    private long snapshotId;
+
+    /** Full snapshot or incremental. */
+    private boolean full;
+
+    /**
+     *
+     */
+    public SnapshotRecord(long snapshotId, boolean full) {
+        this.snapshotId = snapshotId;
+        this.full = full;
+    }
+
+    /**
+     *
+     */
+    public long getSnapshotId() {
+        return snapshotId;
+    }
+
+    /**
+     *
+     */
+    public boolean isFull() {
+        return full;
+    }
+
+    /**
+     *
+     */
+    @Override public RecordType type() {
+        return RecordType.SNAPSHOT;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2a02958/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
index 89f3c86..08bba1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/WALRecord.java
@@ -163,8 +163,10 @@ public abstract class WALRecord {
         BTREE_META_PAGE_INIT_ROOT2,
 
         /** Partition destroy. */
-        PARTITION_DESTROY
-        ;
+        PARTITION_DESTROY,
+
+        /** Snapshot record. */
+        SNAPSHOT;
 
         /** */
         private static final RecordType[] VALS = RecordType.values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2a02958/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 2d89942..33f065e 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -2383,6 +2383,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 return new Checkpoint(cpEntry, cpPages, curr);
             }
             else {
+                if (curr.nextSnapshot)
+                    cctx.wal().fsync(null);
+
                 if (printCheckpointStats) {
                     if (log.isInfoEnabled())
                         LT.info(log, String.format("Skipping checkpoint (no pages were modified) [" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2a02958/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
index 3716de2..4998700 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWALPointer.java
@@ -24,6 +24,9 @@ import org.apache.ignite.internal.util.typedef.internal.S;
  * File WAL pointer.
  */
 public class FileWALPointer implements WALPointer, Comparable<FileWALPointer> {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
     /** Absolute WAL segment file index (incrementing counter) */
     private final long idx;
 


[22/29] ignite git commit: Fixed JavaDoc.

Posted by sb...@apache.org.
Fixed JavaDoc.


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

Branch: refs/heads/ignite-5937
Commit: e9764f159262e1aec12c8b418061294cc9914726
Parents: 85261a3
Author: devozerov <vo...@gridgain.com>
Authored: Mon Oct 9 10:48:06 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Oct 9 10:48:06 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/wal/serializer/RecordV2Serializer.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e9764f15/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
index 0a5bf01..a06b6d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/serializer/RecordV2Serializer.java
@@ -39,7 +39,7 @@ import static org.apache.ignite.internal.processors.cache.persistence.wal.serial
  * Record V2 serializer.
  * Stores records in following format:
  * <ul>
- * <li>Record type from {@link WALRecord.RecordType#ordinal()} incremented by 1</li>
+ * <li>Record type from {@link WALRecord.RecordType} incremented by 1</li>
  * <li>WAL pointer to double check consistency</li>
  * <li>Record length</li>
  * <li>Data</li>


[05/29] ignite git commit: IGNITE-6307 If getAll() fails with NPE, onHeap entry is not removed, for local cache

Posted by sb...@apache.org.
IGNITE-6307 If getAll() fails with NPE, onHeap entry is not removed, for local cache


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

Branch: refs/heads/ignite-5937
Commit: e74163a99870bd50b24ea6799b7566dc37682f74
Parents: b2a0295
Author: Nikolay Izhikov <NI...@gmail.com>
Authored: Wed Oct 4 19:31:17 2017 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Oct 4 19:31:17 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      | 16 +++++++
 .../cache/GridCacheAbstractFullApiSelfTest.java | 45 +++++++++++++++++++-
 2 files changed, 59 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e74163a9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 6d9f0d3..8c5d6f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1853,6 +1853,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (tx == null || tx.implicit()) {
             Map<KeyCacheObject, EntryGetResult> misses = null;
 
+            Set<GridCacheEntryEx> newLocalEntries = null;
+
             final AffinityTopologyVersion topVer = tx == null ? ctx.affinity().affinityTopologyVersion() :
                 tx.topologyVersion();
 
@@ -1925,6 +1927,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                             }
 
                             if (!skipEntry) {
+                                boolean isNewLocalEntry = this.map.getEntry(ctx, key) == null;
+
                                 entry = entryEx(key);
 
                                 if (entry == null) {
@@ -1934,6 +1938,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                                     break;
                                 }
 
+                                if (isNewLocalEntry) {
+                                    if (newLocalEntries == null)
+                                        newLocalEntries = new HashSet<>();
+
+                                    newLocalEntries.add(entry);
+                                }
+
                                 if (storeEnabled) {
                                     res = entry.innerGetAndReserveForLoad(updateMetrics,
                                         evt,
@@ -2128,6 +2139,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                         ctx.evicts().touch(peekEx(key0), topVer);
                 }
 
+                if (newLocalEntries != null) {
+                    for (GridCacheEntryEx entry : newLocalEntries)
+                        removeEntry(entry);
+                }
+
                 return new GridFinishedFuture<>(e);
             }
             catch (IgniteCheckedException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/e74163a9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index bf27e26..e6c9589 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -919,13 +919,54 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception In case of error.
      */
-    public void testGetAllWithNulls() throws Exception {
+    public void testGetAllWithLastNull() throws Exception {
         final IgniteCache<String, Integer> cache = jcache();
 
-        final Set<String> c = new HashSet<>();
+        final Set<String> c = new LinkedHashSet<>();
+
+        c.add("key1");
+        c.add(null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.getAll(c);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAllWithFirstNull() throws Exception {
+        final IgniteCache<String, Integer> cache = jcache();
+
+        final Set<String> c = new LinkedHashSet<>();
+
+        c.add(null);
+        c.add("key1");
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.getAll(c);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAllWithInTheMiddle() throws Exception {
+        final IgniteCache<String, Integer> cache = jcache();
+
+        final Set<String> c = new LinkedHashSet<>();
 
         c.add("key1");
         c.add(null);
+        c.add("key2");
 
         GridTestUtils.assertThrows(log, new Callable<Void>() {
             @Override public Void call() throws Exception {


[02/29] ignite git commit: Removed excluding ML from examples/src

Posted by sb...@apache.org.
Removed excluding ML from examples/src


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

Branch: refs/heads/ignite-5937
Commit: 78f77b1fd158955a3203811ba909e01fe7c474f5
Parents: 20033c7
Author: oleg-ostanin <oo...@gridgain.com>
Authored: Wed Oct 4 17:35:50 2017 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Oct 4 17:35:50 2017 +0300

----------------------------------------------------------------------
 assembly/release-fabric-base.xml | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/78f77b1f/assembly/release-fabric-base.xml
----------------------------------------------------------------------
diff --git a/assembly/release-fabric-base.xml b/assembly/release-fabric-base.xml
index a16d996..8e8bd75 100644
--- a/assembly/release-fabric-base.xml
+++ b/assembly/release-fabric-base.xml
@@ -239,7 +239,6 @@
             <excludes>
                 <exclude>**/package.html</exclude>
                 <exclude>src/test/**</exclude>
-                <exclude>src/main/ml/**</exclude>
             </excludes>
         </fileSet>
 


[07/29] ignite git commit: IGNITE-6550 Added serializer version getter

Posted by sb...@apache.org.
IGNITE-6550 Added serializer version getter


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

Branch: refs/heads/ignite-5937
Commit: e98e392e28d5a3209e97bc4ff27e4883d8332899
Parents: 8bd51d8
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Thu Oct 5 10:56:46 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Oct 5 10:56:46 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/wal/FileWriteAheadLogManager.java       | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e98e392e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 6a75dd2..383c605 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -451,6 +451,13 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
     }
 
     /**
+     * @return Latest serializer version.
+     */
+    public int serializerVersion() {
+        return serializerVersion;
+    }
+
+    /**
      * Checks if there was elapsed significant period of inactivity.
      * If WAL auto-archive is enabled using {@link #walAutoArchiveAfterInactivity} > 0 this method will activate
      * roll over by timeout<br>


[09/29] ignite git commit: IGNITE-6529 JDBC: support column metadata 'nullable' property. This closes #2793.

Posted by sb...@apache.org.
IGNITE-6529 JDBC: support column metadata 'nullable' property. This closes #2793.


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

Branch: refs/heads/ignite-5937
Commit: c116bfc68a05376285946b8eae10ed6e51f848e0
Parents: e98e392
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Oct 5 11:55:26 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 5 11:55:26 2017 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |  28 ++--
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  33 ++--
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     |   3 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |   4 +-
 .../internal/jdbc2/JdbcDatabaseMetadata.java    |  70 +++++++--
 .../cache/query/GridCacheQueryManager.java      |  87 ++++++++++-
 .../query/GridCacheQuerySqlMetadataJobV2.java   | 154 +++++++++++++++++++
 .../query/GridCacheQuerySqlMetadataV2.java      | 101 ++++++++++++
 .../cache/query/GridCacheSqlMetadata.java       |   8 +
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |  10 ++
 .../processors/odbc/jdbc/JdbcColumnMetaV2.java  |  74 +++++++++
 .../odbc/jdbc/JdbcConnectionContext.java        |   4 +-
 .../odbc/jdbc/JdbcMetaColumnsResult.java        |  28 +++-
 .../odbc/jdbc/JdbcMetaColumnsResultV2.java      |  50 ++++++
 .../odbc/jdbc/JdbcRequestHandler.java           |  32 +++-
 .../processors/odbc/jdbc/JdbcResult.java        |   8 +
 16 files changed, 642 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
index 6020a3a..bdc6644 100755
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
@@ -42,7 +42,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteVersionUtils;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.query.QueryEntityEx;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -78,11 +78,16 @@ public class JdbcMetadataSelfTest extends GridCommonAbstractTest {
 
         cfg.setCacheConfiguration(
             cacheConfiguration("pers").setQueryEntities(Arrays.asList(
-                new QueryEntity(AffinityKey.class, Person.class)
-                    .setIndexes(Arrays.asList(
-                        new QueryIndex("orgId"),
-                        new QueryIndex().setFields(persFields))))
-            ),
+                new QueryEntityEx(
+                    new QueryEntity(AffinityKey.class.getName(), Person.class.getName())
+                        .addQueryField("name", String.class.getName(), null)
+                        .addQueryField("age", Integer.class.getName(), null)
+                        .addQueryField("orgId", Integer.class.getName(), null)
+                        .setIndexes(Arrays.asList(
+                            new QueryIndex("orgId"),
+                            new QueryIndex().setFields(persFields))))
+                    .setNotNullFields(new HashSet<>(Arrays.asList("age", "name")))
+            )),
             cacheConfiguration("org").setQueryEntities(Arrays.asList(
                 new QueryEntity(AffinityKey.class, Organization.class))));
 
@@ -208,7 +213,6 @@ public class JdbcMetadataSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testGetColumns() throws Exception {
-        final boolean primitivesInformationIsLostAfterStore = ignite(0).configuration().getMarshaller() instanceof BinaryMarshaller;
         try (Connection conn = DriverManager.getConnection(BASE_URL)) {
             DatabaseMetaData meta = conn.getMetaData();
 
@@ -232,11 +236,15 @@ public class JdbcMetadataSelfTest extends GridCommonAbstractTest {
                 if ("NAME".equals(name)) {
                     assertEquals(VARCHAR, rs.getInt("DATA_TYPE"));
                     assertEquals("VARCHAR", rs.getString("TYPE_NAME"));
-                    assertEquals(1, rs.getInt("NULLABLE"));
-                } else if ("AGE".equals(name) || "ORGID".equals(name)) {
+                    assertEquals(0, rs.getInt("NULLABLE"));
+                } else if ("AGE".equals(name)) {
+                    assertEquals(INTEGER, rs.getInt("DATA_TYPE"));
+                    assertEquals("INTEGER", rs.getString("TYPE_NAME"));
+                    assertEquals(0, rs.getInt("NULLABLE"));
+                } else if ("ORGID".equals(name)) {
                     assertEquals(INTEGER, rs.getInt("DATA_TYPE"));
                     assertEquals("INTEGER", rs.getString("TYPE_NAME"));
-                    assertEquals(primitivesInformationIsLostAfterStore ? 1 : 0, rs.getInt("NULLABLE"));
+                    assertEquals(1, rs.getInt("NULLABLE"));
                 }
 
                 cnt++;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index 01b2e8a..abbe4e1 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -41,7 +41,7 @@ import org.apache.ignite.cache.affinity.AffinityKey;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteVersionUtils;
-import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.query.QueryEntityEx;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -118,13 +118,15 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
         persFields.put("age", false);
 
         IgniteCache<AffinityKey, Person> personCache = jcache(grid(0), cacheConfiguration(
-            new QueryEntity(AffinityKey.class.getName(), Person.class.getName())
-                .addQueryField("name", String.class.getName(), null)
-                .addQueryField("age", Integer.class.getName(), null)
-                .addQueryField("orgId", Integer.class.getName(), null)
-                .setIndexes(Arrays.asList(
-                    new QueryIndex("orgId"),
-                    new QueryIndex().setFields(persFields)))
+            new QueryEntityEx(
+                new QueryEntity(AffinityKey.class.getName(), Person.class.getName())
+                    .addQueryField("name", String.class.getName(), null)
+                    .addQueryField("age", Integer.class.getName(), null)
+                    .addQueryField("orgId", Integer.class.getName(), null)
+                    .setIndexes(Arrays.asList(
+                        new QueryIndex("orgId"),
+                        new QueryIndex().setFields(persFields))))
+                .setNotNullFields(new HashSet<>(Arrays.asList("age", "name")))
             ), "pers");
 
         assert personCache != null;
@@ -251,9 +253,6 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testGetColumns() throws Exception {
-        final boolean primitivesInformationIsLostAfterStore = ignite(0).configuration().getMarshaller()
-            instanceof BinaryMarshaller;
-
         try (Connection conn = DriverManager.getConnection(URL)) {
             conn.setSchema("pers");
 
@@ -279,18 +278,22 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 if ("NAME".equals(name)) {
                     assert rs.getInt("DATA_TYPE") == VARCHAR;
                     assert "VARCHAR".equals(rs.getString("TYPE_NAME"));
+                    assert rs.getInt("NULLABLE") == 0;
+                } else if ("ORGID".equals(name)) {
+                    assert rs.getInt("DATA_TYPE") == INTEGER;
+                    assert "INTEGER".equals(rs.getString("TYPE_NAME"));
                     assert rs.getInt("NULLABLE") == 1;
-                } else if ("AGE".equals(name) || "ORGID".equals(name)) {
+                } else if ("AGE".equals(name)) {
                     assert rs.getInt("DATA_TYPE") == INTEGER;
                     assert "INTEGER".equals(rs.getString("TYPE_NAME"));
-                    assertEquals(primitivesInformationIsLostAfterStore ? 1 : 0, rs.getInt("NULLABLE"));
+                    assert rs.getInt("NULLABLE") == 0;
                 }
-                if ("_KEY".equals(name)) {
+                else if ("_KEY".equals(name)) {
                     assert rs.getInt("DATA_TYPE") == OTHER;
                     assert "OTHER".equals(rs.getString("TYPE_NAME"));
                     assert rs.getInt("NULLABLE") == 0;
                 }
-                if ("_VAL".equals(name)) {
+                else if ("_VAL".equals(name)) {
                     assert rs.getInt("DATA_TYPE") == OTHER;
                     assert "OTHER".equals(rs.getString("TYPE_NAME"));
                     assert rs.getInt("NULLABLE") == 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
index 2ce7983..8b26900 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
@@ -29,7 +29,6 @@ import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 import org.apache.ignite.internal.IgniteVersionUtils;
-import org.apache.ignite.internal.jdbc2.JdbcUtils;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcColumnMeta;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcIndexMeta;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaColumnsRequest;
@@ -845,7 +844,7 @@ public class JdbcThinDatabaseMetadata implements DatabaseMetaData {
         row.add((Integer)null);
         row.add((Integer)null);
         row.add(10);
-        row.add(JdbcUtils.nullable(colMeta.columnName(), colMeta.dataTypeClass()) ? 1 : 0 );
+        row.add(colMeta.isNullable() ? 1 : 0);
         row.add((String)null);
         row.add((String)null);
         row.add(Integer.MAX_VALUE);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
index 7ac9c2c..688f908 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
@@ -50,10 +50,10 @@ public class JdbcThinTcpIo {
     private static final ClientListenerProtocolVersion VER_2_1_0 = ClientListenerProtocolVersion.create(2, 1, 0);
 
     /** Version 2.3.1. */
-    private static final ClientListenerProtocolVersion VER_2_3_1 = ClientListenerProtocolVersion.create(2, 3, 1);
+    private static final ClientListenerProtocolVersion VER_2_3_0 = ClientListenerProtocolVersion.create(2, 3, 0);
 
     /** Current version. */
-    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_3_1;
+    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_3_0;
 
     /** Initial output stream capacity for handshake. */
     private static final int HANDSHAKE_MSG_SIZE = 13;

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
index 4c21cbd..03fde79 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -59,7 +60,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     private final JdbcConnection conn;
 
     /** Metadata. */
-    private Map<String, Map<String, Map<String, String>>> meta;
+    private Map<String, Map<String, Map<String, ColumnInfo>>> meta;
 
     /** Index info. */
     private Collection<List<Object>> indexes;
@@ -714,7 +715,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
         List<List<?>> rows = new LinkedList<>();
 
         if (validCatalogPattern(catalog) && (tblTypes == null || Arrays.asList(tblTypes).contains("TABLE"))) {
-            for (Map.Entry<String, Map<String, Map<String, String>>> schema : meta.entrySet()) {
+            for (Map.Entry<String, Map<String, Map<String, ColumnInfo>>> schema : meta.entrySet()) {
                 if (matches(schema.getKey(), schemaPtrn)) {
                     for (String tbl : schema.getValue().keySet()) {
                         if (matches(tbl, tblNamePtrn))
@@ -796,14 +797,14 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
         int cnt = 0;
 
         if (validCatalogPattern(catalog)) {
-            for (Map.Entry<String, Map<String, Map<String, String>>> schema : meta.entrySet()) {
+            for (Map.Entry<String, Map<String, Map<String, ColumnInfo>>> schema : meta.entrySet()) {
                 if (matches(schema.getKey(), schemaPtrn)) {
-                    for (Map.Entry<String, Map<String, String>> tbl : schema.getValue().entrySet()) {
+                    for (Map.Entry<String, Map<String, ColumnInfo>> tbl : schema.getValue().entrySet()) {
                         if (matches(tbl.getKey(), tblNamePtrn)) {
-                            for (Map.Entry<String, String> col : tbl.getValue().entrySet()) {
+                            for (Map.Entry<String, ColumnInfo> col : tbl.getValue().entrySet()) {
                                 rows.add(columnRow(schema.getKey(), tbl.getKey(), col.getKey(),
-                                    JdbcUtils.type(col.getValue()), JdbcUtils.typeName(col.getValue()),
-                                    JdbcUtils.nullable(col.getKey(), col.getValue()), ++cnt));
+                                    JdbcUtils.type(col.getValue().typeName()), JdbcUtils.typeName(col.getValue().typeName()),
+                                    !col.getValue().isNotNull(), ++cnt));
                             }
                         }
                     }
@@ -925,9 +926,9 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
         List<List<?>> rows = new LinkedList<>();
 
         if (validCatalogPattern(catalog)) {
-            for (Map.Entry<String, Map<String, Map<String, String>>> schema : meta.entrySet()) {
+            for (Map.Entry<String, Map<String, Map<String, ColumnInfo>>> schema : meta.entrySet()) {
                 if (matches(schema.getKey(), schemaPtrn)) {
-                    for (Map.Entry<String, Map<String, String>> tbl : schema.getValue().entrySet()) {
+                    for (Map.Entry<String, Map<String, ColumnInfo>> tbl : schema.getValue().entrySet()) {
                         if (matches(tbl.getKey(), tblNamePtrn))
                             rows.add(Arrays.<Object>asList(null, schema.getKey(), tbl.getKey(), "_KEY", 1, "_KEY"));
                     }
@@ -1361,10 +1362,21 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
                 Collection<String> types = m.types();
 
-                Map<String, Map<String, String>> typesMap = U.newHashMap(types.size());
+                Map<String, Map<String, ColumnInfo>> typesMap = U.newHashMap(types.size());
 
                 for (String type : types) {
-                    typesMap.put(type.toUpperCase(), m.fields(type));
+                    Collection<String> notNullFields = m.notNullFields(type);
+
+                    Map<String, ColumnInfo> fields = new LinkedHashMap<>();
+
+                    for (Map.Entry<String, String> fld : m.fields(type).entrySet()) {
+                        ColumnInfo colInfo = new ColumnInfo(fld.getValue(),
+                            notNullFields == null ? false : notNullFields.contains(fld.getKey()));
+
+                        fields.put(fld.getKey(), colInfo);
+                    }
+
+                    typesMap.put(type.toUpperCase(), fields);
 
                     for (GridCacheSqlIndexMetadata idx : m.indexes(type)) {
                         int cnt = 0;
@@ -1435,7 +1447,41 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
         @Override public Collection<GridCacheSqlMetadata> call() throws Exception {
             IgniteCache cache = ignite.cache(cacheName);
 
-            return ((IgniteCacheProxy)cache).context().queries().sqlMetadata();
+            return ((IgniteCacheProxy)cache).context().queries().sqlMetadataV2();
+        }
+    }
+
+    /**
+     * Column info.
+     */
+    private static class ColumnInfo {
+        /** Class name. */
+        private final String typeName;
+
+        /** Not null flag. */
+        private final boolean notNull;
+
+        /**
+         * @param typeName Type name.
+         * @param notNull Not null flag.
+         */
+        private ColumnInfo(String typeName, boolean notNull) {
+            this.typeName = typeName;
+            this.notNull = notNull;
+        }
+
+        /**
+         * @return Type name.
+         */
+        public String typeName() {
+            return typeName;
+        }
+
+        /**
+         * @return Not null flag.
+         */
+        public boolean isNotNull() {
+            return notNull;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index f873461..64e74fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -113,6 +113,7 @@ import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteReducer;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.resources.IgniteInstanceResource;
@@ -145,6 +146,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /** Maximum number of query detail metrics to evict at once. */
     private static final int QRY_DETAIL_METRICS_EVICTION_LIMIT = 10_000;
 
+    /** Support 'not null' field constraint since v 2.3.0. */
+    private static final IgniteProductVersion NOT_NULLS_SUPPORT_VER = IgniteProductVersion.fromString("2.3.0");
+
     /** Comparator for priority queue with query detail metrics with priority to new metrics. */
     private static final Comparator<GridCacheQueryDetailMetricsAdapter> QRY_DETAIL_METRICS_PRIORITY_NEW_CMP =
         new Comparator<GridCacheQueryDetailMetricsAdapter>() {
@@ -1908,6 +1912,79 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     }
 
     /**
+     * Gets SQL metadata with not nulls fields.
+     *
+     * @return SQL metadata.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public Collection<GridCacheSqlMetadata> sqlMetadataV2() throws IgniteCheckedException {
+        if (!enterBusy())
+            throw new IllegalStateException("Failed to get metadata (grid is stopping).");
+
+        try {
+            Callable<Collection<CacheSqlMetadata>> job = new GridCacheQuerySqlMetadataJobV2();
+
+            // Remote nodes that have current cache.
+            Collection<ClusterNode> nodes = CU.affinityNodes(cctx, AffinityTopologyVersion.NONE);
+
+            Collection<Collection<CacheSqlMetadata>> res = new ArrayList<>(nodes.size() + 1);
+
+            IgniteInternalFuture<Collection<Collection<CacheSqlMetadata>>> rmtFut = null;
+
+            // Get metadata from remote nodes.
+            if (!nodes.isEmpty()) {
+                boolean allNodesNew = true;
+
+                for (ClusterNode n : nodes) {
+                    if (n.version().compareTo(NOT_NULLS_SUPPORT_VER) < 0)
+                        allNodesNew = false;
+                }
+
+                if (!allNodesNew)
+                    return sqlMetadata();
+
+                rmtFut = cctx.closures().callAsyncNoFailover(BROADCAST, Collections.singleton(job), nodes, true, 0);
+            }
+
+            // Get local metadata.
+            IgniteInternalFuture<Collection<CacheSqlMetadata>> locFut = cctx.closures().callLocalSafe(job, true);
+
+            if (rmtFut != null)
+                res.addAll(rmtFut.get());
+
+            res.add(locFut.get());
+
+            Map<String, Collection<CacheSqlMetadata>> map = new HashMap<>();
+
+            for (Collection<CacheSqlMetadata> col : res) {
+                for (CacheSqlMetadata meta : col) {
+                    String name = meta.cacheName();
+
+                    Collection<CacheSqlMetadata> cacheMetas = map.get(name);
+
+                    if (cacheMetas == null)
+                        map.put(name, cacheMetas = new LinkedList<>());
+
+                    cacheMetas.add(meta);
+                }
+            }
+
+            Collection<GridCacheSqlMetadata> col = new ArrayList<>(map.size());
+
+            // Metadata for current cache must be first in list.
+            col.add(new GridCacheQuerySqlMetadataV2(map.remove(cacheName)));
+
+            for (Collection<CacheSqlMetadata> metas : map.values())
+                col.add(new GridCacheQuerySqlMetadataV2(metas));
+
+            return col;
+        }
+        finally {
+            leaveBusy();
+        }
+    }
+
+    /**
      * @param <K> Key type.
      * @param <V> Value type.
      * @param includeBackups Include backups.
@@ -2079,7 +2156,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * Cache metadata.
      */
-    private static class CacheSqlMetadata implements GridCacheSqlMetadata {
+    public static class CacheSqlMetadata implements GridCacheSqlMetadata {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2183,6 +2260,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
+        @Override public Collection<String> notNullFields(String type) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
         @Override public Map<String, String> keyClasses() {
             return keyClasses;
         }
@@ -2236,7 +2318,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * Cache metadata index.
      */
-    private static class CacheSqlIndexMetadata implements GridCacheSqlIndexMetadata {
+    public static class CacheSqlIndexMetadata implements GridCacheSqlIndexMetadata {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2371,6 +2453,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         /**
          * @return Metadata.
+         * @throws IgniteCheckedException On error.
          */
         public List<GridQueryFieldMetadata> metaData() throws IgniteCheckedException {
             get(); // Ensure that result is ready.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQuerySqlMetadataJobV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQuerySqlMetadataJobV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQuerySqlMetadataJobV2.java
new file mode 100644
index 0000000..9907d1a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQuerySqlMetadataJobV2.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.C1;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.P1;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Metadata job.
+ */
+@GridInternal
+class GridCacheQuerySqlMetadataJobV2 implements IgniteCallable<Collection<GridCacheQueryManager.CacheSqlMetadata>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Number of fields to report when no fields defined. Includes _key and _val columns. */
+    private static final int NO_FIELDS_COLUMNS_COUNT = 2;
+
+    /** Grid */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridCacheQueryManager.CacheSqlMetadata> call() {
+        final GridKernalContext ctx = ((IgniteKernal)ignite).context();
+
+        Collection<String> cacheNames = F.viewReadOnly(ctx.cache().caches(),
+            new C1<IgniteInternalCache<?, ?>, String>() {
+                @Override public String apply(IgniteInternalCache<?, ?> c) {
+                    return c.name();
+                }
+            },
+            new P1<IgniteInternalCache<?, ?>>() {
+                @Override public boolean apply(IgniteInternalCache<?, ?> c) {
+                    return !CU.isSystemCache(c.name()) && !DataStructuresProcessor.isDataStructureCache(c.name());
+                }
+            }
+        );
+
+        return F.transform(cacheNames, new C1<String, GridCacheQueryManager.CacheSqlMetadata>() {
+            @Override public GridCacheQueryManager.CacheSqlMetadata apply(String cacheName) {
+                Collection<GridQueryTypeDescriptor> types = ctx.query().types(cacheName);
+
+                Collection<String> names = U.newHashSet(types.size());
+                Map<String, String> keyClasses = U.newHashMap(types.size());
+                Map<String, String> valClasses = U.newHashMap(types.size());
+                Map<String, Map<String, String>> fields = U.newHashMap(types.size());
+                Map<String, Collection<GridCacheSqlIndexMetadata>> indexes = U.newHashMap(types.size());
+                Map<String, Set<String>> notNullFields = U.newHashMap(types.size());
+
+                for (GridQueryTypeDescriptor type : types) {
+                    // Filter internal types (e.g., data structures).
+                    if (type.name().startsWith("GridCache"))
+                        continue;
+
+                    names.add(type.name());
+
+                    keyClasses.put(type.name(), type.keyClass().getName());
+                    valClasses.put(type.name(), type.valueClass().getName());
+
+                    int size = type.fields().isEmpty() ? NO_FIELDS_COLUMNS_COUNT : type.fields().size();
+
+                    Map<String, String> fieldsMap = U.newLinkedHashMap(size);
+                    HashSet<String> notNullFieldsSet = U.newHashSet(1);
+
+                    // _KEY and _VAL are not included in GridIndexingTypeDescriptor.valueFields
+                    if (type.fields().isEmpty()) {
+                        fieldsMap.put("_KEY", type.keyClass().getName());
+                        fieldsMap.put("_VAL", type.valueClass().getName());
+                    }
+
+                    for (Map.Entry<String, Class<?>> e : type.fields().entrySet()) {
+                        String fieldName = e.getKey();
+
+                        fieldsMap.put(fieldName.toUpperCase(), e.getValue().getName());
+
+                        if (type.property(fieldName).notNull())
+                            notNullFieldsSet.add(fieldName.toUpperCase());
+                    }
+
+                    fields.put(type.name(), fieldsMap);
+                    notNullFields.put(type.name(), notNullFieldsSet);
+
+                    Map<String, GridQueryIndexDescriptor> idxs = type.indexes();
+
+                    Collection<GridCacheSqlIndexMetadata> indexesCol = new ArrayList<>(idxs.size());
+
+                    for (Map.Entry<String, GridQueryIndexDescriptor> e : idxs.entrySet()) {
+                        GridQueryIndexDescriptor desc = e.getValue();
+
+                        // Add only SQL indexes.
+                        if (desc.type() == QueryIndexType.SORTED) {
+                            Collection<String> idxFields = new LinkedList<>();
+                            Collection<String> descendings = new LinkedList<>();
+
+                            for (String idxField : e.getValue().fields()) {
+                                String idxFieldUpper = idxField.toUpperCase();
+
+                                idxFields.add(idxFieldUpper);
+
+                                if (desc.descending(idxField))
+                                    descendings.add(idxFieldUpper);
+                            }
+
+                            indexesCol.add(new GridCacheQueryManager.CacheSqlIndexMetadata(e.getKey().toUpperCase(),
+                                idxFields, descendings, false));
+                        }
+                    }
+
+                    indexes.put(type.name(), indexesCol);
+                }
+
+                return new GridCacheQuerySqlMetadataV2(cacheName, names, keyClasses, valClasses, fields, indexes,
+                    notNullFields);
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQuerySqlMetadataV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQuerySqlMetadataV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQuerySqlMetadataV2.java
new file mode 100644
index 0000000..66821c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQuerySqlMetadataV2.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.query;
+
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Cache metadata with not null field.
+ */
+public class GridCacheQuerySqlMetadataV2 extends GridCacheQueryManager.CacheSqlMetadata {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Not null fields. */
+    private Map<String, Set<String>> notNullFields;
+
+    /**
+     * Required by {@link Externalizable}.
+     */
+    public GridCacheQuerySqlMetadataV2() {
+        // No-op.
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param types Types.
+     * @param keyClasses Key classes map.
+     * @param valClasses Value classes map.
+     * @param fields Fields maps.
+     * @param indexes Indexes.
+     * @param notNullFields Not null fields.
+     */
+    GridCacheQuerySqlMetadataV2(@Nullable String cacheName, Collection<String> types, Map<String, String> keyClasses,
+        Map<String, String> valClasses, Map<String, Map<String, String>> fields,
+        Map<String, Collection<GridCacheSqlIndexMetadata>> indexes, Map<String, Set<String>> notNullFields) {
+        super(cacheName, types, keyClasses, valClasses, fields, indexes);
+
+        this.notNullFields = notNullFields;
+    }
+
+    /**
+     * @param metas Meta data instances from different nodes.
+     */
+    GridCacheQuerySqlMetadataV2(Iterable<GridCacheQueryManager.CacheSqlMetadata> metas) {
+        super(metas);
+
+        notNullFields = new HashMap<>();
+
+        for (GridCacheQueryManager.CacheSqlMetadata meta : metas) {
+            if (meta instanceof GridCacheQuerySqlMetadataV2) {
+                GridCacheQuerySqlMetadataV2 metaV2 = (GridCacheQuerySqlMetadataV2)meta;
+
+                notNullFields.putAll(metaV2.notNullFields);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> notNullFields(String type) {
+        return notNullFields.get(type);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        U.writeMap(out, notNullFields);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        notNullFields = U.readHashMap(in);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlMetadata.java
index 724962e..ddc2860 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlMetadata.java
@@ -78,6 +78,14 @@ public interface GridCacheSqlMetadata extends Externalizable {
     @Nullable public Map<String, String> fields(String type);
 
     /**
+     * Gets not null fields.
+     *
+     * @param type Type name.
+     * @return Not null fields collection map or {@code null} if type name is unknown.
+     */
+    Collection<String> notNullFields(String type);
+
+    /**
      * @return Key classes.
      */
     public Map<String, String> keyClasses();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
index 9f145e0..d927c26 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.odbc.jdbc;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.jdbc.thin.JdbcThinUtils;
+import org.apache.ignite.internal.jdbc2.JdbcUtils;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -126,6 +127,15 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
         return dataTypeClass;
     }
 
+    /**
+     * Return 'nullable' flag in compatibility mode (according with column name and column type).
+     *
+     * @return {@code true} in case the column allows null values. Otherwise returns {@code false}
+     */
+    public boolean isNullable() {
+        return JdbcUtils.nullable(colName, dataTypeClass);
+    }
+
     /** {@inheritDoc} */
     @Override public void writeBinary(BinaryWriterExImpl writer) {
         writer.writeString(schemaName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV2.java
new file mode 100644
index 0000000..a2b4acf
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV2.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.odbc.jdbc;
+
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC column metadata.
+ */
+public class JdbcColumnMetaV2 extends JdbcColumnMeta{
+    /** Allow nulls . */
+    private boolean nullable;
+
+    /**
+     * Default constructor is used for serialization.
+     */
+    JdbcColumnMetaV2() {
+        // No-op.
+    }
+
+    /**
+     * @param schemaName Schema.
+     * @param tblName Table.
+     * @param colName Column.
+     * @param cls Type.
+     * @param nullable Allow nulls.
+     */
+    public JdbcColumnMetaV2(String schemaName, String tblName, String colName, Class<?> cls, boolean nullable) {
+        super(schemaName, tblName, colName, cls);
+
+        this.nullable = nullable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isNullable() {
+        return nullable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) {
+        super.writeBinary(writer);
+
+        writer.writeBoolean(nullable);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) {
+        super.readBinary(reader);
+
+        nullable = reader.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcColumnMetaV2.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
index 38d1972..a6a7aa5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
@@ -38,10 +38,10 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext {
     private static final ClientListenerProtocolVersion VER_2_1_5 = ClientListenerProtocolVersion.create(2, 1, 5);
 
     /** Version 2.3.1: added "multiple statements query" feature. */
-    public static final ClientListenerProtocolVersion VER_2_3_1 = ClientListenerProtocolVersion.create(2, 3, 1);
+    public static final ClientListenerProtocolVersion VER_2_3_0 = ClientListenerProtocolVersion.create(2, 3, 0);
 
     /** Current version. */
-    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_3_1;
+    private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_3_0;
 
     /** Supported versions. */
     private static final Set<ClientListenerProtocolVersion> SUPPORTED_VERS = new HashSet<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
index da270de..9931ce0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
@@ -51,6 +51,25 @@ public class JdbcMetaColumnsResult extends JdbcResult {
     }
 
     /**
+     * Used by children classes.
+     * @param type Type ID.
+     */
+    protected JdbcMetaColumnsResult(byte type) {
+        super(type);
+    }
+
+    /**
+     * Used by children classes.
+     * @param type Type ID.
+     * @param meta Columns metadata.
+     */
+    protected JdbcMetaColumnsResult(byte type, Collection<JdbcColumnMeta> meta) {
+        super(type);
+
+        this.meta = new ArrayList<>(meta);
+    }
+
+    /**
      * @return Columns metadata.
      */
     public List<JdbcColumnMeta> meta() {
@@ -83,7 +102,7 @@ public class JdbcMetaColumnsResult extends JdbcResult {
             meta = new ArrayList<>(size);
 
             for (int i = 0; i < size; ++i) {
-                JdbcColumnMeta m = new JdbcColumnMeta();
+                JdbcColumnMeta m = createMetaColumn();
 
                 m.readBinary(reader);
 
@@ -92,6 +111,13 @@ public class JdbcMetaColumnsResult extends JdbcResult {
         }
     }
 
+    /**
+     * @return Empty columns metadata to deserialization.
+     */
+    protected JdbcColumnMeta createMetaColumn() {
+        return new JdbcColumnMeta();
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(JdbcMetaColumnsResult.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResultV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResultV2.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResultV2.java
new file mode 100644
index 0000000..2673a13
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResultV2.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.odbc.jdbc;
+
+import java.util.Collection;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC columns metadata result.
+ */
+public class JdbcMetaColumnsResultV2 extends JdbcMetaColumnsResult {
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaColumnsResultV2() {
+        super(META_COLUMNS_V2);
+    }
+
+    /**
+     * @param meta Columns metadata.
+     */
+    JdbcMetaColumnsResultV2(Collection<JdbcColumnMeta> meta) {
+        super(META_COLUMNS_V2, meta);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected JdbcColumnMeta createMetaColumn() {
+        return new JdbcColumnMetaV2();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaColumnsResultV2.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 202f813..166402f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -43,6 +44,7 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
 import org.apache.ignite.internal.processors.odbc.odbc.OdbcQueryGetColumnsMetaRequest;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
@@ -51,6 +53,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_3_0;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BATCH_EXEC;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_COLUMNS;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_INDEXES;
@@ -291,7 +294,7 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
             qry.setSchema(schemaName);
 
             List<FieldsQueryCursor<List<?>>> results = ctx.query().querySqlFieldsNoCache(qry, true,
-                protocolVer.compareTo(JdbcConnectionContext.VER_2_3_1) < 0);
+                protocolVer.compareTo(VER_2_3_0) < 0);
 
             if (results.size() == 1) {
                 FieldsQueryCursor<List<?>> qryCur = results.get(0);
@@ -559,9 +562,10 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
      * @param req Get columns metadata request.
      * @return Response.
      */
+    @SuppressWarnings("unchecked")
     private JdbcResponse getColumnsMeta(JdbcMetaColumnsRequest req) {
         try {
-            Collection<JdbcColumnMeta> meta = new HashSet<>();
+            Collection<JdbcColumnMeta> meta = new LinkedHashSet<>();
 
             for (String cacheName : ctx.cache().publicCacheNames()) {
                 for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
@@ -572,11 +576,22 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
                         continue;
 
                     for (Map.Entry<String, Class<?>> field : table.fields().entrySet()) {
-                        if (!matches(field.getKey(), req.columnName()))
+                        String colName = field.getKey();
+
+                        if (!matches(colName, req.columnName()))
                             continue;
 
-                        JdbcColumnMeta columnMeta = new JdbcColumnMeta(table.schemaName(), table.tableName(),
-                            field.getKey(), field.getValue());
+                        JdbcColumnMeta columnMeta;
+
+                        if (protocolVer.compareTo(VER_2_3_0) >= 0) {
+                            GridQueryProperty prop = table.property(colName);
+
+                            columnMeta = new JdbcColumnMetaV2(table.schemaName(), table.tableName(),
+                                field.getKey(), field.getValue(), !prop.notNull());
+                        }
+                        else
+                            columnMeta = new JdbcColumnMeta(table.schemaName(), table.tableName(),
+                                field.getKey(), field.getValue());
 
                         if (!meta.contains(columnMeta))
                             meta.add(columnMeta);
@@ -584,7 +599,12 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
                 }
             }
 
-            JdbcMetaColumnsResult res = new JdbcMetaColumnsResult(meta);
+            JdbcMetaColumnsResult res;
+
+            if (protocolVer.compareTo(VER_2_3_0) >= 0)
+                res = new JdbcMetaColumnsResultV2(meta);
+            else
+                res = new JdbcMetaColumnsResult(meta);
 
             return new JdbcResponse(res);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c116bfc6/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
index c6c7438..6d460e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
@@ -59,6 +59,9 @@ public class JdbcResult implements JdbcRawBinarylizable {
     /** Multiple statements query results. */
     static final byte QRY_EXEC_MULT = 13;
 
+    /** Columns metadata result V2. */
+    static final byte META_COLUMNS_V2 = 14;
+
     /** Success status. */
     private byte type;
 
@@ -147,6 +150,11 @@ public class JdbcResult implements JdbcRawBinarylizable {
 
                 break;
 
+            case META_COLUMNS_V2:
+                res = new JdbcMetaColumnsResultV2();
+
+                break;
+
             default:
                 throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']');
         }


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

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


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

Branch: refs/heads/ignite-5937
Commit: 921404a6f5b256cf6404822bc7e439f2cad44c5d
Parents: 410c84f 90624c6
Author: sboikov <sb...@gridgain.com>
Authored: Mon Oct 9 13:41:58 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Oct 9 13:41:58 2017 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |  28 +-
 .../internal/jdbc2/JdbcStatementSelfTest.java   | 130 +++++++-
 .../JdbcThinAbstractDmlStatementSelfTest.java   |   2 +-
 .../thin/JdbcThinAutoCloseServerCursorTest.java |   8 +-
 .../jdbc/thin/JdbcThinComplexQuerySelfTest.java |   2 +-
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |   4 +
 .../JdbcThinDynamicIndexAbstractSelfTest.java   |   2 +-
 .../jdbc/thin/JdbcThinEmptyCacheSelfTest.java   |   2 +-
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  39 +--
 .../JdbcThinMissingLongArrayResultsTest.java    |   2 +-
 .../jdbc/thin/JdbcThinNoDefaultSchemaTest.java  |   5 +-
 .../thin/JdbcThinPreparedStatementSelfTest.java |   2 +-
 .../jdbc/thin/JdbcThinResultSetSelfTest.java    |   2 +-
 .../jdbc/thin/JdbcThinStatementSelfTest.java    |  46 +--
 .../org/apache/ignite/IgniteJdbcDriver.java     |   9 +-
 .../connection/GridClientNioTcpConnection.java  |   2 +
 .../internal/jdbc/thin/JdbcThinConnection.java  |   6 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     |   3 +-
 .../jdbc/thin/JdbcThinPreparedStatement.java    |   5 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   |   9 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |   4 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |  13 +
 .../internal/jdbc2/JdbcDatabaseMetadata.java    | 124 +++++---
 .../jdbc2/JdbcQueryMultipleStatementsTask.java  | 167 ++++++++++
 .../ignite/internal/jdbc2/JdbcQueryTask.java    | 154 +++-------
 .../internal/jdbc2/JdbcQueryTaskResult.java     | 120 ++++++++
 .../ignite/internal/jdbc2/JdbcQueryTaskV3.java  |  94 ++++++
 .../ignite/internal/jdbc2/JdbcResultSet.java    | 175 ++++++++---
 .../ignite/internal/jdbc2/JdbcStatement.java    | 270 +++++++++--------
 .../internal/jdbc2/JdbcStatementResultInfo.java |  73 +++++
 .../jdbc2/JdbcStreamedPreparedStatement.java    |  19 +-
 .../internal/pagemem/wal/record/DataEntry.java  |  13 +-
 .../pagemem/wal/record/UnwrapDataEntry.java     |  22 +-
 .../GridCacheDatabaseSharedManager.java         |   2 +
 .../persistence/pagemem/PageMemoryImpl.java     |  18 ++
 .../reader/StandaloneWalRecordsIterator.java    |  35 ++-
 .../wal/serializer/RecordV2Serializer.java      |   2 +-
 .../cache/query/GridCacheQueryManager.java      |  87 +++++-
 .../query/GridCacheQuerySqlMetadataJobV2.java   | 154 ++++++++++
 .../query/GridCacheQuerySqlMetadataV2.java      | 101 +++++++
 .../cache/query/GridCacheSqlMetadata.java       |   8 +
 .../cache/query/IgniteQueryErrorCode.java       |   2 +-
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |  10 +
 .../processors/odbc/jdbc/JdbcColumnMetaV2.java  |  74 +++++
 .../odbc/jdbc/JdbcConnectionContext.java        |   4 +-
 .../odbc/jdbc/JdbcMetaColumnsResult.java        |  28 +-
 .../odbc/jdbc/JdbcMetaColumnsResultV2.java      |  50 +++
 .../odbc/jdbc/JdbcRequestHandler.java           |  32 +-
 .../processors/odbc/jdbc/JdbcResult.java        |   8 +
 .../utils/PlatformConfigurationUtils.java       |   3 +
 .../processors/query/GridQueryIndexing.java     |  22 +-
 .../processors/query/GridQueryProcessor.java    |   8 +-
 .../query/QueryTypeDescriptorImpl.java          |  42 ++-
 .../query/property/QueryBinaryProperty.java     |   1 -
 .../handlers/cache/GridCacheCommandHandler.java |  12 +-
 .../internal/visor/query/VisorQueryTask.java    |  36 ++-
 ...IgniteClientCacheInitializationFailTest.java |  18 +-
 .../db/wal/reader/IgniteWalReaderTest.java      | 103 ++++++-
 .../query/h2/DmlStatementsProcessor.java        |  26 +-
 .../internal/processors/query/h2/H2Schema.java  |  17 +-
 .../internal/processors/query/h2/H2TypeKey.java |  64 ++++
 .../processors/query/h2/IgniteH2Indexing.java   |  33 +-
 .../query/h2/ddl/DdlStatementsProcessor.java    |  36 ++-
 .../query/h2/sql/GridSqlCreateTable.java        |  34 +++
 .../query/h2/sql/GridSqlQueryParser.java        |  58 +++-
 ...ynamicColumnsAbstractConcurrentSelfTest.java |  57 ++--
 ...umnsConcurrentAtomicPartitionedSelfTest.java |   2 +-
 ...lumnsConcurrentAtomicReplicatedSelfTest.java |   2 +-
 ...currentTransactionalPartitionedSelfTest.java |   2 +-
 ...ncurrentTransactionalReplicatedSelfTest.java |   5 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |  43 +++
 .../cache/index/H2DynamicTableSelfTest.java     | 301 +++++++++++++++++--
 .../query/IgniteSqlNotNullConstraintTest.java   |   2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  47 +--
 .../Cache/Query/CacheDmlQueriesTest.cs          |   4 +-
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs   |   5 +
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |   4 +
 modules/web-console/backend/app/agentSocket.js  |  21 +-
 .../web-console/backend/app/browsersHandler.js  |   9 +-
 modules/web-console/backend/package.json        |   4 +-
 .../app/modules/agent/AgentManager.service.js   |  25 +-
 .../app/modules/agent/decompress.worker.js      |  34 +++
 .../frontend/app/modules/sql/sql.controller.js  |  36 ++-
 .../frontend/app/utils/SimpleWorkerPool.js      | 119 ++++++++
 modules/web-console/frontend/package.json       |   8 +-
 .../web-console/frontend/views/sql/sql.tpl.pug  |  14 +-
 86 files changed, 2783 insertions(+), 645 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/921404a6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------


[10/29] ignite git commit: IGNITE-6358: JDBC thick: support multiple statements. This closes #2777.

Posted by sb...@apache.org.
IGNITE-6358: JDBC thick: support multiple statements. This closes #2777.


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

Branch: refs/heads/ignite-5937
Commit: df3c407f8c40d4dcd603ee35215199cd1d60c38a
Parents: c116bfc
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Oct 5 16:32:33 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Oct 5 16:32:33 2017 +0300

----------------------------------------------------------------------
 .../internal/jdbc2/JdbcStatementSelfTest.java   | 130 ++++++++-
 .../org/apache/ignite/IgniteJdbcDriver.java     |   9 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |  13 +
 .../internal/jdbc2/JdbcDatabaseMetadata.java    |  54 ++--
 .../jdbc2/JdbcQueryMultipleStatementsTask.java  | 167 ++++++++++++
 .../ignite/internal/jdbc2/JdbcQueryTask.java    | 154 +++--------
 .../internal/jdbc2/JdbcQueryTaskResult.java     | 120 +++++++++
 .../ignite/internal/jdbc2/JdbcQueryTaskV3.java  |  94 +++++++
 .../ignite/internal/jdbc2/JdbcResultSet.java    | 175 +++++++++---
 .../ignite/internal/jdbc2/JdbcStatement.java    | 270 ++++++++++---------
 .../internal/jdbc2/JdbcStatementResultInfo.java |  73 +++++
 .../jdbc2/JdbcStreamedPreparedStatement.java    |  19 +-
 12 files changed, 966 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementSelfTest.java
index 138eef5..d3f77e0 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementSelfTest.java
@@ -45,7 +45,8 @@ public class JdbcStatementSelfTest extends GridCommonAbstractTest {
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** JDBC URL. */
-    private static final String BASE_URL = CFG_URL_PREFIX + "cache=default@modules/clients/src/test/config/jdbc-config.xml";
+    private static final String BASE_URL = CFG_URL_PREFIX
+        + "cache=default:multipleStatementsAllowed=true@modules/clients/src/test/config/jdbc-config.xml";
 
     /** SQL query. */
     private static final String SQL = "select * from Person where age > 30";
@@ -250,6 +251,133 @@ public class JdbcStatementSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testExecuteQueryMultipleOnlyResultSets() throws Exception {
+        assert conn.getMetaData().supportsMultipleResultSets();
+
+        int stmtCnt = 10;
+
+        StringBuilder sql = new StringBuilder();
+
+        for (int i = 0; i < stmtCnt; ++i)
+            sql.append("select ").append(i).append("; ");
+
+        assert stmt.execute(sql.toString());
+
+        for (int i = 0; i < stmtCnt; ++i) {
+            assert stmt.getMoreResults();
+
+            ResultSet rs = stmt.getResultSet();
+
+            assert rs.next();
+            assert rs.getInt(1) == i;
+            assert !rs.next();
+        }
+
+        assert !stmt.getMoreResults();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExecuteQueryMultipleOnlyDml() throws Exception {
+        assert conn.getMetaData().supportsMultipleResultSets();
+
+        conn.setSchema(null);
+
+        int stmtCnt = 10;
+
+        StringBuilder sql = new StringBuilder(
+            "drop table if exists test; create table test(ID int primary key, NAME varchar(20)); ");
+
+        for (int i = 0; i < stmtCnt; ++i)
+            sql.append("insert into test (ID, NAME) values (" + i + ", 'name_" + i +"'); ");
+
+        assert !stmt.execute(sql.toString());
+
+        // DROP TABLE statement
+        assert stmt.getResultSet() == null;
+        assert stmt.getUpdateCount() == 0;
+
+        // CREATE TABLE statement
+        assert stmt.getResultSet() == null;
+        assert stmt.getUpdateCount() == 0;
+
+        for (int i = 0; i < stmtCnt; ++i) {
+            assert stmt.getMoreResults();
+
+            assert stmt.getResultSet() == null;
+            assert stmt.getUpdateCount() == 1;
+        }
+
+        assert !stmt.getMoreResults();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExecuteQueryMultipleMixed() throws Exception {
+        assert conn.getMetaData().supportsMultipleResultSets();
+
+        conn.setSchema(null);
+
+        int stmtCnt = 10;
+
+        StringBuilder sql = new StringBuilder(
+            "drop table if exists test; create table test(ID int primary key, NAME varchar(20)); ");
+
+        for (int i = 0; i < stmtCnt; ++i) {
+            if (i % 2 == 0)
+                sql.append(" insert into test (ID, NAME) values (" + i + ", 'name_" + i + "'); ");
+            else
+                sql.append(" select * from test where id < " + i + "; ");
+        }
+
+        assert !stmt.execute(sql.toString());
+
+        // DROP TABLE statement
+        assert stmt.getResultSet() == null;
+        assert stmt.getUpdateCount() == 0;
+
+        // CREATE TABLE statement
+        assert stmt.getResultSet() == null;
+        assert stmt.getUpdateCount() == 0;
+
+        boolean notEmptyResult = false;
+
+        for (int i = 0; i < stmtCnt; ++i) {
+            assert stmt.getMoreResults();
+
+            if (i % 2 == 0) {
+                assert stmt.getResultSet() == null;
+                assert stmt.getUpdateCount() == 1;
+            }
+            else {
+                assert stmt.getUpdateCount() == -1;
+
+                ResultSet rs = stmt.getResultSet();
+
+                assert rs.getMetaData().getColumnCount() == 2;
+
+                int rowsCnt = 0;
+
+                while(rs.next())
+                    rowsCnt++;
+
+                assert rowsCnt <= (i + 1) / 2;
+
+                if (rowsCnt == (i + 1) / 2)
+                    notEmptyResult = true;
+            }
+        }
+
+        assert notEmptyResult;
+
+        assert !stmt.getMoreResults();
+    }
+
+    /**
      * Person.
      */
     @SuppressWarnings("UnusedDeclaration")

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
index f519589..b03e387 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
@@ -331,6 +331,9 @@ public class IgniteJdbcDriver implements Driver {
     /** Whether DML streaming will overwrite existing cache entries. */
     private static final String PARAM_STREAMING_ALLOW_OVERWRITE = "streamingAllowOverwrite";
 
+    /** Allow queries with multiple statements. */
+    private static final String PARAM_MULTIPLE_STMTS = "multipleStatementsAllowed";
+
     /** Hostname property name. */
     public static final String PROP_HOST = PROP_PREFIX + "host";
 
@@ -376,6 +379,9 @@ public class IgniteJdbcDriver implements Driver {
     /** Whether DML streaming will overwrite existing cache entries. */
     public static final String PROP_STREAMING_ALLOW_OVERWRITE = PROP_PREFIX + PARAM_STREAMING_ALLOW_OVERWRITE;
 
+    /** Allow query with multiple statements. */
+    public static final String PROP_MULTIPLE_STMTS = PROP_PREFIX + PARAM_MULTIPLE_STMTS;
+
     /** Cache name property name. */
     public static final String PROP_CFG = PROP_PREFIX + "cfg";
 
@@ -447,7 +453,8 @@ public class IgniteJdbcDriver implements Driver {
             new JdbcDriverPropertyInfo("Distributed Joins", info.getProperty(PROP_DISTRIBUTED_JOINS), ""),
             new JdbcDriverPropertyInfo("Enforce Join Order", info.getProperty(JdbcThinUtils.PROP_ENFORCE_JOIN_ORDER), ""),
             new JdbcDriverPropertyInfo("Lazy query execution", info.getProperty(JdbcThinUtils.PROP_LAZY), ""),
-            new JdbcDriverPropertyInfo("Transactions Allowed", info.getProperty(PROP_TX_ALLOWED), "")
+            new JdbcDriverPropertyInfo("Transactions Allowed", info.getProperty(PROP_TX_ALLOWED), ""),
+            new JdbcDriverPropertyInfo("Queries with multiple statements allowed", info.getProperty(PROP_MULTIPLE_STMTS), "")
         );
 
         if (info.getProperty(PROP_CFG) != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
index fde16ff..ccc09ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
@@ -80,6 +80,7 @@ import static org.apache.ignite.IgniteJdbcDriver.PROP_DISTRIBUTED_JOINS;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_ENFORCE_JOIN_ORDER;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_LAZY;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_LOCAL;
+import static org.apache.ignite.IgniteJdbcDriver.PROP_MULTIPLE_STMTS;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_NODE_ID;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_TX_ALLOWED;
 import static org.apache.ignite.IgniteJdbcDriver.PROP_STREAMING;
@@ -164,6 +165,9 @@ public class JdbcConnection implements Connection {
     /** Allow overwrites for duplicate keys on streamed {@code INSERT}s. */
     private final boolean streamAllowOverwrite;
 
+    /** Allow queries with multiple statements. */
+    private final boolean multipleStmts;
+
     /** Statements. */
     final Set<JdbcStatement> statements = new HashSet<>();
 
@@ -204,6 +208,8 @@ public class JdbcConnection implements Connection {
         // by IgniteDataStreamer.DFLT_PARALLEL_OPS_MULTIPLIER will be used
         streamNodeParOps = Integer.parseInt(props.getProperty(PROP_STREAMING_PER_NODE_PAR_OPS, "0"));
 
+        multipleStmts = Boolean.parseBoolean(props.getProperty(PROP_MULTIPLE_STMTS));
+
         String nodeIdProp = props.getProperty(PROP_NODE_ID);
 
         if (nodeIdProp != null)
@@ -841,6 +847,13 @@ public class JdbcConnection implements Connection {
     }
 
     /**
+     * @return {@code true} if multiple statements allowed, {@code false} otherwise.
+     */
+    boolean isMultipleStatementsAllowed() {
+        return multipleStmts;
+    }
+
+    /**
      * @return Local query flag.
      */
     boolean isLocalQuery() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
index 03fde79..2fe24bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
@@ -319,7 +319,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public boolean supportsMultipleResultSets() {
-        return false;
+        return conn.isMultipleStatementsAllowed();
     }
 
     /** {@inheritDoc} */
@@ -675,7 +675,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getProcedures(String catalog, String schemaPtrn,
         String procedureNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("PROCEDURE_CAT", "PROCEDURE_SCHEM", "PROCEDURE_NAME",
@@ -689,7 +689,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getProcedureColumns(String catalog, String schemaPtrn, String procedureNamePtrn,
         String colNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("PROCEDURE_CAT", "PROCEDURE_SCHEM", "PROCEDURE_NAME",
@@ -725,7 +725,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
             }
         }
 
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "TABLE_TYPE", "REMARKS", "TYPE_CAT",
@@ -766,7 +766,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public ResultSet getCatalogs() throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.singletonList("TABLE_CAT"),
@@ -778,7 +778,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public ResultSet getTableTypes() throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.singletonList("TABLE_TYPE"),
@@ -812,7 +812,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
             }
         }
 
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "DATA_TYPE",
@@ -870,7 +870,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getColumnPrivileges(String catalog, String schema, String tbl,
         String colNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -883,7 +883,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getTablePrivileges(String catalog, String schemaPtrn,
         String tblNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -896,7 +896,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getBestRowIdentifier(String catalog, String schema, String tbl, int scope,
         boolean nullable) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -908,7 +908,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public ResultSet getVersionColumns(String catalog, String schema, String tbl) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -936,7 +936,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
             }
         }
 
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "COLUMN_NAME", "KEY_SEQ", "PK_NAME"),
@@ -948,7 +948,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public ResultSet getImportedKeys(String catalog, String schema, String tbl) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -960,7 +960,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public ResultSet getExportedKeys(String catalog, String schema, String tbl) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -973,7 +973,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTbl,
         String foreignCatalog, String foreignSchema, String foreignTbl) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -985,7 +985,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public ResultSet getTypeInfo() throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -1000,7 +1000,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
         boolean approximate) throws SQLException {
         updateMetaData();
 
-        Collection<List<?>> rows = new ArrayList<>(indexes.size());
+        List<List<?>> rows = new ArrayList<>(indexes.size());
 
         if (validCatalogPattern(catalog)) {
             for (List<Object> idx : indexes) {
@@ -1029,7 +1029,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
             }
         }
 
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("TABLE_CAT", "TABLE_SCHEM", "TABLE_NAME", "NON_UNIQUE", "INDEX_QUALIFIER",
@@ -1106,7 +1106,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getUDTs(String catalog, String schemaPtrn, String typeNamePtrn,
         int[] types) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -1144,7 +1144,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getSuperTypes(String catalog, String schemaPtrn,
         String typeNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -1157,7 +1157,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getSuperTables(String catalog, String schemaPtrn,
         String tblNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -1170,7 +1170,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getAttributes(String catalog, String schemaPtrn, String typeNamePtrn,
         String attributeNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -1233,7 +1233,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
             }
         }
 
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("TABLE_SCHEM", "TABLE_CATALOG"),
@@ -1259,7 +1259,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public ResultSet getClientInfoProperties() throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),
@@ -1272,7 +1272,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getFunctions(String catalog, String schemaPtrn,
         String functionNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("FUNCTION_CAT", "FUNCTION_SCHEM", "FUNCTION_NAME",
@@ -1286,7 +1286,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getFunctionColumns(String catalog, String schemaPtrn, String functionNamePtrn,
         String colNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Arrays.asList("FUNCTION_CAT", "FUNCTION_SCHEM", "FUNCTION_NAME",
@@ -1305,7 +1305,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
     /** {@inheritDoc} */
     @Override public ResultSet getPseudoColumns(String catalog, String schemaPtrn, String tblNamePtrn,
         String colNamePtrn) throws SQLException {
-        return new JdbcResultSet(null,
+        return new JdbcResultSet(true, null,
             conn.createStatement0(),
             Collections.<String>emptyList(),
             Collections.<String>emptyList(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsTask.java
new file mode 100644
index 0000000..bf7c24e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryMultipleStatementsTask.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.jdbc2;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteJdbcDriver;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+
+/**
+ * Task for SQL queries execution through {@link IgniteJdbcDriver}.
+ * The query can contains several SQL statements.
+ */
+class JdbcQueryMultipleStatementsTask implements IgniteCallable<List<JdbcStatementResultInfo>> {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Ignite. */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** Schema name. */
+    private final String schemaName;
+
+    /** Sql. */
+    private final String sql;
+
+    /** Operation type flag - query or not. */
+    private Boolean isQry;
+
+    /** Args. */
+    private final Object[] args;
+
+    /** Fetch size. */
+    private final int fetchSize;
+
+    /** Local execution flag. */
+    private final boolean loc;
+
+    /** Local query flag. */
+    private final boolean locQry;
+
+    /** Collocated query flag. */
+    private final boolean collocatedQry;
+
+    /** Distributed joins flag. */
+    private final boolean distributedJoins;
+
+    /** Enforce join order flag. */
+    private final boolean enforceJoinOrder;
+
+    /** Lazy query execution flag. */
+    private final boolean lazy;
+
+    /**
+     * @param ignite Ignite.
+     * @param schemaName Schema name.
+     * @param sql Sql query.
+     * @param isQry Operation type flag - query or not - to enforce query type check.
+     * @param loc Local execution flag.
+     * @param args Args.
+     * @param fetchSize Fetch size.
+     * @param locQry Local query flag.
+     * @param collocatedQry Collocated query flag.
+     * @param distributedJoins Distributed joins flag.
+     * @param enforceJoinOrder Enforce joins order falg.
+     * @param lazy Lazy query execution flag.
+     */
+    public JdbcQueryMultipleStatementsTask(Ignite ignite, String schemaName, String sql, Boolean isQry, boolean loc,
+        Object[] args, int fetchSize, boolean locQry, boolean collocatedQry, boolean distributedJoins,
+        boolean enforceJoinOrder, boolean lazy) {
+        this.ignite = ignite;
+        this.args = args;
+        this.schemaName = schemaName;
+        this.sql = sql;
+        this.isQry = isQry;
+        this.fetchSize = fetchSize;
+        this.loc = loc;
+        this.locQry = locQry;
+        this.collocatedQry = collocatedQry;
+        this.distributedJoins = distributedJoins;
+        this.enforceJoinOrder = enforceJoinOrder;
+        this.lazy = lazy;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<JdbcStatementResultInfo> call() throws Exception {
+        SqlFieldsQuery qry = (isQry != null ? new JdbcSqlFieldsQuery(sql, isQry) : new SqlFieldsQuery(sql))
+            .setArgs(args);
+
+        qry.setPageSize(fetchSize);
+        qry.setLocal(locQry);
+        qry.setCollocated(collocatedQry);
+        qry.setDistributedJoins(distributedJoins);
+        qry.setEnforceJoinOrder(enforceJoinOrder);
+        qry.setLazy(lazy);
+        qry.setSchema(schemaName);
+
+        GridKernalContext ctx = ((IgniteKernal)ignite).context();
+
+        List<FieldsQueryCursor<List<?>>> curs = ctx.query().querySqlFieldsNoCache(qry, true, false);
+
+        List<JdbcStatementResultInfo> resultsInfo = new ArrayList<>(curs.size());
+
+        for (FieldsQueryCursor<List<?>> cur0 : curs) {
+            QueryCursorImpl<List<?>> cur = (QueryCursorImpl<List<?>>)cur0;
+
+            long updCnt = -1;
+
+            UUID qryId = null;
+
+            if (!cur.isQuery()) {
+                List<List<?>> items = cur.getAll();
+
+                assert items != null && items.size() == 1 && items.get(0).size() == 1
+                    && items.get(0).get(0) instanceof Long :
+                    "Invalid result set for not-SELECT query. [qry=" + sql +
+                        ", res=" + S.toString(List.class, items) + ']';
+
+                updCnt = (Long)items.get(0).get(0);
+
+                cur.close();
+            }
+            else {
+                qryId = UUID.randomUUID();
+
+                JdbcQueryTask.Cursor jdbcCur = new JdbcQueryTask.Cursor(cur, cur.iterator());
+
+                JdbcQueryTask.addCursor(qryId, jdbcCur);
+
+                if (!loc)
+                    JdbcQueryTask.scheduleRemoval(qryId);
+            }
+
+            JdbcStatementResultInfo resInfo = new JdbcStatementResultInfo(cur.isQuery(), qryId, updCnt);
+
+            resultsInfo.add(resInfo);
+        }
+
+        return resultsInfo;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
index 4854129..ecbfb71 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTask.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.jdbc2;
 
-import java.io.Serializable;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -50,7 +49,7 @@ import org.apache.ignite.resources.IgniteInstanceResource;
  * This parameter can be configured via {@link IgniteSystemProperties#IGNITE_JDBC_DRIVER_CURSOR_REMOVE_DELAY}
  * system property.
  */
-class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
+class JdbcQueryTask implements IgniteCallable<JdbcQueryTaskResult> {
     /** Serial version uid. */
     private static final long serialVersionUID = 0L;
 
@@ -132,7 +131,7 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
     }
 
     /** {@inheritDoc} */
-    @Override public JdbcQueryTask.QueryResult call() throws Exception {
+    @Override public JdbcQueryTaskResult call() throws Exception {
         Cursor cursor = CURSORS.get(uuid);
 
         List<String> tbls = null;
@@ -173,7 +172,11 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
             if (isQry == null)
                 isQry = qryCursor.isQuery();
 
-            Collection<GridQueryFieldMetadata> meta = qryCursor.fieldsMeta();
+            CURSORS.put(uuid, cursor = new Cursor(qryCursor, qryCursor.iterator()));
+        }
+
+        if (first || updateMetadata()) {
+            Collection<GridQueryFieldMetadata> meta = cursor.queryCursor().fieldsMeta();
 
             tbls = new ArrayList<>(meta.size());
             cols = new ArrayList<>(meta.size());
@@ -184,8 +187,6 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
                 cols.add(desc.fieldName().toUpperCase());
                 types.add(desc.fieldTypeName());
             }
-
-            CURSORS.put(uuid, cursor = new Cursor(qryCursor, qryCursor.iterator()));
         }
 
         List<List<?>> rows = new ArrayList<>();
@@ -208,14 +209,14 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
             remove(uuid, cursor);
         else if (first) {
             if (!loc)
-                scheduleRemoval(uuid, RMV_DELAY);
+                scheduleRemoval(uuid);
         }
         else if (!loc && !CURSORS.replace(uuid, cursor, new Cursor(cursor.cursor, cursor.iter)))
             assert !CURSORS.containsKey(uuid) : "Concurrent cursor modification.";
 
         assert isQry != null : "Query flag must be set prior to returning result";
 
-        return new QueryResult(uuid, finished, isQry, rows, cols, tbls, types);
+        return new JdbcQueryTaskResult(uuid, finished, isQry, rows, cols, tbls, types);
     }
 
     /**
@@ -233,14 +234,28 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
     }
 
     /**
+     * @return Flag to update metadata on demand.
+     */
+    protected boolean updateMetadata() {
+        return false;
+    }
+
+    /**
      * Schedules removal of stored cursor in case of remote query execution.
      *
      * @param uuid Cursor UUID.
-     * @param delay Delay in milliseconds.
      */
-    private void scheduleRemoval(final UUID uuid, long delay) {
-        assert !loc;
+    static void scheduleRemoval(final UUID uuid) {
+        scheduleRemoval(uuid, RMV_DELAY);
+    }
 
+    /**
+     * Schedules removal of stored cursor in case of remote query execution.
+     *
+     * @param uuid Cursor UUID.
+     * @param delay Delay in milliseconds.
+     */
+    private static void scheduleRemoval(final UUID uuid, long delay) {
         SCHEDULER.schedule(new CAX() {
             @Override public void applyx() {
                 while (true) {
@@ -279,6 +294,14 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
     }
 
     /**
+     * @param uuid Cursor UUID.
+     * @param c Cursor.
+     */
+    static void addCursor(UUID uuid, Cursor c) {
+        CURSORS.putIfAbsent(uuid, c);
+    }
+
+    /**
      * Closes and removes cursor.
      *
      * @param uuid Cursor UUID.
@@ -291,107 +314,9 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
     }
 
     /**
-     * Result of query execution.
-     */
-    static class QueryResult implements Serializable {
-        /** Serial version uid. */
-        private static final long serialVersionUID = 0L;
-
-        /** Uuid. */
-        private final UUID uuid;
-
-        /** Finished. */
-        private final boolean finished;
-
-        /** Result type - query or update. */
-        private final boolean isQry;
-
-        /** Rows. */
-        private final List<List<?>> rows;
-
-        /** Tables. */
-        private final List<String> tbls;
-
-        /** Columns. */
-        private final List<String> cols;
-
-        /** Types. */
-        private final List<String> types;
-
-        /**
-         * @param uuid UUID..
-         * @param finished Finished.
-         * @param isQry
-         * @param rows Rows.
-         * @param cols Columns.
-         * @param tbls Tables.
-         * @param types Types.
-         */
-        public QueryResult(UUID uuid, boolean finished, boolean isQry, List<List<?>> rows, List<String> cols,
-            List<String> tbls, List<String> types) {
-            this.isQry = isQry;
-            this.cols = cols;
-            this.uuid = uuid;
-            this.finished = finished;
-            this.rows = rows;
-            this.tbls = tbls;
-            this.types = types;
-        }
-
-        /**
-         * @return Query result rows.
-         */
-        public List<List<?>> getRows() {
-            return rows;
-        }
-
-        /**
-         * @return Tables metadata.
-         */
-        public List<String> getTbls() {
-            return tbls;
-        }
-
-        /**
-         * @return Columns metadata.
-         */
-        public List<String> getCols() {
-            return cols;
-        }
-
-        /**
-         * @return Types metadata.
-         */
-        public List<String> getTypes() {
-            return types;
-        }
-
-        /**
-         * @return Query UUID.
-         */
-        public UUID getUuid() {
-            return uuid;
-        }
-
-        /**
-         * @return {@code True} if it is finished query.
-         */
-        public boolean isFinished() {
-            return finished;
-        }
-
-        /**
-         * @return {@code true} if it is result of a query operation, not update; {@code false} otherwise.
-         */
-        public boolean isQuery() {
-            return isQry;
-        }
-    }
-
-    /**
      * Cursor.
      */
-    private static final class Cursor implements Iterable<List<?>> {
+    static final class Cursor implements Iterable<List<?>> {
         /** Cursor. */
         final QueryCursor<List<?>> cursor;
 
@@ -405,7 +330,7 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
          * @param cursor Cursor.
          * @param iter Iterator.
          */
-        private Cursor(QueryCursor<List<?>> cursor, Iterator<List<?>> iter) {
+        Cursor(QueryCursor<List<?>> cursor, Iterator<List<?>> iter) {
             this.cursor = cursor;
             this.iter = iter;
             this.lastAccessTime = U.currentTimeMillis();
@@ -422,5 +347,12 @@ class JdbcQueryTask implements IgniteCallable<JdbcQueryTask.QueryResult> {
         public boolean hasNext() {
             return iter.hasNext();
         }
+
+        /**
+         * @return Cursor.
+         */
+        public QueryCursorImpl<List<?>> queryCursor() {
+            return (QueryCursorImpl<List<?>>)cursor;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskResult.java
new file mode 100644
index 0000000..607bb38
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskResult.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.jdbc2;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * Result of query execution.
+ */
+class JdbcQueryTaskResult implements Serializable {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Uuid. */
+    private final UUID uuid;
+
+    /** Finished. */
+    private final boolean finished;
+
+    /** Result type - query or update. */
+    private final boolean isQry;
+
+    /** Rows. */
+    private final List<List<?>> rows;
+
+    /** Tables. */
+    private final List<String> tbls;
+
+    /** Columns. */
+    private final List<String> cols;
+
+    /** Types. */
+    private final List<String> types;
+
+    /**
+     * @param uuid UUID..
+     * @param finished Finished.
+     * @param isQry Is query flag.
+     * @param rows Rows.
+     * @param cols Columns.
+     * @param tbls Tables.
+     * @param types Types.
+     */
+    public JdbcQueryTaskResult(UUID uuid, boolean finished, boolean isQry, List<List<?>> rows, List<String> cols,
+        List<String> tbls, List<String> types) {
+        this.isQry = isQry;
+        this.cols = cols;
+        this.uuid = uuid;
+        this.finished = finished;
+        this.rows = rows;
+        this.tbls = tbls;
+        this.types = types;
+    }
+
+    /**
+     * @return Query result rows.
+     */
+    public List<List<?>> getRows() {
+        return rows;
+    }
+
+    /**
+     * @return Tables metadata.
+     */
+    public List<String> getTbls() {
+        return tbls;
+    }
+
+    /**
+     * @return Columns metadata.
+     */
+    public List<String> getCols() {
+        return cols;
+    }
+
+    /**
+     * @return Types metadata.
+     */
+    public List<String> getTypes() {
+        return types;
+    }
+
+    /**
+     * @return Query UUID.
+     */
+    public UUID getUuid() {
+        return uuid;
+    }
+
+    /**
+     * @return {@code True} if it is finished query.
+     */
+    public boolean isFinished() {
+        return finished;
+    }
+
+    /**
+     * @return {@code true} if it is result of a query operation, not update; {@code false} otherwise.
+     */
+    public boolean isQuery() {
+        return isQry;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV3.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV3.java
new file mode 100644
index 0000000..cb2d452
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcQueryTaskV3.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.jdbc2;
+
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+
+/**
+ * Task for fetch results of multi-statement query.
+ */
+class JdbcQueryTaskV3 extends JdbcQueryTaskV2 {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Update metadata on demand flag. */
+    private final boolean updateMeta;
+
+    /**
+     * @param ignite Ignite.
+     * @param cacheName Cache name.
+     * @param schemaName Schema name.
+     * @param sql Sql query.
+     * @param isQry Operation type flag - query or not - to enforce query type check.
+     * @param loc Local execution flag.
+     * @param args Args.
+     * @param fetchSize Fetch size.
+     * @param uuid UUID.
+     * @param locQry Local query flag.
+     * @param collocatedQry Collocated query flag.
+     * @param distributedJoins Distributed joins flag.
+     * @param enforceJoinOrder Enforce joins order flag.
+     * @param lazy Lazy query execution flag.
+     * @param updateMeta Update metadata on demand.
+     */
+    public JdbcQueryTaskV3(Ignite ignite, String cacheName, String schemaName, String sql, Boolean isQry, boolean loc,
+        Object[] args, int fetchSize, UUID uuid, boolean locQry, boolean collocatedQry, boolean distributedJoins,
+        boolean enforceJoinOrder, boolean lazy, boolean updateMeta) {
+        super(ignite, cacheName, schemaName, sql, isQry, loc, args, fetchSize, uuid, locQry,
+            collocatedQry, distributedJoins, enforceJoinOrder, lazy);
+
+        this.updateMeta = updateMeta;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean updateMetadata() {
+        return updateMeta;
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @param cacheName Cache name.
+     * @param schemaName Schema name.
+     * @param sql Sql query.
+     * @param isQry Operation type flag - query or not - to enforce query type check.
+     * @param loc Local execution flag.
+     * @param args Args.
+     * @param fetchSize Fetch size.
+     * @param uuid UUID.
+     * @param locQry Local query flag.
+     * @param collocatedQry Collocated query flag.
+     * @param distributedJoins Distributed joins flag.
+     * @param enforceJoinOrder Enforce joins order flag.
+     * @param lazy Lazy query execution flag.
+     * @param updateMeta Update metadata on demand.
+     * @return Appropriate task JdbcQueryTask or JdbcQueryTaskV2.
+     */
+    public static JdbcQueryTask createTask(Ignite ignite, String cacheName, String schemaName, String sql,
+        Boolean isQry, boolean loc, Object[] args, int fetchSize, UUID uuid, boolean locQry,
+        boolean collocatedQry, boolean distributedJoins,
+        boolean enforceJoinOrder, boolean lazy, boolean updateMeta) {
+
+        if (updateMeta)
+            return new JdbcQueryTaskV3(ignite, cacheName, schemaName, sql, isQry, loc, args, fetchSize,
+                uuid, locQry, collocatedQry, distributedJoins, enforceJoinOrder, lazy, true);
+        else
+            return JdbcQueryTaskV2.createTask(ignite, cacheName, schemaName, sql, isQry, loc, args, fetchSize,
+                uuid, locQry, collocatedQry, distributedJoins, enforceJoinOrder, lazy);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
index 04b4041..69d4252 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcResultSet.java
@@ -39,13 +39,14 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Calendar;
-import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.internal.processors.odbc.SqlStateCode;
+import org.apache.ignite.internal.util.typedef.F;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.jdbc2.JdbcUtils.convertToSqlException;
@@ -54,6 +55,12 @@ import static org.apache.ignite.internal.jdbc2.JdbcUtils.convertToSqlException;
  * JDBC result set implementation.
  */
 public class JdbcResultSet implements ResultSet {
+    /** Is query. */
+    private final boolean isQry;
+
+    /** Update count. */
+    private final long updCnt;
+
     /** Uuid. */
     private final UUID uuid;
 
@@ -61,13 +68,13 @@ public class JdbcResultSet implements ResultSet {
     private final JdbcStatement stmt;
 
     /** Table names. */
-    private final List<String> tbls;
+    private List<String> tbls;
 
     /** Column names. */
-    private final List<String> cols;
+    private List<String> cols;
 
     /** Class names. */
-    private final List<String> types;
+    private List<String> types;
 
     /** Rows cursor iterator. */
     private Iterator<List<?>> it;
@@ -93,6 +100,7 @@ public class JdbcResultSet implements ResultSet {
     /**
      * Creates new result set.
      *
+     * @param isQry Is query flag.
      * @param uuid Query UUID.
      * @param stmt Statement.
      * @param tbls Table names.
@@ -100,26 +108,56 @@ public class JdbcResultSet implements ResultSet {
      * @param types Types.
      * @param fields Fields.
      * @param finished Result set finished flag (the last result set).
+     * @throws SQLException On error.
      */
-    JdbcResultSet(@Nullable UUID uuid, JdbcStatement stmt, List<String> tbls, List<String> cols,
-        List<String> types, Collection<List<?>> fields, boolean finished) {
-        assert stmt != null;
-        assert tbls != null;
-        assert cols != null;
-        assert types != null;
-        assert fields != null;
-
-        this.uuid = uuid;
+    JdbcResultSet(boolean isQry, @Nullable UUID uuid, JdbcStatement stmt, List<String> tbls, List<String> cols,
+        List<String> types, List<List<?>> fields, boolean finished) throws SQLException {
+        this.isQry = isQry;
         this.stmt = stmt;
-        this.tbls = tbls;
-        this.cols = cols;
-        this.types = types;
-        this.finished = finished;
 
-        this.it = fields.iterator();
+        if (isQry) {
+            this.uuid = uuid;
+            updCnt = -1;
+            this.tbls = tbls;
+            this.cols = cols;
+            this.types = types;
+            this.finished = finished;
+
+            if (fields != null)
+                it = fields.iterator();
+            else
+                it = Collections.emptyIterator();
+        }
+        else {
+            updCnt = updateCounterFromQueryResult(fields);
+
+            this.uuid = null;
+            this.tbls = null;
+            this.cols = null;
+            this.types = null;
+            this.finished = true;
+            it = null;
+        }
     }
 
-    /** {@inheritDoc} */
+    /**
+     * @param stmt Statement.
+     * @param updCnt Update count.
+     */
+    JdbcResultSet(JdbcStatement stmt, long updCnt) {
+        isQry = false;
+        this.updCnt = updCnt;
+        this.stmt = stmt;
+
+        uuid = null;
+        tbls = null;
+        cols = null;
+        types = null;
+        finished = true;
+        it = null;
+    }
+
+        /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public boolean next() throws SQLException {
         ensureNotClosed();
@@ -140,37 +178,52 @@ public class JdbcResultSet implements ResultSet {
             return true;
         }
         else if (!finished) {
-            JdbcConnection conn = (JdbcConnection)stmt.getConnection();
+            fetchPage();
 
-            Ignite ignite = conn.ignite();
+            return next();
+        }
 
-            UUID nodeId = conn.nodeId();
+        it = null;
 
-            boolean loc = nodeId == null;
+        return false;
+    }
 
-            // Connections from new clients send queries with new tasks, so we have to continue in the same manner
-            JdbcQueryTask qryTask = JdbcQueryTaskV2.createTask(loc ? ignite : null, conn.cacheName(), conn.schemaName(),
-                null,true, loc, null, fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(),
-                    conn.isDistributedJoins(), conn.isEnforceJoinOrder(), conn.isLazy());
+    /**
+     *
+     */
+    private void fetchPage() throws SQLException {
+        JdbcConnection conn = (JdbcConnection)stmt.getConnection();
 
-            try {
-                JdbcQueryTask.QueryResult res =
-                    loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
+        Ignite ignite = conn.ignite();
 
-                finished = res.isFinished();
+        UUID nodeId = conn.nodeId();
 
-                it = res.getRows().iterator();
+        boolean loc = nodeId == null;
 
-                return next();
-            }
-            catch (Exception e) {
-                throw convertToSqlException(e, "Failed to query Ignite.");
-            }
-        }
+        boolean updateMetadata = tbls == null;
 
-        it = null;
+        // Connections from new clients send queries with new tasks, so we have to continue in the same manner
+        JdbcQueryTask qryTask = JdbcQueryTaskV3.createTask(loc ? ignite : null, conn.cacheName(), conn.schemaName(),
+            null,true, loc, null, fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(),
+            conn.isDistributedJoins(), conn.isEnforceJoinOrder(), conn.isLazy(), updateMetadata);
 
-        return false;
+        try {
+            JdbcQueryTaskResult res =
+                loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
+
+            finished = res.isFinished();
+
+            it = res.getRows().iterator();
+
+            if (updateMetadata) {
+                tbls = res.getTbls();
+                cols = res.getCols();
+                types = res.getTypes();
+            }
+        }
+        catch (Exception e) {
+            throw convertToSqlException(e, "Failed to query Ignite.");
+        }
     }
 
     /** {@inheritDoc} */
@@ -421,6 +474,9 @@ public class JdbcResultSet implements ResultSet {
     @Override public ResultSetMetaData getMetaData() throws SQLException {
         ensureNotClosed();
 
+        if (tbls == null)
+            fetchPage();
+
         return new JdbcResultSetMetadata(tbls, cols, types);
     }
 
@@ -1523,4 +1579,43 @@ public class JdbcResultSet implements ResultSet {
         if (curr == null)
             throw new SQLException("Result set is not positioned on a row.");
     }
+
+    /**
+     * @return Is Query flag.
+     */
+    public boolean isQuery() {
+        return isQry;
+    }
+
+    /**
+     * @return Update count.
+     */
+    public long updateCount() {
+        return updCnt;
+    }
+
+    /**
+     * @param rows query result.
+     * @return update counter, if found.
+     * @throws SQLException if getting an update counter from result proved to be impossible.
+     */
+    private static long updateCounterFromQueryResult(List<List<?>> rows) throws SQLException {
+        if (F.isEmpty(rows))
+            return -1;
+
+        if (rows.size() != 1)
+            throw new SQLException("Expected fetch size of 1 for update operation.");
+
+        List<?> row = rows.get(0);
+
+        if (row.size() != 1)
+            throw new SQLException("Expected row size of 1 for update operation.");
+
+        Object objRes = row.get(0);
+
+        if (!(objRes instanceof Long))
+            throw new SQLException("Unexpected update result type.");
+
+        return (Long)objRes;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
index a94b8fd..acac123 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
@@ -24,6 +24,7 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.sql.SQLWarning;
 import java.sql.Statement;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -57,9 +58,6 @@ public class JdbcStatement implements Statement {
     /** Rows limit. */
     private int maxRows;
 
-    /** Current result set. */
-    protected ResultSet rs;
-
     /** Query arguments. */
     protected ArrayList<Object> args;
 
@@ -72,12 +70,15 @@ public class JdbcStatement implements Statement {
     /** Fields indexes. */
     Map<String, Integer> fieldsIdxs = new HashMap<>();
 
-    /** Current updated items count. */
-    long updateCnt = -1;
-
     /** Batch of statements. */
     private List<String> batch;
 
+    /** Results. */
+    protected List<JdbcResultSet> results;
+
+    /** Current result set index. */
+    protected int curRes = 0;
+
     /**
      * Creates new statement.
      *
@@ -92,11 +93,20 @@ public class JdbcStatement implements Statement {
     /** {@inheritDoc} */
     @SuppressWarnings("deprecation")
     @Override public ResultSet executeQuery(String sql) throws SQLException {
-        ensureNotClosed();
+        execute0(sql, true);
 
-        rs = null;
+        return getResultSet();
+    }
 
-        updateCnt = -1;
+    /**
+     * @param sql SQL query.
+     * @param isQuery Expected type of statements are contained in the query.
+     * @throws SQLException On error.
+     */
+    private void executeMultipleStatement(String sql, Boolean isQuery) throws SQLException {
+        ensureNotClosed();
+
+        closeResults();
 
         if (F.isEmpty(sql))
             throw new SQLException("SQL query is empty");
@@ -105,53 +115,37 @@ public class JdbcStatement implements Statement {
 
         UUID nodeId = conn.nodeId();
 
-        UUID uuid = UUID.randomUUID();
-
         boolean loc = nodeId == null;
 
-        JdbcQueryTask qryTask = JdbcQueryTaskV2.createTask(loc ? ignite : null, conn.cacheName(), conn.schemaName(),
-            sql, true, loc, getArgs(), fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(),
+        JdbcQueryMultipleStatementsTask qryTask = new JdbcQueryMultipleStatementsTask(loc ? ignite : null, conn.schemaName(),
+            sql, isQuery, loc, getArgs(), fetchSize, conn.isLocalQuery(), conn.isCollocatedQuery(),
             conn.isDistributedJoins(), conn.isEnforceJoinOrder(), conn.isLazy());
 
         try {
-            JdbcQueryTask.QueryResult res =
+            List<JdbcStatementResultInfo> rsInfos =
                 loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
 
-            JdbcResultSet rs = new JdbcResultSet(uuid, this, res.getTbls(), res.getCols(), res.getTypes(),
-                res.getRows(), res.isFinished());
+            results = new ArrayList<>(rsInfos.size());
 
-            rs.setFetchSize(fetchSize);
-
-            resSets.add(rs);
-
-            return rs;
+            for (JdbcStatementResultInfo rsInfo : rsInfos) {
+                if (rsInfo.isQuery())
+                    results.add(new JdbcResultSet(true, rsInfo.queryId(), this, null, null, null, null, false));
+                else
+                    results.add(new JdbcResultSet(this, rsInfo.updateCount()));
+            }
         }
         catch (Exception e) {
             throw convertToSqlException(e, "Failed to query Ignite.");
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public int executeUpdate(String sql) throws SQLException {
-        ensureNotClosed();
-
-        rs = null;
-
-        updateCnt = -1;
-
-        return Long.valueOf(doUpdate(sql, getArgs())).intValue();
-    }
-
     /**
-     * Run update query.
      * @param sql SQL query.
-     * @param args Update arguments.
-     * @return Number of affected items.
-     * @throws SQLException If failed.
+     * @param isQuery Expected type of statements are contained in the query.
+     * @throws SQLException On error.
      */
-    long doUpdate(String sql, Object[] args) throws SQLException {
-        if (F.isEmpty(sql))
-            throw new SQLException("SQL query is empty");
+    private void executeSingle(String sql, Boolean isQuery) throws SQLException {
+        ensureNotClosed();
 
         Ignite ignite = conn.ignite();
 
@@ -162,46 +156,50 @@ public class JdbcStatement implements Statement {
         boolean loc = nodeId == null;
 
         if (!conn.isDmlSupported())
-            throw new SQLException("Failed to query Ignite: DML operations are supported in versions 1.8.0 and newer");
+            if(isQuery != null && !isQuery)
+                throw new SQLException("Failed to query Ignite: DML operations are supported in versions 1.8.0 and newer");
+            else
+                isQuery = true;
 
         JdbcQueryTask qryTask = JdbcQueryTaskV2.createTask(loc ? ignite : null, conn.cacheName(), conn.schemaName(),
-            sql, false, loc, args, fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(),
+            sql, isQuery, loc, getArgs(), fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(),
             conn.isDistributedJoins(), conn.isEnforceJoinOrder(), conn.isLazy());
 
         try {
-            JdbcQueryTask.QueryResult qryRes =
+            JdbcQueryTaskResult qryRes =
                 loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
 
-            return updateCnt = updateCounterFromQueryResult(qryRes.getRows());
+            JdbcResultSet rs = new JdbcResultSet(qryRes.isQuery(), uuid, this, qryRes.getTbls(), qryRes.getCols(),
+                qryRes.getTypes(), qryRes.getRows(), qryRes.isFinished());
+
+            rs.setFetchSize(fetchSize);
+
+            results = Collections.singletonList(rs);
+            curRes = 0;
         }
         catch (Exception e) {
             throw convertToSqlException(e, "Failed to query Ignite.");
         }
+
     }
 
     /**
-     * @param rows query result.
-     * @return update counter, if found.
-     * @throws SQLException if getting an update counter from result proved to be impossible.
+     * @param sql SQL query.
+     * @param isQuery Expected type of statements are contained in the query.
+     * @throws SQLException On error.
      */
-    private static long updateCounterFromQueryResult(List<List<?>> rows) throws SQLException {
-         if (F.isEmpty(rows))
-            return -1;
-
-        if (rows.size() != 1)
-            throw new SQLException("Expected fetch size of 1 for update operation");
-
-        List<?> row = rows.get(0);
-
-        if (row.size() != 1)
-            throw new SQLException("Expected row size of 1 for update operation");
-
-        Object objRes = row.get(0);
+    protected void execute0(String sql, Boolean isQuery) throws SQLException {
+        if (conn.isMultipleStatementsAllowed())
+            executeMultipleStatement(sql, isQuery);
+        else
+            executeSingle(sql, isQuery);
+    }
 
-        if (!(objRes instanceof Long))
-            throw new SQLException("Unexpected update result type");
+    /** {@inheritDoc} */
+    @Override public int executeUpdate(String sql) throws SQLException {
+        execute0(sql, false);
 
-        return (Long)objRes;
+        return getUpdateCount();
     }
 
     /** {@inheritDoc} */
@@ -302,86 +300,48 @@ public class JdbcStatement implements Statement {
 
     /** {@inheritDoc} */
     @Override public boolean execute(String sql) throws SQLException {
-        if (!conn.isDmlSupported()) {
-            // We attempt to run a query without any checks as long as server does not support DML anyway,
-            // so it simply will throw an exception when given a DML statement instead of a query.
-            rs = executeQuery(sql);
-
-            return true;
-        }
-
-        ensureNotClosed();
-
-        rs = null;
-
-        updateCnt = -1;
-
-        if (F.isEmpty(sql))
-            throw new SQLException("SQL query is empty");
-
-        Ignite ignite = conn.ignite();
+        execute0(sql, null);
 
-        UUID nodeId = conn.nodeId();
-
-        UUID uuid = UUID.randomUUID();
-
-        boolean loc = nodeId == null;
-
-        JdbcQueryTask qryTask = JdbcQueryTaskV2.createTask(loc ? ignite : null, conn.cacheName(), conn.schemaName(),
-            sql, null, loc, getArgs(), fetchSize, uuid, conn.isLocalQuery(), conn.isCollocatedQuery(),
-            conn.isDistributedJoins(), conn.isEnforceJoinOrder(), conn.isLazy());
-
-        try {
-            JdbcQueryTask.QueryResult res =
-                loc ? qryTask.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(qryTask);
-
-            if (res.isQuery()) {
-                JdbcResultSet rs = new JdbcResultSet(uuid, this, res.getTbls(), res.getCols(),
-                    res.getTypes(), res.getRows(), res.isFinished());
-
-                rs.setFetchSize(fetchSize);
-
-                resSets.add(rs);
-
-                this.rs = rs;
-            }
-            else
-                updateCnt = updateCounterFromQueryResult(res.getRows());
-
-            return res.isQuery();
-        }
-        catch (Exception e) {
-            throw convertToSqlException(e, "Failed to query Ignite.");
-        }
+        return results.get(0).isQuery();
     }
 
     /** {@inheritDoc} */
     @Override public ResultSet getResultSet() throws SQLException {
-        ensureNotClosed();
+        JdbcResultSet rs = nextResultSet();
 
-        ResultSet rs0 = rs;
+        if (rs == null)
+            return null;
 
-        rs = null;
+        if (!rs.isQuery()) {
+            curRes--;
 
-        return rs0;
+            return null;
+        }
+
+        return rs;
     }
 
     /** {@inheritDoc} */
     @Override public int getUpdateCount() throws SQLException {
-        ensureNotClosed();
+        JdbcResultSet rs = nextResultSet();
 
-        long res = updateCnt;
+        if (rs == null)
+            return -1;
 
-        updateCnt = -1;
+        if (rs.isQuery()) {
+            curRes--;
+
+            return -1;
+        }
 
-        return Long.valueOf(res).intValue();
+        return (int)rs.updateCount();
     }
 
     /** {@inheritDoc} */
     @Override public boolean getMoreResults() throws SQLException {
         ensureNotClosed();
 
-        return false;
+        return getMoreResults(CLOSE_CURRENT_RESULT);
     }
 
     /** {@inheritDoc} */
@@ -472,9 +432,8 @@ public class JdbcStatement implements Statement {
      */
     protected int[] doBatchUpdate(String command, List<String> batch, List<List<Object>> batchArgs)
         throws SQLException {
-        rs = null;
 
-        updateCnt = -1;
+        closeResults();
 
         if ((F.isEmpty(command) || F.isEmpty(batchArgs)) && F.isEmpty(batch))
             throw new SQLException("Batch is empty.");
@@ -495,7 +454,11 @@ public class JdbcStatement implements Statement {
         try {
             int[] res = loc ? task.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(task);
 
-            updateCnt = F.isEmpty(res)? -1 : res[res.length - 1];
+            long updateCnt = F.isEmpty(res)? -1 : res[res.length - 1];
+
+            results = Collections.singletonList(new JdbcResultSet(this, updateCnt));
+
+            curRes = 0;
 
             return res;
         }
@@ -515,10 +478,32 @@ public class JdbcStatement implements Statement {
     @Override public boolean getMoreResults(int curr) throws SQLException {
         ensureNotClosed();
 
-        if (curr == KEEP_CURRENT_RESULT || curr == CLOSE_ALL_RESULTS)
-            throw new SQLFeatureNotSupportedException("Multiple open results are not supported.");
+        if (results != null) {
+            assert curRes <= results.size() : "Invalid results state: [resultsCount=" + results.size() +
+                ", curRes=" + curRes + ']';
 
-        return false;
+            switch (curr) {
+                case CLOSE_CURRENT_RESULT:
+                    if (curRes > 0)
+                        results.get(curRes - 1).close();
+
+                    break;
+
+                case CLOSE_ALL_RESULTS:
+                    for (int i = 0; i < curRes; ++i)
+                        results.get(i).close();
+
+                    break;
+
+                case KEEP_CURRENT_RESULT:
+                    break;
+
+                default:
+                    throw new SQLException("Invalid 'current' parameter.");
+            }
+        }
+
+        return (results != null && curRes < results.size());
     }
 
     /** {@inheritDoc} */
@@ -657,4 +642,35 @@ public class JdbcStatement implements Statement {
         if (closed)
             throw new SQLException("Connection is closed.", SqlStateCode.CONNECTION_CLOSED);
     }
+
+    /**
+     * Get last result set if any.
+     *
+     * @return Result set or null.
+     * @throws SQLException If failed.
+     */
+    private JdbcResultSet nextResultSet() throws SQLException {
+        ensureNotClosed();
+
+        if (results == null || curRes >= results.size())
+            return null;
+        else
+            return results.get(curRes++);
+    }
+
+    /**
+     * Close results.
+     *
+     * @throws SQLException On error.
+     */
+    private void closeResults() throws SQLException {
+        if (results != null) {
+            for (JdbcResultSet rs : results)
+                rs.close();
+
+            results = null;
+            curRes = 0;
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatementResultInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatementResultInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatementResultInfo.java
new file mode 100644
index 0000000..8aa02f1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatementResultInfo.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.jdbc2;
+
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC statement result information. Keeps statement type (SELECT or UPDATE) and
+ * queryId or update count (depends on statement type).
+ */
+public class JdbcStatementResultInfo {
+    /** Query flag. */
+    private boolean isQuery;
+
+    /** Update count. */
+    private long updCnt;
+
+    /** Query ID. */
+    private UUID qryId;
+
+    /**
+     * @param isQuery Query flag.
+     * @param qryId Query ID.
+     * @param updCnt Update count.
+     */
+    public JdbcStatementResultInfo(boolean isQuery, UUID qryId, long updCnt) {
+        this.isQuery = isQuery;
+        this.updCnt = updCnt;
+        this.qryId = qryId;
+    }
+
+    /**
+     * @return Query flag.
+     */
+    public boolean isQuery() {
+        return isQuery;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public UUID queryId() {
+        return qryId;
+    }
+
+    /**
+     * @return Update count.
+     */
+    public long updateCount() {
+        return updCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcStatementResultInfo.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c407f/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java
index 9f76700..408f089 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStreamedPreparedStatement.java
@@ -19,8 +19,8 @@ package org.apache.ignite.internal.jdbc2;
 
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
+import java.util.Collections;
 import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.internal.IgniteEx;
 
 /**
  * Prepared statement associated with a data streamer.
@@ -33,8 +33,9 @@ class JdbcStreamedPreparedStatement extends JdbcPreparedStatement {
      * Creates new prepared statement.
      *
      * @param conn Connection.
-     * @param sql  SQL query.
+     * @param sql SQL query.
      * @param streamer Data streamer to use with this statement. Will be closed on statement close.
+     * @param nativeStmt Native statement.
      */
     JdbcStreamedPreparedStatement(JdbcConnection conn, String sql, IgniteDataStreamer<?, ?> streamer,
         PreparedStatement nativeStmt) {
@@ -53,8 +54,16 @@ class JdbcStreamedPreparedStatement extends JdbcPreparedStatement {
     }
 
     /** {@inheritDoc} */
-    @Override long doUpdate(String sql, Object[] args) throws SQLException {
-        return conn.ignite().context().query().streamUpdateQuery(conn.cacheName(), conn.schemaName(),
-            streamer, sql, args);
+    @Override protected void execute0(String sql, Boolean isQuery) throws SQLException {
+        assert isQuery != null && !isQuery;
+
+        long updCnt = conn.ignite().context().query().streamUpdateQuery(conn.cacheName(), conn.schemaName(),
+            streamer, sql, getArgs());
+
+        JdbcResultSet rs = new JdbcResultSet(this, updCnt);
+
+        results = Collections.singletonList(rs);
+
+        curRes = 0;
     }
 }


[16/29] ignite git commit: IGNITE-5224 .NET: PadLeft and PadRight support in LINQ

Posted by sb...@apache.org.
IGNITE-5224 .NET: PadLeft and PadRight support in LINQ

This closes #2808


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

Branch: refs/heads/ignite-5937
Commit: e5c25b444ae605f0b76289f5c8466c850a26e344
Parents: 2410f07
Author: Alexey Popov <ta...@gmail.com>
Authored: Fri Oct 6 12:18:38 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Oct 6 12:18:38 2017 +0300

----------------------------------------------------------------------
 .../Cache/Query/Linq/CacheLinqTest.Strings.cs                   | 5 +++++
 .../platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs   | 4 ++++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e5c25b44/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
index 7457d0a..1139c4d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Strings.cs
@@ -46,6 +46,11 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
         {
             var strings = GetSecondPersonCache().AsCacheQueryable().Select(x => x.Value.Name);
 
+            CheckFunc(x => x.PadLeft(20), strings);
+            CheckFunc(x => x.PadLeft(20, 'l'), strings);
+            CheckFunc(x => x.PadRight(20), strings);
+            CheckFunc(x => x.PadRight(20, 'r'), strings);
+
             CheckFunc(x => x.ToLower(), strings);
             CheckFunc(x => x.ToUpper(), strings);
             CheckFunc(x => x.StartsWith("Person_9"), strings);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e5c25b44/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
index 9446af3..8abf2a6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/MethodVisitor.cs
@@ -66,6 +66,10 @@ namespace Apache.Ignite.Linq.Impl
             GetParameterizedTrimMethod("TrimStart", "ltrim"),
             GetParameterizedTrimMethod("TrimEnd", "rtrim"),
             GetStringMethod("Replace", "replace", typeof(string), typeof(string)),
+            GetStringMethod("PadLeft", "lpad", typeof (int)),
+            GetStringMethod("PadLeft", "lpad", typeof (int), typeof (char)),
+            GetStringMethod("PadRight", "rpad", typeof (int)),
+            GetStringMethod("PadRight", "rpad", typeof (int), typeof (char)),
 
             GetMethod(typeof (Regex), "Replace", new[] {typeof (string), typeof (string), typeof (string)}, 
                 GetFunc("regexp_replace")),