IGNITE-21835 MVCC removal: final cleanup (#11330)

diff --git a/docs/_docs/setup.adoc b/docs/_docs/setup.adoc
index a54918b..cdb2040 100644
--- a/docs/_docs/setup.adoc
+++ b/docs/_docs/setup.adoc
@@ -326,7 +326,7 @@
 ignite.sh -systemProps
 IGNITE_AFFINITY_HISTORY_SIZE                                    - [Integer] Maximum size for affinity assignment history. Default is 25.
 IGNITE_ALLOW_ATOMIC_OPS_IN_TX                                   - [Boolean] Allows atomic operations inside transactions. Default is true.
-IGNITE_ALLOW_DML_INSIDE_TRANSACTION                             - [Boolean] When set to true, Ignite will allow executing DML operation (MERGE|INSERT|UPDATE|DELETE) within transactions for non MVCC mode. Default is false.
+IGNITE_ALLOW_DML_INSIDE_TRANSACTION                             - [Boolean] When set to true, Ignite will allow executing DML operation (MERGE|INSERT|UPDATE|DELETE) within transactions. Default is false.
 IGNITE_ALLOW_START_CACHES_IN_PARALLEL                           - [Boolean] Allows to start multiple caches in parallel. Default is true.
 ...
 
diff --git a/modules/core/src/main/java/org/apache/ignite/transactions/TransactionUnsupportedConcurrencyException.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionCalciteSelfTest.java
similarity index 62%
copy from modules/core/src/main/java/org/apache/ignite/transactions/TransactionUnsupportedConcurrencyException.java
copy to modules/calcite/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionCalciteSelfTest.java
index c8ce9bc..f885972 100644
--- a/modules/core/src/main/java/org/apache/ignite/transactions/TransactionUnsupportedConcurrencyException.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionCalciteSelfTest.java
@@ -15,21 +15,16 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.transactions;
+package org.apache.ignite.internal.processors.cache;
 
-/**
- * Exception thrown whenever transaction concurrency level is not supported.
- */
-public class TransactionUnsupportedConcurrencyException extends TransactionException {
-    /** */
-    private static final long serialVersionUID = 0L;
+import org.apache.ignite.calcite.CalciteQueryEngineConfiguration;
+import org.apache.ignite.configuration.QueryEngineConfiguration;
 
-    /**
-     * Creates new exception with given error message.
-     *
-     * @param msg Error message.
-     */
-    public TransactionUnsupportedConcurrencyException(String msg) {
-        super(msg);
+/** */
+public class DdlTransactionCalciteSelfTest extends DdlTransactionSelfTestBase {
+    /** {@inheritDoc} */
+    @Override protected QueryEngineConfiguration getQueryEngineConfiguration() {
+        return new CalciteQueryEngineConfiguration()
+            .setDefault(true);
     }
 }
diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IntegrationTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IntegrationTestSuite.java
index 5ab4bd4..fa1ca91 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IntegrationTestSuite.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IntegrationTestSuite.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.testsuites;
 
+import org.apache.ignite.internal.processors.cache.DdlTransactionCalciteSelfTest;
 import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessorTest;
 import org.apache.ignite.internal.processors.query.calcite.CancelTest;
 import org.apache.ignite.internal.processors.query.calcite.DateTimeTest;
@@ -133,6 +134,7 @@
     JoinRehashIntegrationTest.class,
     IndexWithSameNameCalciteTest.class,
     AuthorizationIntegrationTest.class,
+    DdlTransactionCalciteSelfTest.class,
 })
 public class IntegrationTestSuite {
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index f13f328..d8b89ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -1554,12 +1554,12 @@
 
     /**
      * When set to {@code true}, Ignite will allow execute DML operation (MERGE|INSERT|UPDATE|DELETE)
-     * within transaction for non MVCC mode.
+     * within transaction.
      *
      * Default is {@code false}.
      */
     @SystemProperty("When set to true, Ignite will allow executing DML operation " +
-        "(MERGE|INSERT|UPDATE|DELETE) within transactions for non MVCC mode")
+        "(MERGE|INSERT|UPDATE|DELETE) within transactions")
     public static final String IGNITE_ALLOW_DML_INSIDE_TRANSACTION = "IGNITE_ALLOW_DML_INSIDE_TRANSACTION";
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
index b38c939..fb37779 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java
@@ -126,9 +126,6 @@
     /** Force rebuild, list or request indexes rebuild status from control script. */
     INDEXES_MANIPULATIONS_FROM_CONTROL_SCRIPT(42),
 
-    /** Optimization of recovery protocol for cluster which doesn't contain MVCC caches. */
-    MVCC_TX_RECOVERY_PROTOCOL_V2(44),
-
     /** Pk index keys are applied in correct order. */
     SPECIFIED_SEQ_PK_KEYS(45),
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
index 4f14bf8..a0a339c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
@@ -490,7 +490,6 @@
      * @param pageMemory Page memory to work with.
      * @param removeId Global remove id.
      * @param reuseList Reuse list where free pages should be stored.
-     * @param mvccEnabled Whether mvcc is enabled.
      * @throws IgniteCheckedException If failed.
      */
     public void destroyOrphanIndex(
@@ -500,8 +499,7 @@
         int grpId,
         PageMemory pageMemory,
         GridAtomicLong removeId,
-        ReuseList reuseList,
-        boolean mvccEnabled) throws IgniteCheckedException {
+        ReuseList reuseList) throws IgniteCheckedException {
 
         assert ctx.cache().context().database().checkpointLockIsHeldByThread();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java
index 00c76ba..3b111d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/WalRecordsConsumer.java
@@ -32,7 +32,7 @@
 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.UnwrappedDataEntry;
+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.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
@@ -91,7 +91,7 @@
 
     /** Event transformer. */
     static final IgniteClosure<DataEntry, CdcEvent> CDC_EVENT_TRANSFORMER = e -> {
-        UnwrappedDataEntry ue = (UnwrappedDataEntry)e;
+        UnwrapDataEntry ue = (UnwrapDataEntry)e;
 
         return new CdcEventImpl(
             ue.unwrappedKey(),
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
index a87a46a..9d0f447 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/store/IgnitePageStoreManager.java
@@ -25,7 +25,6 @@
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManager;
-import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageMetrics;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PageReadWriteManager;
 import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport;
 
@@ -44,18 +43,6 @@
     public void finishRecover() throws IgniteCheckedException;
 
     /**
-     * Initializes disk store structures.
-     *
-     * @param cacheId Cache id.
-     * @param partitions Partitions count.
-     * @param cacheName Cache name.
-     * @param pageMetrics Page metrics.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void initialize(int cacheId, int partitions, String cacheName, PageMetrics pageMetrics)
-        throws IgniteCheckedException;
-
-    /**
      * Callback called when a cache is starting.
      *
      * @param grpDesc Cache group descriptor.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java
index 45bd54a..5819496 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/LazyDataEntry.java
@@ -31,7 +31,7 @@
  * Represents Data Entry ({@link #key}, {@link #val value}) pair update {@link #op operation}. <br>
  * This Data entry was not converted to key, value pair during record deserialization.
  */
-public class LazyDataEntry extends DataEntry implements MarshalledDataEntry {
+public class LazyDataEntry extends DataEntry {
     /** */
     private GridCacheSharedContext cctx;
 
@@ -126,23 +126,23 @@
         return val;
     }
 
-    /** {@inheritDoc} */
-    @Override public byte getKeyType() {
+    /** @return Data Entry Key type code. See {@link CacheObject} for built-in value type codes */
+    public byte getKeyType() {
         return keyType;
     }
 
-    /** {@inheritDoc} */
-    @Override public byte[] getKeyBytes() {
+    /** @return Key value bytes. */
+    public byte[] getKeyBytes() {
         return keyBytes;
     }
 
-    /** {@inheritDoc} */
-    @Override public byte getValType() {
+    /** @return Data Entry Value type code. See {@link CacheObject} for built-in value type codes */
+    public byte getValType() {
         return valType;
     }
 
-    /** {@inheritDoc} */
-    @Override public byte[] getValBytes() {
+    /** @return Value value bytes. */
+    public byte[] getValBytes() {
         return valBytes;
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MarshalledDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MarshalledDataEntry.java
deleted file mode 100644
index c977d52..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/MarshalledDataEntry.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.processors.cache.CacheObject;
-
-/**
- * Interface for Data Entry record that was not converted to key, value pair during record deserialization.
- */
-public interface MarshalledDataEntry {
-    /**
-     * @return Data Entry Key type code. See {@link CacheObject} for built-in value type codes.
-     */
-    byte getKeyType();
-
-    /**
-     * @return Key value bytes.
-     */
-    byte[] getKeyBytes();
-
-    /**
-     * @return Data Entry Value type code. See {@link CacheObject} for built-in value type codes.
-     */
-    byte getValType();
-
-    /**
-     * @return Value value bytes.
-     */
-    byte[] getValBytes();
-}
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 cba9f37..e535e22 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
@@ -29,7 +29,7 @@
 /**
  * Data Entry for automatic unwrapping key and value from Data Entry
  */
-public class UnwrapDataEntry extends DataEntry implements UnwrappedDataEntry {
+public class UnwrapDataEntry extends DataEntry {
     /** Cache object value context. Context is used for unwrapping objects. */
     private final CacheObjectValueContext cacheObjValCtx;
 
@@ -68,8 +68,13 @@
         this.keepBinary = keepBinary;
     }
 
-    /** {@inheritDoc} */
-    @Override public Object unwrappedKey() {
+    /**
+     * Unwraps key 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.
+     *
+     * @return Key which was placed into cache. Or null if failed to convert.
+     */
+    public Object unwrappedKey() {
         try {
             return unwrapKey(key, keepBinary, cacheObjValCtx);
         }
@@ -81,8 +86,13 @@
         }
     }
 
-    /** {@inheritDoc} */
-    @Override public Object unwrappedValue() {
+    /**
+     * Unwraps 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.
+     *
+     * @return Value which was placed into cache. Or null for delete operation or for failure.
+     */
+    public Object unwrappedValue() {
         try {
             return unwrapValue(val, keepBinary, cacheObjValCtx);
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrappedDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrappedDataEntry.java
deleted file mode 100644
index b3a20b9..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrappedDataEntry.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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;
-
-/**
- * Interface for Data Entry for automatic unwrapping key and value from Data Entry
- */
-public interface UnwrappedDataEntry {
-    /**
-     * 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.
-     *
-     * @return Key which was placed into cache. Or null if failed to convert.
-     */
-    Object unwrappedKey();
-
-    /**
-     * 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.
-     *
-     * @return Value which was placed into cache. Or null for delete operation or for failure.
-     */
-    Object unwrappedValue();
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
index b6da77f..fab0ac2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEntryInfoCollection.java
@@ -43,13 +43,6 @@
     }
 
     /**
-     * @param infos List of cache entry info.
-     */
-    public CacheEntryInfoCollection(List<GridCacheEntryInfo> infos) {
-        this.infos = infos;
-    }
-
-    /**
      *
      */
     public void init() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
index 1f4f0da..6bafa74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
@@ -1029,24 +1029,6 @@
     }
 
     /**
-     * @return {@code True} if there is at least one cache with registered CQ exists in this group.
-     */
-    public boolean hasContinuousQueryCaches() {
-        List<GridCacheContext<?, ?>> contQryCaches;
-
-        listenerLock.readLock().lock();
-
-        try {
-            contQryCaches = this.contQryCaches;
-
-            return !F.isEmpty(contQryCaches);
-        }
-        finally {
-            listenerLock.readLock().unlock();
-        }
-    }
-
-    /**
      * @throws IgniteCheckedException If failed.
      */
     public void start() throws IgniteCheckedException {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java
index cb79a55..e7dbbad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheInvokeEntry.java
@@ -117,15 +117,6 @@
     }
 
     /**
-     * Entry processor operation.
-     *
-     * @return Operation.
-     */
-    public Operation op() {
-        return op;
-    }
-
-    /**
      * @return Return origin value, before modification.
      */
     public V oldVal() {
@@ -167,7 +158,7 @@
     /**
      *
      */
-    public static enum Operation {
+    private static enum Operation {
         /** */
         NONE,
 
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 ba44189..70032fc 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
@@ -1783,11 +1783,6 @@
         final boolean needVer
     );
 
-    /** */
-    protected GridNearTxLocal checkCurrentTx() {
-        return ctx.tm().threadLocalTx(ctx);
-    }
-
     /** {@inheritDoc} */
     @Override public final V getAndPut(K key, V val) throws IgniteCheckedException {
         return getAndPut(key, val, null);
@@ -3603,7 +3598,7 @@
     @Nullable private <T> T syncOp(SyncOp<T> op) throws IgniteCheckedException {
         checkJta();
 
-        GridNearTxLocal tx = checkCurrentTx();
+        GridNearTxLocal tx = ctx.tm().threadLocalTx(ctx);
 
         if (tx == null || tx.implicit()) {
             lastAsyncFuture().await();
@@ -3723,7 +3718,7 @@
         if (log.isDebugEnabled())
             log.debug("Performing async op: " + op);
 
-        GridNearTxLocal tx = checkCurrentTx();
+        GridNearTxLocal tx = ctx.tm().threadLocalTx(ctx);
 
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
@@ -4241,7 +4236,7 @@
         IgniteInternalFuture<R> orig,
         Function<IgniteConsistencyViolationException, IgniteInternalFuture<Void>> repair,
         Supplier<IgniteInternalFuture<R>> retry) {
-        final GridNearTxLocal tx = checkCurrentTx();
+        final GridNearTxLocal tx = ctx.tm().threadLocalTx(ctx);
         final CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         GridFutureAdapter<R> fut = new GridFutureAdapter<>();
@@ -4320,7 +4315,7 @@
         boolean skipVals) {
         assert ctx.transactional();
 
-        final GridNearTxLocal orig = checkCurrentTx();
+        final GridNearTxLocal orig = ctx.tm().threadLocalTx(ctx);
 
         assert orig == null || orig.optimistic() || orig.readCommitted() || /*contains*/ skipVals :
             "Pessimistic non-read-committed 'get' should be fixed inside its own tx, the only exception is 'contains' " +
@@ -4336,7 +4331,7 @@
                 try (Transaction tx = ctx.grid().transactions().txStart(PESSIMISTIC, SERIALIZABLE)) {
                     get((K)key, null, !ctx.keepBinary(), false); // Repair.
 
-                    final GridNearTxLocal tx0 = checkCurrentTx();
+                    final GridNearTxLocal tx0 = ctx.tm().threadLocalTx(ctx);
 
                     final IgniteTxKey txKey = ctx.txKey(ctx.toCacheKeyObject(key));
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 2c15eee..fbf11bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -83,7 +83,6 @@
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
 import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
@@ -2289,15 +2288,6 @@
     }
 
     /**
-     * @param tx Transaction.
-     * @return {@code True} if it is need to notify continuous query listeners.
-     */
-    public boolean hasContinuousQueryListeners(@Nullable IgniteInternalTx tx) {
-        return grp.sharedGroup() ? grp.hasContinuousQueryCaches() :
-            contQryMgr.notifyContinuousQueries(tx) && !F.isEmpty(contQryMgr.updateListeners(false, false));
-    }
-
-    /**
      * Apply changes from {@link SchemaAddQueryEntityOperation}.
      *
      * @param op Add query entity schema operation.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 0a12ac3..35936cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -32,7 +32,6 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.LongAdder;
 import javax.cache.Cache;
 import javax.cache.CacheException;
 import javax.cache.configuration.Factory;
@@ -565,30 +564,6 @@
     }
 
     /**
-     * @return Long reducer.
-     */
-    public static IgniteReducer<Long, Long> longReducer() {
-        return new IgniteReducer<Long, Long>() {
-            private final LongAdder res = new LongAdder();
-
-            @Override public boolean collect(Long l) {
-                if (l != null)
-                    res.add(l);
-
-                return true;
-            }
-
-            @Override public Long reduce() {
-                return res.sum();
-            }
-
-            @Override public String toString() {
-                return "Long reducer: " + res;
-            }
-        };
-    }
-
-    /**
      * Gets reducer that aggregates maps into one.
      *
      * @param size Predicted size of the resulting map to avoid resizings.
@@ -2098,25 +2073,6 @@
      * @param cacheIds Cache ids.
      * @return First partitioned cache or {@code null} in case no partitioned cache ids are in list.
      */
-    public static GridCacheContext<?, ?> firstPartitioned(GridCacheSharedContext<?, ?> sctx, int[] cacheIds) {
-        for (int i = 0; i < cacheIds.length; i++) {
-            GridCacheContext<?, ?> cctx = sctx.cacheContext(cacheIds[i]);
-
-            if (cctx == null)
-                throw new CacheException("Failed to find cache.");
-
-            if (!cctx.isReplicated())
-                return cctx;
-        }
-
-        return null;
-    }
-
-    /**
-     * @param sctx Shared context.
-     * @param cacheIds Cache ids.
-     * @return First partitioned cache or {@code null} in case no partitioned cache ids are in list.
-     */
     public static GridCacheContext<?, ?> firstPartitioned(GridCacheSharedContext<?, ?> sctx, Iterable<Integer> cacheIds) {
         for (Integer i : cacheIds) {
             GridCacheContext<?, ?> cctx = sctx.cacheContext(i);
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 edfe8d0..f1b0f21 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
@@ -913,7 +913,7 @@
 
             @Override protected boolean onHasNext() throws IgniteCheckedException {
                 if (cur == null)
-                    cur = data.cursor(CacheDataRowAdapter.RowData.FULL_WITH_HINTS);
+                    cur = data.cursor();
 
                 if (next != null)
                     return true;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterMvccImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterMvccImpl.java
deleted file mode 100644
index 5c1b284..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/PartitionUpdateCounterMvccImpl.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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;
-
-import java.util.TreeMap;
-
-/**
- * Update counter implementation for MVCC mode.
- */
-public class PartitionUpdateCounterMvccImpl extends PartitionUpdateCounterTrackingImpl {
-    /**
-     * @param grp Group.
-     */
-    public PartitionUpdateCounterMvccImpl(CacheGroupContext grp) {
-        super(grp);
-    }
-
-    /** {@inheritDoc} */
-    @Override public long reserve(long delta) {
-        return next(delta);
-    }
-
-    /** {@inheritDoc} */
-    @Override public long reserved() {
-        return get();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected PartitionUpdateCounterTrackingImpl createInstance() {
-        return new PartitionUpdateCounterMvccImpl(grp);
-    }
-
-    /** {@inheritDoc} */
-    @Override public PartitionUpdateCounter copy() {
-        PartitionUpdateCounterMvccImpl copy = new PartitionUpdateCounterMvccImpl(grp);
-
-        copy.lwm.set(lwm.get());
-        copy.first = first;
-        copy.queue = new TreeMap<>(queue);
-        copy.initCntr = initCntr;
-        copy.reservedCntr.set(reservedCntr.get());
-
-        return copy;
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
index 44fc036..a5f7807 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
@@ -18,13 +18,8 @@
 package org.apache.ignite.internal.processors.cache.distributed;
 
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -40,17 +35,10 @@
  * Transaction node mapping.
  */
 public class GridDistributedTxMapping {
-    /** */
-    private static final AtomicReferenceFieldUpdater<GridDistributedTxMapping, Set> BACKUPS_FIELD_UPDATER
-        = AtomicReferenceFieldUpdater.newUpdater(GridDistributedTxMapping.class, Set.class, "backups");
-
     /** Mapped node. */
     @GridToStringExclude
     private final ClusterNode primary;
 
-    /** Mapped backup nodes. */
-    private volatile Set<UUID> backups;
-
     /** Entries. */
     @GridToStringInclude
     private final Collection<IgniteTxEntry> entries;
@@ -58,9 +46,6 @@
     /** Explicit lock flag. */
     private boolean explicitLock;
 
-    /** Query update flag. */
-    private boolean queryUpdate;
-
     /** DHT version. */
     private GridCacheVersion dhtVer;
 
@@ -148,20 +133,6 @@
     }
 
     /**
-     * @return {@code True} if mapping was created for a query update.
-     */
-    public boolean queryUpdate() {
-        return queryUpdate;
-    }
-
-    /**
-     * Sets query update flag to {@code true}.
-     */
-    public void markQueryUpdate() {
-        queryUpdate = true;
-    }
-
-    /**
      * @return {@code True} if lock is explicit.
      */
     public boolean explicitLock() {
@@ -294,26 +265,6 @@
         return entries.isEmpty();
     }
 
-    /**
-     * @param newBackups Backups to be added to this mapping.
-     */
-    public void addBackups(Collection<UUID> newBackups) {
-        if (newBackups == null)
-            return;
-
-        if (backups == null)
-            BACKUPS_FIELD_UPDATER.compareAndSet(this, null, Collections.newSetFromMap(new ConcurrentHashMap<>()));
-
-        backups.addAll(newBackups);
-    }
-
-    /**
-     * @return Mapped backup nodes.
-     */
-    public Set<UUID> backups() {
-        return backups != null ? backups : Collections.emptySet();
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridDistributedTxMapping.class, this, "node", primary.id());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
index 96eeee2..2d6da9c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
@@ -78,9 +78,6 @@
     /** */
     public static final int STORE_WRITE_THROUGH_FLAG_MASK = 0x20;
 
-    /** */
-    public static final int QUERY_UPDATE_FLAG_MASK = 0x40;
-
     /** Collection to message converter. */
     private static final C1<Collection<UUID>, UUIDCollectionMessage> COL_TO_MSG = new C1<Collection<UUID>, UUIDCollectionMessage>() {
         @Override public UUIDCollectionMessage apply(Collection<UUID> uuids) {
@@ -256,25 +253,6 @@
     }
 
     /**
-     *
-     * @return Flag indicating whether it is a query update.
-     */
-    public boolean queryUpdate() {
-        return (flags & QUERY_UPDATE_FLAG_MASK) != 0;
-    }
-
-    /**
-     *
-     * @param queryUpdate Query update value.
-     */
-    public void queryUpdate(boolean queryUpdate) {
-        if (queryUpdate)
-            flags = (byte)(flags | QUERY_UPDATE_FLAG_MASK);
-        else
-            flags &= ~QUERY_UPDATE_FLAG_MASK;
-    }
-
-    /**
      * @return IO policy.
      */
     public byte policy() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/DhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/DhtLockFuture.java
deleted file mode 100644
index b729dcd..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/DhtLockFuture.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.distributed.dht;
-
-import org.apache.ignite.internal.IgniteInternalFuture;
-
-/**
- * Marker interface.
- */
-public interface DhtLockFuture<T> extends IgniteInternalFuture<T> {
-    /**
-     * @param error Error.
-     */
-    public void onError(Throwable error);
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 19bbb68..b47447f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -1565,7 +1565,6 @@
      * @return {@code True} if cache affinity changed and operation should be remapped.
      */
     protected final boolean needRemap(AffinityTopologyVersion rmtVer) {
-        // TODO IGNITE-7164 check mvcc crd for mvcc enabled txs.
         return !ctx.affinity().isCompatibleWithCurrentTopologyVersion(rmtVer);
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 3f2e813..f5ed1f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -85,7 +85,7 @@
  * Cache lock future.
  */
 public final class GridDhtLockFuture extends GridCacheCompoundIdentityFuture<Boolean>
-    implements GridCacheVersionedFuture<Boolean>, GridDhtFuture<Boolean>, GridCacheMappedVersion, DhtLockFuture<Boolean> {
+    implements GridCacheVersionedFuture<Boolean>, GridDhtFuture<Boolean>, GridCacheMappedVersion {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -638,7 +638,7 @@
     /**
      * @param t Error.
      */
-    @Override public void onError(Throwable t) {
+    public void onError(Throwable t) {
         synchronized (this) {
             if (err != null)
                 return;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index 143c3aa..72155ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -445,7 +445,7 @@
 
             GridDistributedTxMapping nearMapping = nearMap.get(n.id());
 
-            if (!dhtMapping.queryUpdate() && dhtMapping.empty() && nearMapping != null && nearMapping.empty())
+            if (dhtMapping.empty() && nearMapping != null && nearMapping.empty())
                 // Nothing to send.
                 continue;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
index 151f5dd..7b8d329 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
@@ -101,7 +101,7 @@
      * @param addDepInfo Deployment info flag.
      * @param retVal Need return value
      * @param waitRemoteTxs Wait remote transactions flag
-     * @param updCntrs Update counters for mvcc Tx.
+     * @param updCntrs Update counters for Tx.
      */
     public GridDhtTxFinishRequest(
         UUID nearNodeId,
@@ -187,7 +187,7 @@
      * @param addDepInfo Deployment info flag.
      * @param retVal Need return value
      * @param waitRemoteTxs Wait remote transactions flag
-     * @param updCntrs Update counters for mvcc Tx.
+     * @param updCntrs Update counters for Tx.
      */
     public GridDhtTxFinishRequest(
         UUID nearNodeId,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index 49f0091..d4929da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@ -417,8 +417,8 @@
             final IgniteInternalFuture<?> lockFut = tryRollbackAsync();
 
             if (lockFut != null) {
-                if (lockFut instanceof DhtLockFuture)
-                    ((DhtLockFuture<?>)lockFut).onError(rollbackException());
+                if (lockFut instanceof GridDhtLockFuture)
+                    ((GridDhtLockFuture)lockFut).onError(rollbackException());
                 else if (!lockFut.isDone()) {
                     /*
                      * Prevents race with {@link GridDhtTransactionalCacheAdapter#lockAllAsync
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 0840711..97f5e35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -1076,7 +1076,7 @@
         assert req != null;
         try (MTC.TraceSurroundings ignored =
                  MTC.supportContinual(span = cctx.kernalContext().tracing().create(TX_DHT_PREPARE, MTC.span()))) {
-            if (tx.empty() && !req.queryUpdate()) {
+            if (tx.empty()) {
                 tx.setRollbackOnly();
 
                 onDone((GridNearTxPrepareResponse)null);
@@ -1326,7 +1326,7 @@
 
                     assert entry != null && entry.cached() != null : entry;
 
-                    // Counter shouldn't be reserved for mvcc, local cache entries, NOOP operations and NOOP transforms.
+                    // Counter shouldn't be reserved for local cache entries, NOOP operations and NOOP transforms.
                     if (!entry.cached().isLocal() && entry.op() != NOOP &&
                         !(entry.op() == TRANSFORM &&
                             (entry.entryProcessorCalculatedValue() == null || // Possible for txs over cachestore
@@ -1394,7 +1394,7 @@
 
         // Create mini futures.
         for (GridDistributedTxMapping dhtMapping : tx.dhtMap().values()) {
-            assert !dhtMapping.empty() || dhtMapping.queryUpdate();
+            assert !dhtMapping.empty();
 
             ClusterNode n = dhtMapping.primary();
 
@@ -1406,7 +1406,7 @@
 
             Collection<IgniteTxEntry> dhtWrites = dhtMapping.writes();
 
-            if (!dhtMapping.queryUpdate() && F.isEmpty(dhtWrites) && F.isEmpty(nearWrites))
+            if (F.isEmpty(dhtWrites) && F.isEmpty(nearWrites))
                 continue;
 
             MiniFuture fut = new MiniFuture(n.id(), ++miniId, dhtMapping, nearMapping);
@@ -1433,8 +1433,6 @@
                 retVal,
                 cctx.tm().txHandler().filterUpdateCountersForBackupNode(tx, n));
 
-            req.queryUpdate(dhtMapping.queryUpdate());
-
             int idx = 0;
 
             for (IgniteTxEntry entry : dhtWrites) {
@@ -1940,7 +1938,7 @@
                         }
                     }
 
-                    if (!dhtMapping.queryUpdate() && dhtMapping.empty()) {
+                    if (dhtMapping.empty()) {
                         dhtMap.remove(nodeId);
 
                         if (log.isDebugEnabled())
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
index c2f70cf..b5c94db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
@@ -132,7 +132,7 @@
      * @param addDepInfo Deployment info flag.
      * @param storeWriteThrough Cache store write through flag.
      * @param retVal Need return value flag
-     * @param updCntrs Update counters for mvcc Tx.
+     * @param updCntrs Update counters for Tx.
      */
     public GridDhtTxPrepareRequest(
         IgniteUuid futId,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java
index b35cf6d..0c49966 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/PartitionUpdateCountersMessage.java
@@ -111,19 +111,6 @@
 
     /**
      * @param idx Item number.
-     * @param value Initial partition counter.
-     */
-    public void initialCounter(int idx, long value) {
-        if (idx >= size)
-            throw new ArrayIndexOutOfBoundsException();
-
-        long off = GridUnsafe.BYTE_ARR_OFF + idx * ITEM_SIZE + 4;
-
-        GridUnsafe.putLong(data, off, value);
-    }
-
-    /**
-     * @param idx Item number.
      * @return Update counter delta.
      */
     public long updatesCount(int idx) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index db5a001..6d548ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -189,14 +189,13 @@
         final boolean needVer) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
-        GridNearTxLocal tx = checkCurrentTx();
+        GridNearTxLocal tx = ctx.tm().threadLocalTx(ctx);
 
         final CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         final boolean recovery = opCtx != null && opCtx.recovery();
         final ReadRepairStrategy readRepairStrategy = opCtx != null ? opCtx.readRepairStrategy() : null;
 
-        // Get operation bypass Tx in Mvcc mode.
         if (tx != null && !tx.implicit() && !skipTx) {
             return asyncOp(tx, new AsyncOp<V>() {
                 @Override public IgniteInternalFuture<V> op(GridNearTxLocal tx, AffinityTopologyVersion readyTopVer) {
@@ -293,7 +292,7 @@
 
         warnIfUnordered(keys, BulkOperation.GET);
 
-        GridNearTxLocal tx = checkCurrentTx();
+        GridNearTxLocal tx = ctx.tm().threadLocalTx(ctx);
 
         final CacheOperationContext opCtx = ctx.operationContextPerCall();
 
@@ -340,7 +339,7 @@
                 tx).multi();
         }
 
-        IgniteInternalFuture<Map<K, V>> fut = loadAsync(
+        return loadAsync(
             ctx.cacheKeysView(keys),
             opCtx == null || !opCtx.skipStore(),
             forcePrimary,
@@ -354,8 +353,6 @@
             false,
             null
         );
-
-        return fut;
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
index 573a428..a213543 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplier.java
@@ -370,10 +370,13 @@
                 if (!remainingParts.contains(part))
                     continue;
 
-                GridCacheEntryInfo info = extractEntryInfo(row);
+                GridCacheEntryInfo info = new GridCacheEntryInfo();
 
-                if (info == null)
-                    continue;
+                info.key(row.key());
+                info.cacheId(row.cacheId());
+                info.value(row.value());
+                info.version(row.version());
+                info.expireTime(row.expireTime());
 
                 supplyMsg.addEntry0(part, iter.historical(part), info, grp.shared(), grp.cacheObjectContext());
 
@@ -520,24 +523,6 @@
     }
 
     /**
-     * Extracts entry info from row.
-     *
-     * @param row Cache data row.
-     * @return Entry info.
-     */
-    private GridCacheEntryInfo extractEntryInfo(CacheDataRow row) {
-        GridCacheEntryInfo info = new GridCacheEntryInfo();
-
-        info.key(row.key());
-        info.cacheId(row.cacheId());
-        info.value(row.value());
-        info.version(row.version());
-        info.expireTime(row.expireTime());
-
-        return info;
-    }
-
-    /**
      * Sends supply message to demand node.
      *
      * @param demander Recipient of supply message.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
index 245f580..55b15ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
@@ -951,7 +951,7 @@
     }
 
     /**
-     * Updates MVCC cache update counter on backup node.
+     * Updates cache update counter on backup node.
      *
      * @param start Start position
      * @param delta Delta.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index 2aa76f9..76a4561 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -240,8 +240,6 @@
             tx.activeCachesDeploymentEnabled(),
             tx.txState().recovery());
 
-        req.queryUpdate(m.queryUpdate());
-
         for (IgniteTxEntry txEntry : writes) {
             if (txEntry.op() == TRANSFORM)
                 req.addDhtVersion(txEntry.txKey(), null);
@@ -296,8 +294,6 @@
 
         boolean hasNearCache = false;
 
-        Map<UUID, Collection<UUID>> txNodes;
-
         GridDhtTxMapping txMapping = new GridDhtTxMapping();
 
         for (IgniteTxEntry txEntry : tx.allEntries()) {
@@ -335,7 +331,7 @@
             txMapping.addMapping(nodes);
         }
 
-        txNodes = txMapping.transactionNodes();
+        Map<UUID, Collection<UUID>> txNodes = txMapping.transactionNodes();
 
         tx.transactionNodes(txNodes);
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 0f83b2b..f86c4e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -739,11 +739,11 @@
     private void finish(int miniId, GridDistributedTxMapping m, boolean commit) {
         ClusterNode n = m.primary();
 
-        assert !m.empty() || m.queryUpdate() : m + " " + tx.state();
+        assert !m.empty() : m + " " + tx.state();
 
         CacheWriteSynchronizationMode syncMode = tx.syncMode();
 
-        if (m.explicitLock() || m.queryUpdate())
+        if (m.explicitLock())
             syncMode = FULL_SYNC;
 
         GridNearTxFinishRequest req = new GridNearTxFinishRequest(
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 3b00043..a3205a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -2158,7 +2158,7 @@
      * @param skipVals Skip values flag.
      * @param keepCacheObjects Keep cache objects flag.
      * @param skipStore Skip store flag.
-     * @param recovery Recovery flag..
+     * @param recovery Recovery flag.
      * @return Enlisted keys.
      * @throws IgniteCheckedException If failed.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
index 24ef5b6..eac1bb8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFutureAdapter.java
@@ -262,7 +262,7 @@
                 txEntry.ttl(CU.toTtl(expiry.getExpiryForAccess()));
         }
 
-        if (m.queryUpdate() || !m.empty()) {
+        if (!m.empty()) {
             // This step is very important as near and DHT versions grow separately.
             cctx.versions().onReceived(nodeId, res.dhtVersion());
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
deleted file mode 100644
index 4c7f395..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/mvcc/MvccUtils.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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 org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.transactions.TransactionUnsupportedConcurrencyException;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Utils for MVCC.
- */
-public class MvccUtils {
-    /**
-     * @param ctx Grid kernal context.
-     * @return Currently started user transaction, or {@code null} if none started.
-     * @throws TransactionUnsupportedConcurrencyException If transaction mode is not supported when MVCC is enabled.
-     */
-    @Nullable public static GridNearTxLocal tx(GridKernalContext ctx) {
-        return tx(ctx, null);
-    }
-
-    /**
-     * @param ctx Grid kernal context.
-     * @param txId Transaction ID.
-     * @return Currently started user transaction, or {@code null} if none started.
-     * @throws TransactionUnsupportedConcurrencyException If transaction mode is not supported when MVCC is enabled.
-     */
-    @Nullable public static GridNearTxLocal tx(GridKernalContext ctx, @Nullable GridCacheVersion txId) {
-        IgniteTxManager tm = ctx.cache().context().tm();
-
-        IgniteInternalTx tx0 = txId == null ? tm.tx() : tm.tx(txId);
-
-        GridNearTxLocal tx = tx0 != null && tx0.user() ? (GridNearTxLocal)tx0 : null;
-
-        if (tx != null) {
-            if (!tx.pessimistic()) {
-                tx.setRollbackOnly();
-
-                throw new TransactionUnsupportedConcurrencyException("Only pessimistic transactions are supported when MVCC is enabled.");
-            }
-        }
-
-        return tx;
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
index 735d2ac..dce8064 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
@@ -50,7 +50,6 @@
 import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId;
 import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId;
 import static org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.RowData.KEY_ONLY;
-import static org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter.RowData.LINK_WITH_HEADER;
 import static org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO.T_DATA;
 import static org.apache.ignite.internal.util.GridUnsafe.wrapPointer;
 
@@ -377,8 +376,6 @@
 
         long nextLink = data.nextLink();
 
-        int hdrLen = 0;
-
         if (incomplete == null) {
             if (nextLink == 0) {
                 // Fast path for a single page row.
@@ -386,18 +383,12 @@
 
                 return null;
             }
-
-            // Assume that row header is always located entirely on the very first page.
-            hdrLen = readHeader(sharedCtx, pageAddr, data.offset(), rowData);
-
-            if (rowData == LINK_WITH_HEADER)
-                return null;
         }
 
         ByteBuffer buf = wrapPointer(pageAddr, pageSize);
 
-        int off = data.offset() + hdrLen;
-        int payloadSize = data.payloadSize() - hdrLen;
+        int off = data.offset();
+        int payloadSize = data.payloadSize();
 
         buf.position(off);
         buf.limit(off + payloadSize);
@@ -413,20 +404,6 @@
     }
 
     /**
-     * Reads row header (i.e. MVCC info) which should be located on the very first page od data.
-     *
-     * @param sharedCtx Shared context.
-     * @param addr Address.
-     * @param off Offset
-     * @param rowData Required row data.
-     * @return Number of bytes read.
-     */
-    protected int readHeader(GridCacheSharedContext<?, ?> sharedCtx, long addr, int off, RowData rowData) {
-        // No-op.
-        return 0;
-    }
-
-    /**
      * @param sharedCtx Cache shared context.
      * @param coctx Cache object context.
      * @param buf Buffer.
@@ -532,11 +509,6 @@
     ) throws IgniteCheckedException {
         int off = 0;
 
-        off += readHeader(sharedCtx, addr, off, rowData);
-
-        if (rowData == LINK_WITH_HEADER)
-            return;
-
         if (readCacheId) {
             cacheId = PageUtils.getInt(addr, off);
 
@@ -549,7 +521,7 @@
         int len = PageUtils.getInt(addr, off);
         off += 4;
 
-        if (rowData != RowData.NO_KEY && rowData != RowData.NO_KEY_WITH_HINTS) {
+        if (rowData != RowData.NO_KEY) {
             byte type = PageUtils.getByte(addr, off);
             off++;
 
@@ -919,11 +891,6 @@
         return len + (cacheId() != 0 ? 4 : 0);
     }
 
-    /** {@inheritDoc} */
-    @Override public int headerSize() {
-        return 0;
-    }
-
     /**
      *
      */
@@ -935,19 +902,7 @@
         KEY_ONLY,
 
         /** */
-        NO_KEY,
-
-        /** */
-        LINK_ONLY,
-
-        /** */
-        LINK_WITH_HEADER,
-
-        /** Force instant hints actualization for rebalance (to avoid races with vacuum). */
-        FULL_WITH_HINTS,
-
-        /** Force instant hints actualization for update operation with history (to avoid races with vacuum). */
-        NO_KEY_WITH_HINTS
+        NO_KEY
     }
 
     /** {@inheritDoc} */
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 9de2efe..9643b65 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
@@ -1226,8 +1226,7 @@
                         grp.groupId(),
                         grp.dataRegion().pageMemory(),
                         globalRemoveId(),
-                        reuseListForIndex(name),
-                        false
+                        reuseListForIndex(name)
                     );
 
                     indexStorage.dropIndex(name);
@@ -1647,11 +1646,6 @@
         }
 
         /** {@inheritDoc} */
-        @Override public int headerSize() {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
         @Override public long link() {
             return 0;
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
index 422f1fd..cffcf9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
@@ -27,7 +27,6 @@
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
-import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
 import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
@@ -158,30 +157,6 @@
     }
 
     /**
-     * Run page handler operation over the row.
-     *
-     * @param link Row link.
-     * @param pageHnd Page handler.
-     * @param arg Page handler argument.
-     * @throws IgniteCheckedException If failed.
-     */
-    public <S, R> void updateDataRow(long link, PageHandler<S, R> pageHnd, S arg,
-        IoStatisticsHolder statHolder) throws IgniteCheckedException {
-        if (!persistenceEnabled)
-            freeList.updateDataRow(link, pageHnd, arg, statHolder);
-        else {
-            ctx.database().checkpointReadLock();
-
-            try {
-                freeList.updateDataRow(link, pageHnd, arg, statHolder);
-            }
-            finally {
-                ctx.database().checkpointReadUnlock();
-            }
-        }
-    }
-
-    /**
      * @return Free list.
      */
     public FreeList freeList() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java
index d8fe3a4..a0f5ab1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/Storable.java
@@ -47,12 +47,6 @@
     public int size() throws IgniteCheckedException;
 
     /**
-     * @return Row header size in page. Header is indivisible part of row
-     * which is entirely available on the very first page followed by the row link.
-     */
-    public int headerSize();
-
-    /**
      * @return I/O for handling this storable.
      */
     public IOVersions<? extends AbstractDataPageIO> ioVersions();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
index df220b2..c86c2ec 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java
@@ -59,7 +59,6 @@
 import org.apache.ignite.internal.pagemem.store.PageStoreCollection;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
@@ -437,32 +436,6 @@
     }
 
     /** {@inheritDoc} */
-    @Override public void initialize(int cacheId, int partitions, String cacheName, PageMetrics pageMetrics)
-        throws IgniteCheckedException {
-        assert storeWorkDir != null;
-
-        if (!idxCacheStores.containsKey(cacheId)) {
-            GridCacheContext<?, ?> cctx = this.cctx.cacheContext(cacheId);
-
-            CacheStoreHolder holder = initDir(
-                new File(storeWorkDir, cacheName),
-                cacheId,
-                cacheName,
-                partitions,
-                pageMetrics,
-                cctx != null && cctx.config().isEncryptionEnabled(),
-                cctx != null
-                    ? cctx.group().caches().stream().map(GridCacheContext::name).collect(Collectors.toSet())
-                    : null
-            );
-
-            CacheStoreHolder old = idxCacheStores.put(cacheId, holder);
-
-            assert old == null : "Non-null old store holder for cacheId: " + cacheId;
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override public void initializeForCache(CacheGroupDescriptor grpDesc, CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
         assert storeWorkDir != null;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java
index 1759e22..7e149c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java
@@ -803,32 +803,6 @@
     }
 
     /** {@inheritDoc} */
-    @Override public <S, R> R updateDataRow(long link, PageHandler<S, R> pageHnd, S arg,
-        IoStatisticsHolder statHolder) throws IgniteCheckedException {
-        assert link != 0;
-
-        try {
-            long pageId = PageIdUtils.pageId(link);
-            int itemId = PageIdUtils.itemId(link);
-
-            R updRes = write(pageId, pageHnd, arg, itemId, null, statHolder);
-
-            assert updRes != null; // Can't fail here.
-
-            return updRes;
-        }
-        catch (AssertionError e) {
-            throw corruptedFreeListException(e);
-        }
-        catch (IgniteCheckedException | Error e) {
-            throw e;
-        }
-        catch (Throwable t) {
-            throw new CorruptedFreeListException("Failed to update data row", t, grpId);
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override public void removeDataRowByLink(long link, IoStatisticsHolder statHolder) throws IgniteCheckedException {
         assert link != 0;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java
index 5914ae52..7d492fd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java
@@ -22,7 +22,6 @@
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.processors.cache.persistence.Storable;
-import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
 
 /**
  */
@@ -52,19 +51,6 @@
 
     /**
      * @param link Row link.
-     * @param pageHnd Page handler.
-     * @param arg Handler argument.
-     * @param <S> Argument type.
-     * @param <R> Result type.
-     * @param statHolder Statistics holder to track IO operations.
-     * @return Result.
-     * @throws IgniteCheckedException If failed.
-     */
-    public <S, R> R updateDataRow(long link, PageHandler<S, R> pageHnd, S arg, IoStatisticsHolder statHolder)
-        throws IgniteCheckedException;
-
-    /**
-     * @param link Row link.
      * @param statHolder Statistics holder to track IO operations.
      * @throws IgniteCheckedException If failed.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/SimpleDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/SimpleDataRow.java
index 382869c..a18a4f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/SimpleDataRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/SimpleDataRow.java
@@ -79,11 +79,6 @@
         return 2 /** Fragment size */ + 2 /** Row size */ + value().length;
     }
 
-    /** {@inheritDoc} */
-    @Override public int headerSize() {
-        return 0;
-    }
-
     /**
      * @return Value.
      */
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 9816a18..aae7287 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
@@ -28,7 +28,6 @@
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Supplier;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureType;
@@ -1367,63 +1366,6 @@
         }
     }
 
-    /**
-     * @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();
-
-        ClosureCursor cursor = new ClosureCursor(lower, upper, c);
-
-        try {
-            cursor.iterate();
-        }
-        catch (CorruptedDataStructureException e) {
-            throw e;
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteCheckedException("Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]", e);
-        }
-        catch (RuntimeException | AssertionError e) {
-            throw corruptedTreeException(
-                "Runtime failure on bounds: [lower=" + lower + ", upper=" + upper + "]",
-                e, grpId, pages(cursor.getCursor != null, () -> new long[]{cursor.getCursor.pageId})
-            );
-        }
-        finally {
-            checkDestroyed();
-        }
-    }
-
-    /**
-     * @param lower Lower bound inclusive.
-     * @param upper Upper bound inclusive.
-     * @param c Closure applied for all found items.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void visit(L lower, L upper, TreeVisitorClosure<L, T> c) throws IgniteCheckedException {
-        checkDestroyed();
-
-        try {
-            new TreeVisitor(lower, upper, c).visit();
-        }
-        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 {
         return findFirst(null);
@@ -3138,87 +3080,6 @@
     }
 
     /**
-     * @param c Get.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void doVisit(TreeVisitor c) throws IgniteCheckedException {
-        for (;;) { // Go down with retries.
-            c.init();
-
-            switch (visitDown(c, c.rootId, 0L, c.rootLvl)) {
-                case RETRY:
-                case RETRY_ROOT:
-                    checkInterrupted();
-
-                    continue;
-
-                default:
-                    return;
-            }
-        }
-    }
-
-    /**
-     * @param v Tree visitor.
-     * @param pageId Page ID.
-     * @param fwdId Expected forward page ID.
-     * @param lvl Level.
-     * @return Result code.
-     * @throws IgniteCheckedException If failed.
-     */
-    private Result visitDown(final TreeVisitor v, final long pageId, final long fwdId, final int lvl)
-            throws IgniteCheckedException {
-        long page = acquirePage(pageId);
-
-        try {
-            for (;;) {
-                v.checkLockRetry();
-
-                // Init args.
-                v.pageId = pageId;
-                v.fwdId = fwdId;
-
-                Result res = read(pageId, page, search, v, lvl, RETRY);
-
-                switch (res) {
-                    case GO_DOWN:
-                    case GO_DOWN_X:
-                        assert v.pageId != pageId;
-                        assert v.fwdId != fwdId || fwdId == 0;
-
-                        // Go down recursively.
-                        res = visitDown(v, v.pageId, v.fwdId, lvl - 1);
-
-                        switch (res) {
-                            case RETRY:
-                                checkInterrupted();
-
-                                continue; // The child page got split, need to reread our page.
-
-                            default:
-                                return res;
-                        }
-
-                    case NOT_FOUND:
-                        assert lvl == 0 : lvl;
-
-                        return v.init(pageId, page, fwdId);
-
-                    case FOUND:
-                        throw new IllegalStateException(); // Must never be called because we always have a shift.
-
-                    default:
-                        return res;
-                }
-            }
-        }
-        finally {
-            if (v.canRelease(pageId, lvl))
-                releasePage(pageId, page);
-        }
-    }
-
-    /**
      * @param io IO.
      * @param pageAddr Page address.
      * @param back Backward page.
@@ -3496,259 +3357,6 @@
     }
 
     /**
-     * Get a cursor for range.
-     */
-    private final class TreeVisitor extends Get {
-        /** */
-        long nextPageId;
-
-        /** */
-        L upper;
-
-        /** */
-        TreeVisitorClosure<L, T> p;
-
-        /** */
-        private boolean dirty;
-
-        /** */
-        private boolean writing;
-
-        /**
-         * @param lower Lower bound.
-         */
-        TreeVisitor(L lower, L upper, TreeVisitorClosure<L, T> p) {
-            super(lower, false);
-
-            this.shift = -1;
-            this.upper = upper;
-            this.p = p;
-        }
-
-        /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
-            throw new IllegalStateException(); // Must never be called because we always have a shift.
-        }
-
-        /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
-            if (lvl != 0)
-                return false;
-
-            if (!(writing = (p.state() & TreeVisitorClosure.CAN_WRITE) != 0))
-                init(pageAddr, io, idx);
-
-            return true;
-        }
-
-        /** */
-        Result init(long pageId, long page, long fwdId) throws IgniteCheckedException {
-            // Init args.
-            this.pageId = pageId;
-            this.fwdId = fwdId;
-
-            if (writing) {
-                long pageAddr = writeLock(pageId, page);
-
-                if (pageAddr == 0)
-                    return RETRY;
-
-                try {
-                    BPlusIO<L> io = io(pageAddr);
-
-                    // Check triangle invariant.
-                    if (io.getForward(pageAddr) != fwdId)
-                        return RETRY;
-
-                    init(pageAddr, io, -1);
-                }
-                finally {
-                    unlock(pageId, page, pageAddr);
-                }
-            }
-
-            return NOT_FOUND;
-        }
-
-        /**
-         * @param pageAddr Page address.
-         * @param io IO.
-         * @param startIdx Start index.
-         * @throws IgniteCheckedException If failed.
-         */
-        private void init(long pageAddr, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
-            nextPageId = 0;
-
-            int cnt = io.getCount(pageAddr);
-
-            if (cnt != 0)
-                visit(pageAddr, io, startIdx, cnt);
-        }
-
-        /**
-         * @param pageAddr Page address.
-         * @param io IO.
-         * @param startIdx Start index.
-         * @param cnt Number of rows in the buffer.
-         * @throws IgniteCheckedException If failed.
-         */
-        private void visit(long pageAddr, BPlusIO<L> io, int startIdx, int cnt)
-                throws IgniteCheckedException {
-            assert io.isLeaf() : io;
-            assert cnt != 0 : cnt; // We can not see empty pages (empty tree handled in init).
-            assert startIdx >= -1 : startIdx;
-            assert cnt >= startIdx;
-
-            checkDestroyed();
-
-            nextPageId = io.getForward(pageAddr);
-
-            if (startIdx == -1)
-                startIdx = findLowerBound(pageAddr, io, cnt);
-
-            if (cnt == startIdx)
-                return; // Go to the next page;
-
-            cnt = findUpperBound(pageAddr, io, startIdx, cnt);
-
-            for (int i = startIdx; i < cnt; i++) {
-                int state = p.visit(BPlusTree.this, io, pageAddr, i, wal);
-
-                boolean stop = (state & TreeVisitorClosure.STOP) != 0;
-
-                if (writing)
-                    dirty = dirty || (state & TreeVisitorClosure.DIRTY) != 0;
-
-                if (stop) {
-                    nextPageId = 0; // The End.
-
-                    return;
-                }
-            }
-
-            if (nextPageId != 0) {
-                row = io.getLookupRow(BPlusTree.this, pageAddr, cnt - 1); // Need save last row.
-
-                shift = 1;
-            }
-        }
-
-        /**
-         * @param pageAddr Page address.
-         * @param io IO.
-         * @param cnt Count.
-         * @return Adjusted to lower bound start index.
-         * @throws IgniteCheckedException If failed.
-         */
-        private int findLowerBound(long pageAddr, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
-            assert io.isLeaf();
-
-            // Compare with the first row on the page.
-            int cmp = compare(0, io, pageAddr, 0, row);
-
-            if (cmp < 0 || (cmp == 0 && shift == 1)) {
-                int idx = findInsertionPoint(0, io, pageAddr, 0, cnt, row, shift);
-
-                assert idx < 0;
-
-                return fix(idx);
-            }
-
-            return 0;
-        }
-
-        /**
-         * @param pageAddr Page address.
-         * @param io IO.
-         * @param low Start index.
-         * @param cnt Number of rows in the buffer.
-         * @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 {
-            assert io.isLeaf();
-
-            // Compare with the last row on the page.
-            int cmp = compare(0, io, pageAddr, cnt - 1, upper);
-
-            if (cmp > 0) {
-                int idx = findInsertionPoint(0, io, pageAddr, low, cnt, upper, 1);
-
-                assert idx < 0;
-
-                cnt = fix(idx);
-
-                nextPageId = 0; // The End.
-            }
-
-            return cnt;
-        }
-
-        /**
-         * @throws IgniteCheckedException If failed.
-         */
-        private void nextPage() throws IgniteCheckedException {
-            for (;;) {
-                if (nextPageId == 0)
-                    return;
-
-                long pageId = nextPageId;
-                long page = acquirePage(pageId);
-                try {
-                    long pageAddr = lock(pageId, page); // Doing explicit null check.
-
-                    // If concurrent merge occurred we have to reinitialize cursor from the last returned row.
-                    if (pageAddr == 0L)
-                        break;
-
-                    try {
-                        BPlusIO<L> io = io(pageAddr);
-
-                        visit(pageAddr, io, -1, io.getCount(pageAddr));
-                    }
-                    finally {
-                        unlock(pageId, page, pageAddr);
-                    }
-                }
-                finally {
-                    releasePage(pageId, page);
-                }
-            }
-
-            doVisit(this); // restart from last read row
-        }
-
-        /** */
-        private void unlock(long pageId, long page, long pageAddr) {
-            if (writing) {
-                writeUnlock(pageId, page, pageAddr, dirty);
-
-                dirty = false; // reset dirty flag
-            }
-            else
-                readUnlock(pageId, page, pageAddr);
-        }
-
-        /** */
-        private long lock(long pageId, long page) {
-            if (writing = ((p.state() & TreeVisitorClosure.CAN_WRITE) != 0))
-                return writeLock(pageId, page);
-            else
-                return readLock(pageId, page);
-        }
-
-        /**
-         * @throws IgniteCheckedException If failed.
-         */
-        private void visit() throws IgniteCheckedException {
-            doVisit(this);
-
-            while (nextPageId != 0)
-                nextPage();
-        }
-    }
-
-    /**
      * Get the last item in the tree which matches the passed filter.
      */
     private final class GetLast extends Get {
@@ -6036,101 +5644,6 @@
     }
 
     /**
-     * Closure cursor.
-     */
-    private final class ClosureCursor extends AbstractForwardCursor {
-        /** */
-        private final TreeRowClosure<L, T> p;
-
-        /** */
-        private L lastRow;
-
-        /**
-         * @param lowerBound Lower bound inclusive.
-         * @param upperBound Upper bound inclusive.
-         * @param p Row predicate.
-         */
-        ClosureCursor(L lowerBound, L upperBound, TreeRowClosure<L, T> p) {
-            super(lowerBound, upperBound, true, true);
-
-            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)
-                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.
      */
     private final class ForwardCursor extends AbstractForwardCursor implements GridCursor<T> {
@@ -6439,36 +5952,6 @@
     }
 
     /**
-     * A generic visitor-style interface for performing inspection/modification operations on the tree.
-     */
-    public interface TreeVisitorClosure<L, T extends L> {
-        /** */
-        int STOP = 0x01;
-        /** */
-        int CAN_WRITE = STOP << 1;
-        /** */
-        int DIRTY = CAN_WRITE << 1;
-
-        /**
-         * Performs inspection or operation on a specified row.
-         *
-         * @param tree The tree.
-         * @param io Th tree IO object.
-         * @param pageAddr The page address.
-         * @param idx The item index.
-         * @return state bitset.
-         * @throws IgniteCheckedException If failed.
-         */
-        public int visit(BPlusTree<L, T> tree, BPlusIO<L> io, long pageAddr, int idx, IgniteWriteAheadLogManager wal)
-            throws IgniteCheckedException;
-
-        /**
-         * @return state bitset.
-         */
-        public int state();
-    }
-
-    /**
      * @return Return number of retries.
      */
     protected int getLockRetries() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java
index 043036c..9802a6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/AbstractDataPageIO.java
@@ -1142,16 +1142,6 @@
         int payloadSize = payload != null ? payload.length :
             Math.min(rowSize - written, getFreeSpace(pageAddr));
 
-        if (row != null) {
-            int remain = rowSize - written - payloadSize;
-            int hdrSize = row.headerSize();
-
-            // We need page header (i.e. MVCC info) is located entirely on the very first page in chain.
-            // So we force moving it to the next page if it could not fit entirely on this page.
-            if (remain > 0 && remain < hdrSize)
-                payloadSize -= hdrSize - remain;
-        }
-
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
         int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
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 f457254..7246379 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
@@ -28,7 +28,7 @@
 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.FilteredRecord;
-import org.apache.ignite.internal.pagemem.wal.record.MarshalledDataEntry;
+import org.apache.ignite.internal.pagemem.wal.record.LazyDataEntry;
 import org.apache.ignite.internal.pagemem.wal.record.UnwrapDataEntry;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType;
@@ -431,8 +431,8 @@
         final CacheObject val;
         boolean keepBinary = this.keepBinary || !fakeCacheObjCtx.kernalContext().marshallerContext().initialized();
 
-        if (dataEntry instanceof MarshalledDataEntry) {
-            final MarshalledDataEntry lazyDataEntry = (MarshalledDataEntry)dataEntry;
+        if (dataEntry instanceof LazyDataEntry) {
+            final LazyDataEntry lazyDataEntry = (LazyDataEntry)dataEntry;
 
             key = processor.toKeyCacheObject(fakeCacheObjCtx,
                 lazyDataEntry.getKeyType(),
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index e2a78a2..5e7b7b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -608,16 +608,12 @@
 
         boolean loc = nodes.size() == 1 && F.first(nodes).id().equals(cctx.localNodeId());
 
-        GridCloseableIterator it;
-
         if (loc)
-            it = qryMgr.scanQueryLocal(this, true);
+            return qryMgr.scanQueryLocal(this, true);
         else if (part != null)
-            it = new ScanQueryFallbackClosableIterator(part, this, qryMgr, cctx);
+            return new ScanQueryFallbackClosableIterator(part, this, qryMgr, cctx);
         else
-            it = qryMgr.scanQueryDistributed(this, nodes);
-
-        return it;
+            return qryMgr.scanQueryDistributed(this, nodes);
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index 113ab77..665cf6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -465,9 +465,6 @@
                 if (cctx == null)
                     return;
 
-                if (!needNotify(false, cctx, -1, -1, evt))
-                    return;
-
                 // skipPrimaryCheck is set only when listen locally for replicated cache events.
                 assert !skipPrimaryCheck || (cctx.isReplicated() && ctx.localNodeId().equals(nodeId));
 
@@ -577,9 +574,6 @@
                 if (skipCtx == null)
                     skipCtx = new CounterSkipContext(part, cntr, topVer);
 
-                if (!needNotify(true, cctx, part, cntr, null))
-                    return skipCtx;
-
                 if (loc) {
                     assert !locOnly;
 
@@ -666,27 +660,6 @@
             @Override public boolean isPrimaryOnly() {
                 return locOnly && !skipPrimaryCheck;
             }
-
-            /**
-             * Checks whether it is need to notify listeners.
-             *
-             * @param skipEvt {@code True} if this is a skip counter event.
-             * @param cctx Cache context.
-             * @param part Partition id.
-             * @param cntr Update counter.
-             * @param evt CQ event.
-             * @return {@code True} if notification should happen immediately, or {@code false} if it should be delayed.
-             */
-            private boolean needNotify(boolean skipEvt,
-                GridCacheContext cctx,
-                int part,
-                long cntr,
-                CacheContinuousQueryEvent evt) {
-                assert !skipEvt || evt == null;
-                assert skipEvt || part == -1 && cntr == -1; // part == -1 && cntr == -1 means skip counter.
-
-                return true;
-            }
         };
 
         CacheContinuousQueryManager mgr = manager(ctx);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 085eae0..a3c00ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -66,7 +66,6 @@
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
-import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.continuous.GridContinuousHandler;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridLongList;
@@ -210,15 +209,6 @@
     }
 
     /**
-     * @param tx Transaction.
-     * @return {@code True} if should notify continuous query manager.
-     */
-    public boolean notifyContinuousQueries(@Nullable IgniteInternalTx tx) {
-        return cctx.isReplicated() ||
-            (!cctx.isNear() && !(tx != null && tx.onePhaseCommit() && !tx.local()));
-    }
-
-    /**
      * @param lsnrs Listeners to notify.
      * @param key Entry key.
      * @param partId Partition id.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 8df9267..2224173 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -730,8 +730,6 @@
      * @return {@code True} if cache affinity changed and request should be remapped.
      */
     private boolean needRemap(AffinityTopologyVersion rmtVer, GridNearTxPrepareRequest req) {
-        // TODO IGNITE-6754 check mvcc crd for mvcc enabled txs.
-
         for (IgniteTxEntry e : F.concat(false, req.reads(), req.writes())) {
             if (!e.context().affinity().isCompatibleWithCurrentTopologyVersion(rmtVer))
                 return true;
@@ -1654,7 +1652,7 @@
         GridDhtTxPrepareRequest req,
         GridDhtTxPrepareResponse res
     ) throws IgniteCheckedException {
-        if (req.queryUpdate() || !F.isEmpty(req.writes())) {
+        if (!F.isEmpty(req.writes())) {
             GridDhtTxRemote tx = ctx.tm().tx(req.version());
 
             if (tx == null) {
@@ -1843,7 +1841,7 @@
 
             res.invalidPartitionsByCacheId(tx.invalidPartitions());
 
-            if (!req.queryUpdate() && tx.empty() && req.last()) {
+            if (tx.empty() && req.last()) {
                 tx.skipCompletedVersions(req.skipCompletedVersion());
 
                 tx.rollbackRemoteTx();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index 0c38267..5c30d4a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -688,7 +688,6 @@
      * @param concurrency Concurrency.
      * @param isolation Isolation.
      * @param timeout transaction timeout.
-     * @param mvccOp Whether this transaction is being started via SQL API or not, or {@code null} if unknown.
      * @param txSize Expected transaction size.
      * @param lb Label.
      * @return New transaction.
@@ -1733,35 +1732,6 @@
     }
 
     /**
-     * Removes Tx from manager. Can be used only if there were no updates.
-     *
-     * @param tx Transaction to finish.
-     */
-    public void forgetTx(IgniteInternalTx tx) {
-        assert tx != null;
-
-        if (transactionMap(tx).remove(tx.xidVersion(), tx)) {
-            // 1. Remove from per-thread storage.
-            clearThreadMap(tx);
-
-            // 2. Unregister explicit locks.
-            if (!tx.alternateVersions().isEmpty())
-                for (GridCacheVersion ver : tx.alternateVersions())
-                    idMap.remove(ver);
-
-            // 3. Remove Near-2-DHT mappings.
-            if (tx instanceof GridCacheMappedVersion)
-                mappedVers.remove(((GridCacheMappedVersion)tx).mappedVersion());
-
-            // 4. Clear context.
-            resetContext();
-
-            // 5. Complete finish future.
-            tx.state(UNKNOWN);
-        }
-    }
-
-    /**
      * Tries to minimize damage from partially-committed transaction.
      *
      * @param tx Tx to uncommit.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java
index cd13883..2b7bb02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxCounters.java
@@ -40,21 +40,6 @@
     private volatile Map<Integer, PartitionUpdateCountersMessage> updCntrs;
 
     /**
-     * Accumulates size change for cache partition.
-     *
-     * @param cacheId Cache id.
-     * @param part Partition id.
-     * @param delta Size delta.
-     */
-    public void accumulateSizeDelta(int cacheId, int part, long delta) {
-        AtomicLong accDelta = accumulator(sizeDeltas, cacheId, part);
-
-        // here AtomicLong is used more as a container,
-        // every instance is assumed to be accessed in thread-confined manner
-        accDelta.set(accDelta.get() + delta);
-    }
-
-    /**
      * @return Map of size changes for cache partitions made by transaction.
      */
     public Map<Integer, Map<Integer, AtomicLong>> sizeDeltas() {
@@ -94,16 +79,6 @@
     }
 
     /**
-     * @param cacheId Cache id.
-     * @param part Partition number.
-     */
-    public void decrementUpdateCounter(int cacheId, int part) {
-        long acc = accumulator(updCntrsAcc, cacheId, part).decrementAndGet();
-
-        assert acc >= 0;
-    }
-
-    /**
      * @param accMap Map to obtain accumulator from.
      * @param cacheId Cache id.
      * @param part Partition number.
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 f732833..a3cabe6 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
@@ -56,7 +56,6 @@
             assert row.cacheId() != CU.UNDEFINED_CACHE_ID : row;
 
             PageUtils.putInt(pageAddr, off, row.cacheId());
-            off += 4;
         }
     }
 
@@ -99,7 +98,6 @@
             assert cacheId != CU.UNDEFINED_CACHE_ID;
 
             PageUtils.putInt(dstPageAddr, off, cacheId);
-            off += 4;
         }
     }
 
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 61c4c13..17933f2 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
@@ -55,7 +55,6 @@
             assert row.cacheId() != CU.UNDEFINED_CACHE_ID;
 
             PageUtils.putInt(pageAddr, off, row.cacheId());
-            off += 4;
         }
     }
 
@@ -83,7 +82,6 @@
             assert cacheId != CU.UNDEFINED_CACHE_ID;
 
             PageUtils.putInt(dstPageAddr, off, cacheId);
-            off += 4;
         }
     }
 
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 22eb901..94ece35 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
@@ -367,9 +367,7 @@
 
         assert row.key() != null : row;
 
-        cmp = compareKeys(row.key(), link);
-
-        return cmp;
+        return compareKeys(row.key(), link);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java
index f8d2ac0..362a0d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/DataRow.java
@@ -51,8 +51,7 @@
 
         try {
             // We can not init data row lazily outside of entry lock because underlying buffer can be concurrently cleared.
-            if (rowData != RowData.LINK_ONLY)
-                initFromLink(grp, rowData, skipVer);
+            initFromLink(grp, rowData, skipVer);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException(e);
@@ -85,13 +84,6 @@
     }
 
     /**
-     * @param link Link.
-     */
-    protected DataRow(long link) {
-        super(link);
-    }
-
-    /**
      *
      */
     public DataRow() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerUtils.java
index c5c8d53..8914964 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerUtils.java
@@ -32,7 +32,6 @@
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.transactions.TransactionDuplicateKeyException;
-import org.apache.ignite.transactions.TransactionUnsupportedConcurrencyException;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -298,8 +297,6 @@
             return IgniteQueryErrorCode.QUERY_CANCELED;
         if (e instanceof TransactionDuplicateKeyException)
             return IgniteQueryErrorCode.DUPLICATE_KEY;
-        if (e instanceof TransactionUnsupportedConcurrencyException)
-            return IgniteQueryErrorCode.UNSUPPORTED_OPERATION;
         if (e instanceof IgniteSQLException)
             return ((IgniteSQLException)e).statusCode();
         else
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
index 15637de..5390c58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryNextPageResponse.java
@@ -73,9 +73,6 @@
     /** Last page flag. */
     private boolean last;
 
-    /** Remove mapping flag. */
-    private boolean removeMapping;
-
     /**
      * For {@link Externalizable}.
      */
@@ -242,12 +239,6 @@
                     return false;
 
                 writer.incrementState();
-
-            case 10:
-                if (!writer.writeBoolean("removeMapping", removeMapping))
-                    return false;
-
-                writer.incrementState();
         }
 
         return true;
@@ -340,15 +331,6 @@
                     return false;
 
                 reader.incrementState();
-
-            case 10:
-                removeMapping = reader.readBoolean("removeMapping");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
         }
 
         return reader.afterMessageRead(GridQueryNextPageResponse.class);
@@ -361,7 +343,7 @@
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 11;
+        return 10;
     }
 
     /**
@@ -406,20 +388,6 @@
         this.last = last;
     }
 
-    /**
-     * @param removeMapping Remove mapping flag.
-     */
-    public void removeMapping(boolean removeMapping) {
-        this.removeMapping = removeMapping;
-    }
-
-    /**
-     * @return Remove mapping flag.
-     */
-    public boolean removeMapping() {
-        return removeMapping;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridQueryNextPageResponse.class, this,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
index 698d2a6..7eb95e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
@@ -32,8 +32,6 @@
 import org.apache.ignite.internal.ServiceMXBeanImpl;
 import org.apache.ignite.internal.TransactionsMXBeanImpl;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
-import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
@@ -309,8 +307,6 @@
         try {
             isDdlOnSchemaSupported(cmd.schemaName());
 
-            finishActiveTxIfNecessary();
-
             if (cmd instanceof SqlCreateIndexCommand) {
                 SqlCreateIndexCommand cmd0 = (SqlCreateIndexCommand)cmd;
 
@@ -430,21 +426,4 @@
             throw new IgniteSQLException(e.getMessage(), e);
         }
     }
-
-    /**
-     * Commits active transaction if exists.
-     *
-     * @throws IgniteCheckedException If failed.
-     */
-    protected void finishActiveTxIfNecessary() throws IgniteCheckedException {
-        try (GridNearTxLocal tx = MvccUtils.tx(ctx)) {
-            if (tx == null)
-                return;
-
-            if (!tx.isRollbackOnly())
-                tx.commit();
-            else
-                tx.rollback();
-        }
-    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
index 5ac0e57..35717a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
@@ -251,7 +251,7 @@
                         return cmd;
                     }
                     else
-                        throw errorUnexpectedToken(lex, BEGIN, COMMIT, CREATE, DROP, ROLLBACK, COPY, SET, ALTER, START, KILL);
+                        throw errorUnexpectedToken(lex, CREATE, DROP, COPY, SET, ALTER, KILL);
 
                 case QUOTED:
                 case MINUS:
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index b189027..d64b665 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -737,9 +737,6 @@
 org.apache.ignite.internal.processors.cache.GridCacheLockTimeoutException
 org.apache.ignite.internal.processors.cache.GridCacheLogger
 org.apache.ignite.internal.processors.cache.GridCacheMapEntry$1
-org.apache.ignite.internal.processors.cache.GridCacheMapEntry$MvccAcquireLockListener
-org.apache.ignite.internal.processors.cache.GridCacheMapEntry$MvccRemoveLockListener
-org.apache.ignite.internal.processors.cache.GridCacheMapEntry$MvccUpdateLockListener
 org.apache.ignite.internal.processors.cache.GridCacheMessage
 org.apache.ignite.internal.processors.cache.GridCacheMultiTxFuture$1
 org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate
@@ -1121,8 +1118,6 @@
 org.apache.ignite.internal.processors.cache.distributed.near.consistency.IgniteTransactionalConsistencyViolationException
 org.apache.ignite.internal.processors.cache.dr.GridCacheDrExpirationInfo
 org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo
-org.apache.ignite.internal.processors.cache.mvcc.DeadlockProbe
-org.apache.ignite.internal.processors.cache.mvcc.ProbedTx
 org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter$RowData
 org.apache.ignite.internal.processors.cache.persistence.CheckpointState
 org.apache.ignite.internal.processors.cache.persistence.CorruptedDataStructureException
@@ -1214,7 +1209,6 @@
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$3
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$4
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$1
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$MvccTrackingIterator
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$ScanQueryFallbackClosableIterator
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryDetailMetricsAdapter
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$1
@@ -2433,5 +2427,4 @@
 org.apache.ignite.transactions.TransactionRollbackException
 org.apache.ignite.transactions.TransactionState
 org.apache.ignite.transactions.TransactionTimeoutException
-org.apache.ignite.transactions.TransactionUnsupportedConcurrencyException
 org.apache.ignite.util.AttributeNodeFilter
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTestBase.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTestBase.java
new file mode 100644
index 0000000..d16b124
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTestBase.java
@@ -0,0 +1,209 @@
+/*
+ * 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;
+
+import java.util.List;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.FieldsQueryCursor;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.QueryEngineConfiguration;
+import org.apache.ignite.configuration.SqlConfiguration;
+import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.apache.ignite.transactions.TransactionState;
+import org.junit.Test;
+
+import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
+
+/**
+ *
+ */
+public abstract class DdlTransactionSelfTestBase extends GridCommonAbstractTest {
+    /** Create table request. */
+    public static final String CREATE_TABLE = "CREATE TABLE " +
+        "person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" +
+        "WITH " +
+        "\"cache_name=test_cache,template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"";
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setTransactionConfiguration(new TransactionConfiguration()
+            .setDefaultTxIsolation(TransactionIsolation.REPEATABLE_READ)
+            .setDefaultTxConcurrency(TransactionConcurrency.PESSIMISTIC)
+            .setDefaultTxTimeout(5000));
+
+        cfg.setCacheConfiguration(getCacheConfiguration());
+
+        cfg.setSqlConfiguration(new SqlConfiguration()
+            .setQueryEnginesConfiguration(getQueryEngineConfiguration()));
+
+        return cfg;
+    }
+
+    /** */
+    protected abstract QueryEngineConfiguration getQueryEngineConfiguration();
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<?, ?> getCacheConfiguration() {
+        return defaultCacheConfiguration()
+            .setBackups(1)
+            .setNearConfiguration(null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDdlRequestFailsInsideTxMultinodeClient() throws Exception {
+        startGridsMultiThreaded(4, false);
+
+        Ignite node = startClientGrid(4);
+
+        awaitPartitionMapExchange();
+
+        doTestDdlFailsInsideTx(node);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDdlRequestFailsInsideTxMultinode() throws Exception {
+        Ignite node = startGridsMultiThreaded(4);
+
+        doTestDdlFailsInsideTx(node);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDdlRequestFailsInsideTx() throws Exception {
+        Ignite node = startGrid();
+
+        doTestDdlFailsInsideTx(node);
+    }
+
+    /** */
+    private void doTestDdlFailsInsideTx(Ignite node) {
+        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
+
+        try (Transaction tx = node.transactions().txStart()) {
+            cache.putAll(F.asMap(1, 1, 2, 2, 3, 3));
+
+            String errMsg = "Cannot start/stop cache within lock or transaction [cacheNames=test_cache, " +
+                "operation=dynamicStartCache]";
+
+            Throwable cacheEx = assertThrows(
+                log,
+                () -> cache.query(new SqlFieldsQuery(CREATE_TABLE)
+                        .setSchema("PUBLIC"))
+                    .getAll(),
+                CacheException.class,
+                errMsg);
+
+            assertTrue(X.hasCause(cacheEx, errMsg, IgniteException.class));
+
+            assertTrue(tx.state() != TransactionState.COMMITTED);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDdlRequestWithoutTxMultinodeClient() throws Exception {
+        startGridsMultiThreaded(4, false);
+
+        Ignite node = startClientGrid(4);
+
+        awaitPartitionMapExchange();
+
+        doTestDdlWithoutTx(node);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDdlRequestWithoutTxMultinode() throws Exception {
+        Ignite node = startGridsMultiThreaded(4);
+
+        doTestDdlWithoutTx(node);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testDdlRequestWithoutTx() throws Exception {
+        Ignite node = startGrid();
+
+        doTestDdlWithoutTx(node);
+    }
+
+    /** */
+    private void doTestDdlWithoutTx(Ignite node) {
+        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
+
+        cache.putAll(F.asMap(1, 1, 2, 2, 3, 3));
+
+        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery(CREATE_TABLE).setSchema("PUBLIC"))) {
+            assertNotNull(cur);
+
+            List<List<?>> rows = cur.getAll();
+
+            assertEquals(1, rows.size());
+
+            assertEquals(0L, rows.get(0).get(0));
+        }
+
+        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person")
+            .setSchema("PUBLIC"))
+        ) {
+            assertNotNull(cur);
+
+            List<List<?>> rows = cur.getAll();
+
+            assertEquals(0, rows.size());
+        }
+
+        assertEquals(1, cache.get(1));
+        assertEquals(2, cache.get(2));
+        assertEquals(3, cache.get(3));
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java
index 4ae891d..8afc8fd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/baseline/ClientAffinityAssignmentWithBaselineTest.java
@@ -950,7 +950,6 @@
         ClusterTopologyException ex = X.cause(e, ClusterTopologyException.class);
         IgniteFuture f;
 
-        // For now in MVCC case the topology exception doesn't have a remap future.
         if (ex != null && (f = ex.retryReadyFuture()) != null)
             f.get();
     }
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 70ab8ac..29940f8 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
@@ -62,10 +62,9 @@
 import org.apache.ignite.internal.pagemem.wal.WALIterator;
 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.MarshalledDataEntry;
+import org.apache.ignite.internal.pagemem.wal.record.LazyDataEntry;
 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.UnwrappedDataEntry;
 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;
@@ -1657,12 +1656,12 @@
                             Object unwrappedKeyObj;
                             Object unwrappedValObj;
 
-                            if (entry instanceof UnwrappedDataEntry) {
-                                UnwrappedDataEntry unwrapDataEntry = (UnwrappedDataEntry)entry;
+                            if (entry instanceof UnwrapDataEntry) {
+                                UnwrapDataEntry unwrapDataEntry = (UnwrapDataEntry)entry;
                                 unwrappedKeyObj = unwrapDataEntry.unwrappedKey();
                                 unwrappedValObj = unwrapDataEntry.unwrappedValue();
                             }
-                            else if (entry instanceof MarshalledDataEntry) {
+                            else if (entry instanceof LazyDataEntry) {
                                 unwrappedKeyObj = null;
                                 unwrappedValObj = null;
                                 //can't check value
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
index 03312bb..09ac691 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
@@ -53,12 +53,6 @@
     }
 
     /** {@inheritDoc} */
-    @Override public void initialize(int cacheId, int partitions, String workingDir,
-        PageMetrics pageMetrics) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void initializeForCache(CacheGroupDescriptor grpDesc, CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
         // No-op.
     }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTracker.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTracker.java
index 9e4f2c8..5886b63 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTracker.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/wal/memtracker/PageMemoryTracker.java
@@ -559,7 +559,7 @@
      * @param checkAll Check all tracked pages, otherwise check until first error.
      * @param checkPageCnt Check tracked and allocated pages count. This check can be done only if there is no
      * concurrent modification of pages in the system (for example when checkpointWriteLock is held). Some threads
-     * (for example MVCC vacuum cleaner) can modify pages even if there is no activity from a users point of view.
+     * can modify pages even if there is no activity from a users point of view.
      * @return {@code true} if content of all tracked pages equals to content of these pages in the ignite instance.
      */
     private boolean checkPages(boolean checkAll, boolean checkPageCnt) throws IgniteCheckedException {
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorExternalizableFailedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorExternalizableFailedTest.java
index 75f9970..c4e627b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorExternalizableFailedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorExternalizableFailedTest.java
@@ -559,27 +559,6 @@
         }
     }
 
-    /** */
-    @SuppressWarnings({"unchecked", "ThrowableNotThrown"})
-    private void checkExplicitMvccInvoke(Ignite node, IgniteCache cache, TransactionConcurrency txConcurrency,
-        TransactionIsolation txIsolation) {
-        try (final Transaction tx = node.transactions().txStart(txConcurrency, txIsolation)) {
-            cache.put(KEY, WRONG_VALUE);
-
-            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    cache.invoke(KEY, createEntryProcessor());
-
-                    fail("Should never happened.");
-
-                    tx.commit();
-
-                    return null;
-                }
-            }, UnsupportedOperationException.class);
-        }
-    }
-
     /**
      * @return Entry processor.
      */
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorNonSerializableTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorNonSerializableTest.java
index 277172c..c8ea0b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorNonSerializableTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheEntryProcessorNonSerializableTest.java
@@ -412,32 +412,6 @@
     }
 
     /**
-     * @param node Grid node.
-     * @param cache Node cache.
-     * @param txConcurrency Transaction concurrency.
-     * @param txIsolation Transaction isolation.
-     */
-    @SuppressWarnings({"unchecked", "ThrowableNotThrown"})
-    private void checkMvccInvoke(Ignite node, IgniteCache cache, TransactionConcurrency txConcurrency,
-        TransactionIsolation txIsolation) {
-        try (final Transaction tx = node.transactions().txStart(txConcurrency, txIsolation)) {
-            cache.put(KEY, WRONG_VALUE);
-
-            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
-                @Override public Object call() {
-                    cache.invoke(KEY, new NonSerialazibleEntryProcessor());
-
-                    fail("Should never happened.");
-
-                    tx.commit();
-
-                    return null;
-                }
-            }, NotSerializableException.class);
-        }
-    }
-
-    /**
      * @return Cache configuration.
      */
     private CacheConfiguration<?, ?> cacheConfiguration(CacheWriteSynchronizationMode wrMode, int backup) {
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 0c4a3ad..6697225 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
@@ -857,7 +857,6 @@
             assertNoLocks();
 
             assertEquals(x, tree.findOne(x).longValue());
-            checkIterate(tree, x, x, x, true);
 
             assertNoLocks();
 
@@ -872,15 +871,12 @@
 
         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);
 
         boolean rmvRange = U.safeAbs(RMV_INC) > 1;
 
@@ -907,7 +903,6 @@
 
             assertNull(tree.findOne(x));
             assertNull(tree.findOne(x2));
-            checkIterate(tree, x, x2, null, false);
 
             assertNoLocks();
 
@@ -924,40 +919,6 @@
     }
 
     /**
-     * @param tree Tree.
-     * @param lower Lower bound.
-     * @param upper Upper bound.
-     * @param exp Value to find.
-     * @param expFound {@code True} if value should be found.
-     * @throws IgniteCheckedException If failed.
-     */
-    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);
-    }
-
-    /**
-     * @param tree Tree.
-     * @param lower Lower bound.
-     * @param upper Upper bound.
-     * @param c Closure.
-     * @param expFound {@code True} if value should be found.
-     * @throws IgniteCheckedException If failed.
-     */
-    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.
      */
     @Test
@@ -2537,37 +2498,6 @@
      * @throws Exception If failed.
      */
     @Test
-    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.
-     */
-    @Test
     public void testIterateConcurrentPutRemove() throws Exception {
         iterateConcurrentPutRemove(false);
     }
@@ -2714,45 +2644,6 @@
             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);
@@ -2784,8 +2675,6 @@
                 stop.set(true);
             }
 
-            asyncRunFut.get();
-
             stop.set(false);
         }
     }
@@ -3091,17 +2980,6 @@
 
                         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;
@@ -3438,53 +3316,6 @@
     /**
      *
      */
-    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;
-        }
-    }
-
-    /**
-     *
-     */
     static class TestTreeFindFilteredClosure implements BPlusTree.TreeRowClosure<Long, Long> {
         /** */
         private final Set<Long> vals;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListSelfTest.java
index abe7091..5d778b6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListSelfTest.java
@@ -611,11 +611,6 @@
         }
 
         /** {@inheritDoc} */
-        @Override public int headerSize() {
-            return 0;
-        }
-
-        /** {@inheritDoc} */
         @Override public long link() {
             return link;
         }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserMultiStatementSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserMultiStatementSelfTest.java
index 20fe284..1545c99 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserMultiStatementSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/sql/SqlParserMultiStatementSelfTest.java
@@ -45,9 +45,9 @@
         assertEquals("CREATE INDEX TEST on TABLE1(id)", parser.lastCommandSql());
         assertEquals(" ;   DROP USER test   ;;;", parser.remainingSql());
 
-        SqlCommand begin = parser.nextCommand();
+        SqlCommand dropUser = parser.nextCommand();
 
-        assertTrue(begin instanceof SqlDropUserCommand);
+        assertTrue(dropUser instanceof SqlDropUserCommand);
         assertEquals("DROP USER test", parser.lastCommandSql());
         assertEquals(";;", parser.remainingSql());
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
index 263e102..d56a96f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
@@ -240,8 +240,6 @@
         IgniteInternalFuture fut = null;
 
         try {
-            finishActiveTxIfNecessary();
-
             if (cmdH2 instanceof GridSqlCreateTable) {
                 GridSqlCreateTable cmd = (GridSqlCreateTable)cmdH2;
 
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 eb06429..f355dd7 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
@@ -1946,7 +1946,6 @@
 
             GridCacheContext<?, ?> cctx = plan.cacheContext();
 
-            // For MVCC case, let's enlist batch elements one by one.
             if (plan.hasRows() && plan.mode() == UpdateMode.INSERT) {
                 CacheOperationContext opCtx = DmlUtils.setKeepBinaryContext(cctx);
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
index 949568b..38ca9ea 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
@@ -591,7 +591,6 @@
             plan = UpdatePlanBuilder.planForStatement(
                 planKey,
                 stmt,
-                false,
                 idx,
                 log,
                 forceFillAbsentPKsWithDefaults
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
index 84d70ec..99910b4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlAstUtils.java
@@ -362,7 +362,7 @@
 
         GridSqlElement where = update.where();
 
-        // On no MVCC mode we cannot use lazy mode when UPDATE query contains index with updated columns
+        // We cannot use lazy mode when UPDATE query contains index with updated columns
         // and that index may be chosen to scan by WHERE condition
         // because in this case any rows update may be updated several times.
         // e.g. in the cases below we cannot use lazy mode:
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlDistributedPlanInfo.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlDistributedPlanInfo.java
index bf1f2dd..f7af9e6 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlDistributedPlanInfo.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/DmlDistributedPlanInfo.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.query.h2.dml;
 
 import java.util.List;
-import org.apache.ignite.internal.sql.optimizer.affinity.PartitionResult;
 
 /**
  * Additional information about distributed update plan.
@@ -30,20 +29,15 @@
     /** Identifiers of caches involved in update (used for cluster nodes mapping). */
     private final List<Integer> cacheIds;
 
-    /** Derived partitions info. */
-    private final PartitionResult derivedParts;
-
     /**
      * Constructor.
      * @param replicatedOnly Whether all caches are replicated.
      * @param cacheIds List of cache identifiers.
-     * @param derivedParts PartitionNode tree to calculate derived partition
      *      (reference to PartitionNode#apply(java.lang.Object...)).
      */
-    public DmlDistributedPlanInfo(boolean replicatedOnly, List<Integer> cacheIds, PartitionResult derivedParts) {
+    public DmlDistributedPlanInfo(boolean replicatedOnly, List<Integer> cacheIds) {
         this.replicatedOnly = replicatedOnly;
         this.cacheIds = cacheIds;
-        this.derivedParts = derivedParts;
     }
 
     /**
@@ -59,11 +53,4 @@
     public List<Integer> getCacheIds() {
         return cacheIds;
     }
-
-    /**
-     * @return Query derived partitions info.
-     */
-    public PartitionResult derivedPartitions() {
-        return derivedParts;
-    }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
index b052da8..9ce39f0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/FastUpdate.java
@@ -21,7 +21,6 @@
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.query.h2.UpdateResult;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlElement;
-import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -102,20 +101,4 @@
 
         return res ? UpdateResult.ONE : UpdateResult.ZERO;
     }
-
-    /**
-     *
-     * @param args Query Parameters.
-     * @return Key and value.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgniteBiTuple getRow(Object[] args) throws IgniteCheckedException {
-        Object key = keyArg.get(args);
-
-        assert key != null;
-
-        Object newVal = newValArg.get(args);
-
-        return new IgniteBiTuple(key, newVal);
-    }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
index 68b7dae..22e3743 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/dml/UpdatePlanBuilder.java
@@ -83,7 +83,7 @@
     /** Allow hidden key value columns at the INSERT/UPDATE/MERGE statements (not final for tests). */
     private static boolean ALLOW_KEY_VAL_UPDATES = IgniteSystemProperties.getBoolean(
         IgniteSystemProperties.IGNITE_SQL_ALLOW_KEY_VAL_UPDATES, false);
-    
+
     /**
      * Constructor.
      */
@@ -97,7 +97,6 @@
      *
      * @param planKey Plan key.
      * @param stmt Statement.
-     * @param mvccEnabled MVCC enabled flag.
      * @param idx Indexing.
      * @param forceFillAbsentPKsWithDefaults ForceFillAbsentPKsWithDefaults enabled flag.
      * @return Update plan.
@@ -106,15 +105,14 @@
     public static UpdatePlan planForStatement(
         QueryDescriptor planKey,
         GridSqlStatement stmt,
-        boolean mvccEnabled,
         IgniteH2Indexing idx,
         IgniteLogger log,
         boolean forceFillAbsentPKsWithDefaults
     ) throws IgniteCheckedException {
         if (stmt instanceof GridSqlMerge || stmt instanceof GridSqlInsert)
-            return planForInsert(planKey, stmt, idx, mvccEnabled, log, forceFillAbsentPKsWithDefaults);
+            return planForInsert(planKey, stmt, idx, log, forceFillAbsentPKsWithDefaults);
         else if (stmt instanceof GridSqlUpdate || stmt instanceof GridSqlDelete)
-            return planForUpdate(planKey, stmt, idx, mvccEnabled, log);
+            return planForUpdate(planKey, stmt, idx, log);
         else
             throw new IgniteSQLException("Unsupported operation: " + stmt.getSQL(),
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
@@ -126,7 +124,6 @@
      * @param planKey Plan key.
      * @param stmt INSERT or MERGE statement.
      * @param idx Indexing.
-     * @param mvccEnabled Mvcc flag.
      * @return Update plan.
      * @throws IgniteCheckedException if failed.
      */
@@ -135,7 +132,6 @@
         QueryDescriptor planKey,
         GridSqlStatement stmt,
         IgniteH2Indexing idx,
-        boolean mvccEnabled,
         IgniteLogger log,
         boolean forceFillAbsentPKsWithDefaults
     ) throws IgniteCheckedException {
@@ -232,13 +228,13 @@
         Set<String> rowKeys = desc.getRowKeyColumnNames();
 
         boolean onlyVisibleColumns = true;
-        
+
         for (int i = 0; i < cols.length; i++) {
             GridSqlColumn col = cols[i];
 
             if (!col.column().getVisible())
                 onlyVisibleColumns = false;
-            
+
             String colName = col.columnName();
 
             colNames[i] = colName;
@@ -270,12 +266,12 @@
             else
                 hasValProps = true;
         }
-    
+
         rowKeys.removeIf(rowKey -> desc.type().property(rowKey).defaultValue() != null);
-        
+
         boolean fillAbsentPKsWithNullsOrDefaults = type.fillAbsentPKsWithDefaults()
                 || forceFillAbsentPKsWithDefaults;
-        
+
         if (fillAbsentPKsWithNullsOrDefaults && onlyVisibleColumns && !rowKeys.isEmpty()) {
             String[] extendedColNames = new String[rowKeys.size() + colNames.length];
             int[] extendedColTypes = new int[rowKeys.size() + colTypes.length];
@@ -310,7 +306,6 @@
         if (rowsNum == 0 && !F.isEmpty(selectSql)) {
             distributed = checkPlanCanBeDistributed(
                 idx,
-                mvccEnabled,
                 planKey,
                 selectSql,
                 tbl.dataTable().cacheName(),
@@ -389,7 +384,6 @@
      * @param planKey Plan key.
      * @param stmt UPDATE or DELETE statement.
      * @param idx Indexing.
-     * @param mvccEnabled MVCC flag.
      * @return Update plan.
      * @throws IgniteCheckedException if failed.
      */
@@ -397,7 +391,6 @@
         QueryDescriptor planKey,
         GridSqlStatement stmt,
         IgniteH2Indexing idx,
-        boolean mvccEnabled,
         IgniteLogger log
     ) throws IgniteCheckedException {
         GridSqlElement target;
@@ -493,7 +486,6 @@
                 if (!F.isEmpty(selectSql)) {
                     distributed = checkPlanCanBeDistributed(
                         idx,
-                        mvccEnabled,
                         planKey,
                         selectSql,
                         tbl.dataTable().cacheName(),
@@ -530,7 +522,6 @@
                 if (!F.isEmpty(selectSql)) {
                     distributed = checkPlanCanBeDistributed(
                         idx,
-                        mvccEnabled,
                         planKey,
                         selectSql,
                         tbl.dataTable().cacheName(),
@@ -650,7 +641,7 @@
      * @param colIdx Column index if key or value is present in columns list, {@code -1} if it's not.
      * @param hasProps Whether column list affects individual properties of key or value.
      * @param key Whether supplier should be created for key or for value.
-     * @param forUpdate {@code FOR UPDATE} flag.
+     * @param forUpdate {@code true} if called for {@code UPDATE} statement.
      * @return Closure returning key or value.
      * @throws IgniteCheckedException If failed.
      */
@@ -932,7 +923,6 @@
      * Checks whether the given update plan can be distributed and returns additional info.
      *
      * @param idx Indexing.
-     * @param mvccEnabled Mvcc flag.
      * @param planKey Plan key.
      * @param selectQry Derived select query.
      * @param cacheName Cache name.
@@ -941,14 +931,13 @@
      */
     private static DmlDistributedPlanInfo checkPlanCanBeDistributed(
         IgniteH2Indexing idx,
-        boolean mvccEnabled,
         QueryDescriptor planKey,
         String selectQry,
         String cacheName,
         IgniteLogger log
     )
         throws IgniteCheckedException {
-        if ((!mvccEnabled && !planKey.skipReducerOnUpdate()) || planKey.batched())
+        if (!planKey.skipReducerOnUpdate() || planKey.batched())
             return null;
 
         try (H2PooledConnection conn = idx.connections().connection(planKey.schemaName())) {
@@ -986,7 +975,7 @@
 
                     H2Utils.checkQuery(idx, cacheIds, qry.tables());
 
-                    return new DmlDistributedPlanInfo(qry.isReplicatedOnly(), cacheIds, qry.derivedPartitions());
+                    return new DmlDistributedPlanInfo(qry.isReplicatedOnly(), cacheIds);
                 }
                 else
                     return null;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
index 5046e9d..c9b4db1 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2CacheRow.java
@@ -264,11 +264,6 @@
     }
 
     /** {@inheritDoc} */
-    @Override public int headerSize() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         SB sb = new SB("Row@");
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index aa9923b..3d38856 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -351,7 +351,8 @@
             IoStatisticsQueryHelper.startGatheringQueryStatistics();
 
         // Prepare to run queries.
-        GridCacheContext<?, ?> mainCctx = mainCacheContext(cacheIds);
+        GridCacheContext<?, ?> mainCctx = !F.isEmpty(cacheIds) ? ctx.cache().context().cacheContext(cacheIds.get(0))
+            : null;
 
         MapNodeResults nodeRess = resultsForNode(node.id());
 
@@ -614,14 +615,6 @@
     }
 
     /**
-     * @param cacheIds Cache ids.
-     * @return Id of the first cache in list, or {@code null} if list is empty.
-     */
-    private GridCacheContext mainCacheContext(List<Integer> cacheIds) {
-        return !F.isEmpty(cacheIds) ? ctx.cache().context().cacheContext(cacheIds.get(0)) : null;
-    }
-
-    /**
      * Releases reserved partitions.
      *
      * @param qctx Query context.
diff --git a/modules/core/src/main/java/org/apache/ignite/transactions/TransactionUnsupportedConcurrencyException.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionIndexingSelfTest.java
similarity index 63%
rename from modules/core/src/main/java/org/apache/ignite/transactions/TransactionUnsupportedConcurrencyException.java
rename to modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionIndexingSelfTest.java
index c8ce9bc..990c161 100644
--- a/modules/core/src/main/java/org/apache/ignite/transactions/TransactionUnsupportedConcurrencyException.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionIndexingSelfTest.java
@@ -15,21 +15,18 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.transactions;
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.configuration.QueryEngineConfiguration;
+import org.apache.ignite.indexing.IndexingQueryEngineConfiguration;
 
 /**
- * Exception thrown whenever transaction concurrency level is not supported.
+ *
  */
-public class TransactionUnsupportedConcurrencyException extends TransactionException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates new exception with given error message.
-     *
-     * @param msg Error message.
-     */
-    public TransactionUnsupportedConcurrencyException(String msg) {
-        super(msg);
+public class DdlTransactionIndexingSelfTest extends DdlTransactionSelfTestBase {
+    /** {@inheritDoc} */
+    @Override protected QueryEngineConfiguration getQueryEngineConfiguration() {
+        return new IndexingQueryEngineConfiguration()
+            .setDefault(true);
     }
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTest.java
deleted file mode 100644
index 50c650b..0000000
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/DdlTransactionSelfTest.java
+++ /dev/null
@@ -1,323 +0,0 @@
-/*
- * 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;
-
-import java.util.List;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.TransactionConfiguration;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.transactions.Transaction;
-import org.apache.ignite.transactions.TransactionConcurrency;
-import org.apache.ignite.transactions.TransactionIsolation;
-import org.apache.ignite.transactions.TransactionState;
-import org.junit.Test;
-
-/**
- *
- */
-public class DdlTransactionSelfTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setTransactionConfiguration(new TransactionConfiguration()
-            .setDefaultTxIsolation(TransactionIsolation.REPEATABLE_READ)
-            .setDefaultTxConcurrency(TransactionConcurrency.PESSIMISTIC)
-            .setDefaultTxTimeout(5000));
-
-        cfg.setCacheConfiguration(getCacheConfiguration());
-
-        return cfg;
-    }
-
-    /**
-     * @return Cache configuration.
-     */
-    private CacheConfiguration getCacheConfiguration() {
-        CacheConfiguration<?, ?> ccfg = defaultCacheConfiguration();
-
-        ccfg.setNearConfiguration(null);
-
-        return ccfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testTxIsCommittedOnDdlRequestMultinodeClient() throws Exception {
-        startGridsMultiThreaded(4, false);
-
-        Ignite node = startClientGrid(4);
-
-        awaitPartitionMapExchange();
-
-        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
-
-        try (Transaction tx = node.transactions().txStart()) {
-            cache.putAll(F.asMap(1, 1, 2, 2, 3, 3));
-
-            try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery(
-                "CREATE TABLE " +
-                    "    person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" +
-                    "WITH " +
-                    "    \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) {
-
-                assertNotNull(cur);
-
-                List<List<?>> rows = cur.getAll();
-
-                assertEquals(1, rows.size());
-
-                assertEquals(0L, rows.get(0).get(0));
-            }
-
-            assertTrue(tx.state() == TransactionState.COMMITTED);
-        }
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) {
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(0, rows.size());
-        }
-
-        assertEquals(1, cache.get(1));
-        assertEquals(2, cache.get(2));
-        assertEquals(3, cache.get(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testTxIsCommittedOnDdlRequestMultinode() throws Exception {
-        Ignite node = startGridsMultiThreaded(4);
-
-        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
-
-        try (Transaction tx = node.transactions().txStart()) {
-            cache.putAll(F.asMap(1, 1, 2, 2, 3, 3));
-
-            try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery(
-                "CREATE TABLE " +
-                    "    person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" +
-                    "WITH " +
-                    "    \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) {
-
-                assertNotNull(cur);
-
-                List<List<?>> rows = cur.getAll();
-
-                assertEquals(1, rows.size());
-
-                assertEquals(0L, rows.get(0).get(0));
-            }
-
-            assertTrue(tx.state() == TransactionState.COMMITTED);
-        }
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) {
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(0, rows.size());
-        }
-
-        assertEquals(1, cache.get(1));
-        assertEquals(2, cache.get(2));
-        assertEquals(3, cache.get(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testTxIsCommittedOnDdlRequest() throws Exception {
-        Ignite node = startGrid();
-
-        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
-
-        try (Transaction tx = node.transactions().txStart()) {
-            cache.putAll(F.asMap(1, 1, 2, 2, 3, 3));
-
-            try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery(
-                "CREATE TABLE " +
-                    "    person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" +
-                    "WITH " +
-                    "    \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) {
-
-                assertNotNull(cur);
-
-                List<List<?>> rows = cur.getAll();
-
-                assertEquals(1, rows.size());
-
-                assertEquals(0L, rows.get(0).get(0));
-            }
-
-            assertTrue(tx.state() == TransactionState.COMMITTED);
-        }
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) {
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(0, rows.size());
-        }
-
-        assertEquals(1, cache.get(1));
-        assertEquals(2, cache.get(2));
-        assertEquals(3, cache.get(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testDdlRequestWithoutTxMultinodeClient() throws Exception {
-        startGridsMultiThreaded(4, false);
-
-        Ignite node = startClientGrid(4);
-
-        awaitPartitionMapExchange();
-
-        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
-
-        cache.putAll(F.asMap(1, 1, 2, 2, 3, 3));
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery(
-            "CREATE TABLE " +
-                "    person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" +
-                "WITH " +
-                "    \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) {
-
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(1, rows.size());
-
-            assertEquals(0L, rows.get(0).get(0));
-        }
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) {
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(0, rows.size());
-        }
-
-        assertEquals(1, cache.get(1));
-        assertEquals(2, cache.get(2));
-        assertEquals(3, cache.get(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testDdlRequestWithoutTxMultinode() throws Exception {
-        Ignite node = startGridsMultiThreaded(4);
-
-        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
-
-        cache.putAll(F.asMap(1, 1, 2, 2, 3, 3));
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery(
-            "CREATE TABLE " +
-                "    person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" +
-                "WITH " +
-                "    \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) {
-
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(1, rows.size());
-
-            assertEquals(0L, rows.get(0).get(0));
-        }
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) {
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(0, rows.size());
-        }
-
-        assertEquals(1, cache.get(1));
-        assertEquals(2, cache.get(2));
-        assertEquals(3, cache.get(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @Test
-    public void testDdlRequestWithoutTx() throws Exception {
-        Ignite node = startGrid();
-
-        IgniteCache<Object, Object> cache = node.cache(DEFAULT_CACHE_NAME);
-
-        cache.putAll(F.asMap(1, 1, 2, 2, 3, 3));
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery(
-            "CREATE TABLE " +
-                "    person (id int, name varchar, age int, company varchar, city varchar, primary key (id, name, city))" +
-                "WITH " +
-                "    \"template=PARTITIONED,atomicity=TRANSACTIONAL,affinity_key=city\"").setSchema("PUBLIC"))) {
-
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(1, rows.size());
-
-            assertEquals(0L, rows.get(0).get(0));
-        }
-
-        try (FieldsQueryCursor<List<?>> cur = cache.query(new SqlFieldsQuery("SELECT * FROM person").setSchema("PUBLIC"))) {
-            assertNotNull(cur);
-
-            List<List<?>> rows = cur.getAll();
-
-            assertEquals(0, rows.size());
-        }
-
-        assertEquals(1, cache.get(1));
-        assertEquals(2, cache.get(2));
-        assertEquals(3, cache.get(3));
-    }
-}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java
index 3cc5afa..6f66104 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildSelfTest.java
@@ -139,15 +139,8 @@
      *     <li>Restart the node and block index rebuild;</li>
      *     <li>For half of the keys do cache puts <b>before</b> corresponding key
      *     has been processed during index rebuild;</li>
-     *     <li>Check that:
-     *         <ul>
-     *             <li>For MVCC case: some keys have all versions that existed before restart, while those
-     *             updated concurrently have only put version (one with mark value -1)
-     *             and latest version present before node restart;</li>
-     *             <li>For non MVCC case: keys updated concurrently must have mark values of -1 despite that
-     *             index rebuild for them has happened after put.</li>
-     *         </ul>
-     *     </li>
+     *     <li>Check that keys updated concurrently must have mark values of -1 despite that
+     *     index rebuild for them has happened after put.</li>
      * </ul></p>
      * @throws Exception if failed.
      */
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/QueryDataPageScanTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/QueryDataPageScanTest.java
index 095b164..0f424a5 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/QueryDataPageScanTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/QueryDataPageScanTest.java
@@ -214,7 +214,6 @@
                 if (accountId1 == accountId2)
                     continue;
 
-                // Sort to avoid MVCC deadlock.
                 if (accountId1 > accountId2) {
                     long tmp = accountId1;
                     accountId1 = accountId2;
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index d1e124e..0c84f0ed 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -22,7 +22,7 @@
 import org.apache.ignite.internal.processors.cache.AffinityKeyNameAndValueFieldNameConflictTest;
 import org.apache.ignite.internal.processors.cache.CacheOffheapBatchIndexingMultiTypeTest;
 import org.apache.ignite.internal.processors.cache.CacheQueryBuildValueTest;
-import org.apache.ignite.internal.processors.cache.DdlTransactionSelfTest;
+import org.apache.ignite.internal.processors.cache.DdlTransactionIndexingSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheCrossCacheQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheLazyQueryPartitionsReleaseTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
@@ -339,7 +339,7 @@
     H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest.class,
     H2DynamicIndexingComplexServerTransactionalReplicatedTest.class,
 
-    DdlTransactionSelfTest.class,
+    DdlTransactionIndexingSelfTest.class,
 
 })
 public class IgniteBinaryCacheQueryTestSuite {
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
index c2158ab..a8fe79d 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
@@ -95,15 +95,9 @@
                 /** Invalid cursor state. */
                 S24000_INVALID_CURSOR_STATE,
 
-                /** Invalid transaction state. */
-                S25000_INVALID_TRANSACTION_STATE,
-
                 /** Invalid schema name. */
                 S3F000_INVALID_SCHEMA_NAME,
 
-                /** Serialization failure. */
-                S40001_SERIALIZATION_FAILURE,
-
                 /** Syntax error or access violation. */
                 S42000_SYNTAX_ERROR_OR_ACCESS_VIOLATION,
 
@@ -380,13 +374,7 @@
                 ENTRY_PROCESSING = 4005,
 
                 /** Cache not found. */
-                CACHE_NOT_FOUND = 4006,
-
-                /** Transaction is already completed. */
-                TRANSACTION_COMPLETED = 5004,
-
-                /** Transaction serialization error. */
-                TRANSACTION_SERIALIZATION_ERROR = 5005
+                CACHE_NOT_FOUND = 4006
             };
         };
 
diff --git a/modules/platforms/cpp/odbc/src/common_types.cpp b/modules/platforms/cpp/odbc/src/common_types.cpp
index 72b3cc6..ea8a23d 100644
--- a/modules/platforms/cpp/odbc/src/common_types.cpp
+++ b/modules/platforms/cpp/odbc/src/common_types.cpp
@@ -157,12 +157,6 @@
                 case ResponseStatus::COLUMN_ALREADY_EXISTS:
                     return SqlState::S42S21_COLUMN_ALREADY_EXISTS;
 
-                case ResponseStatus::TRANSACTION_COMPLETED:
-                    return SqlState::S25000_INVALID_TRANSACTION_STATE;
-
-                case ResponseStatus::TRANSACTION_SERIALIZATION_ERROR:
-                    return SqlState::S40001_SERIALIZATION_FAILURE;
-
                 case ResponseStatus::CACHE_NOT_FOUND:
                 case ResponseStatus::NULL_TABLE_DESCRIPTOR:
                 case ResponseStatus::CONVERSION_FAILED:
diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
index d3a3692..7b82b40 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
@@ -79,15 +79,9 @@
     /** SQL state 24000 constant. */
     const std::string STATE_24000 = "24000";
 
-    /** SQL state 25000 constant. */
-    const std::string STATE_25000 = "25000";
-
     /** SQL state 3F000 constant. */
     const std::string STATE_3F000 = "3F000";
 
-    /** SQL state 40001 constant. */
-    const std::string STATE_40001 = "40001";
-
     /** SQL state 42000 constant. */
     const std::string STATE_42000 = "42000";
 
@@ -311,15 +305,9 @@
                     case SqlState::S24000_INVALID_CURSOR_STATE:
                         return STATE_24000;
 
-                    case SqlState::S25000_INVALID_TRANSACTION_STATE:
-                        return STATE_25000;
-
                     case SqlState::S3F000_INVALID_SCHEMA_NAME:
                         return STATE_3F000;
 
-                    case SqlState::S40001_SERIALIZATION_FAILURE:
-                        return STATE_40001;
-
                     case SqlState::S42000_SYNTAX_ERROR_OR_ACCESS_VIOLATION:
                         return STATE_42000;