You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ir...@apache.org on 2020/07/13 18:00:44 UTC

[ignite] branch master updated: IGNITE-12320 Partial index rebuild fails in case indexed cache contains different datatypes - Fixes #8018

This is an automated email from the ASF dual-hosted git repository.

irakov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new b11a868  IGNITE-12320 Partial index rebuild fails in case indexed cache contains different datatypes - Fixes #8018
b11a868 is described below

commit b11a868a5803bf383c06a71c0d3ac368ab33618e
Author: alapin <la...@gmail.com>
AuthorDate: Mon Jul 13 21:00:02 2020 +0300

    IGNITE-12320 Partial index rebuild fails in case indexed cache contains different datatypes - Fixes #8018
    
    Signed-off-by: Ivan Rakov <iv...@gmail.com>
---
 .../processors/query/GridQueryProcessor.java       |  43 -----
 .../schema/SchemaIndexCachePartitionWorker.java    |  13 +-
 .../query/schema/SchemaIndexCacheVisitorImpl.java  |   8 +-
 .../processors/query/h2/IgniteH2Indexing.java      |   4 +-
 .../processors/query/h2/IndexBuildClosure.java     |  54 ------
 .../query/h2/IndexRebuildPartialClosure.java       |  24 ++-
 .../processors/query/h2/SchemaManager.java         |   6 +-
 .../cache/index/AbstractIndexingCommonTest.java    |   2 +-
 ...lRebuildTest.java => GridIndexRebuildTest.java} | 198 ++++++++++++++++++---
 .../IgniteBinaryCacheQueryTestSuite.java           |   4 +-
 10 files changed, 205 insertions(+), 151 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index b6cc81e..2da189c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -84,7 +84,6 @@ import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.query.property.QueryBinaryProperty;
-import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheFilter;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl;
@@ -111,13 +110,11 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.internal.util.lang.IgniteOutClosureX;
-import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
-import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -1860,7 +1857,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                     visitor = new SchemaIndexCacheVisitorImpl(
                         cacheInfo.cacheContext(),
-                        new TableCacheFilter(cctx, op0.tableName()),
                         cancelTok,
                         createIdxFut
                     ) {
@@ -3746,45 +3742,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
     }
 
-    /** */
-    private static class TableCacheFilter implements SchemaIndexCacheFilter {
-        /** */
-        @GridToStringExclude
-        private final GridCacheContext cctx;
-
-        /** */
-        @GridToStringExclude
-        private final GridQueryProcessor query;
-
-        /** */
-        private final String cacheName;
-
-        /** */
-        private final String tableName;
-
-        /**
-         * @param cctx Cache context.
-         * @param tableName Target table name.
-         */
-        TableCacheFilter(GridCacheContext cctx, String tableName) {
-            this.cctx = cctx;
-            this.tableName = tableName;
-
-            cacheName = cctx.name();
-            query = cctx.kernalContext().query();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean apply(CacheDataRow row) throws IgniteCheckedException {
-            return query.belongsToTable(cctx, cacheName, tableName, row.key(), row.value());
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(TableCacheFilter.class, this);
-        }
-    }
-
     /**
      * Function which can throw exception.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCachePartitionWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCachePartitionWorker.java
index 529d93f..b1162da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCachePartitionWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCachePartitionWorker.java
@@ -85,7 +85,6 @@ public class SchemaIndexCachePartitionWorker extends GridWorker {
      * @param cancel Cancellation token between all workers for all caches.
      * @param clo Index closure.
      * @param fut Worker future.
-     * @param rowFilter Row filter.
      * @param partsCnt Count of partitions to be processed.
      */
     public SchemaIndexCachePartitionWorker(
@@ -95,7 +94,6 @@ public class SchemaIndexCachePartitionWorker extends GridWorker {
         SchemaIndexOperationCancellationToken cancel,
         SchemaIndexCacheVisitorClosure clo,
         GridFutureAdapter<SchemaIndexCacheStat> fut,
-        @Nullable SchemaIndexCacheFilter rowFilter,
         AtomicInteger partsCnt
     ) {
         super(
@@ -114,7 +112,7 @@ public class SchemaIndexCachePartitionWorker extends GridWorker {
         assert nonNull(partsCnt);
 
         this.stop = stop;
-        wrappedClo = new SchemaIndexCacheVisitorClosureWrapper(clo, rowFilter);
+        wrappedClo = new SchemaIndexCacheVisitorClosureWrapper(clo);
         this.fut = fut;
         this.partsCnt = partsCnt;
     }
@@ -279,22 +277,17 @@ public class SchemaIndexCachePartitionWorker extends GridWorker {
         /** Object for collecting statistics about index update. */
         @Nullable private final SchemaIndexCacheStat indexCacheStat;
 
-        /** Row filter. */
-        @Nullable private final SchemaIndexCacheFilter rowFilter;
-
         /** */
         private SchemaIndexCacheVisitorClosureWrapper(
-            SchemaIndexCacheVisitorClosure clo,
-            @Nullable SchemaIndexCacheFilter filter
+            SchemaIndexCacheVisitorClosure clo
         ) {
             this.clo = clo;
             indexCacheStat = getBoolean(IGNITE_ENABLE_EXTRA_INDEX_REBUILD_LOGGING, false) ? new SchemaIndexCacheStat() : null;
-            rowFilter = filter;
         }
 
         /** {@inheritDoc} */
         @Override public void apply(CacheDataRow row) throws IgniteCheckedException {
-            if (row != null && (rowFilter == null || rowFilter.apply(row))) {
+            if (row != null) {
                 clo.apply(row);
 
                 if (indexCacheStat != null) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
index 09163b8..3b1fbb5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
@@ -54,9 +54,6 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
     /** Cache context. */
     private final GridCacheContext cctx;
 
-    /** Row filter. */
-    private final SchemaIndexCacheFilter rowFilter;
-
     /** Cancellation token. */
     private final SchemaIndexOperationCancellationToken cancel;
 
@@ -70,13 +67,11 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
      * Constructor.
      *
      * @param cctx Cache context.
-     * @param rowFilter Row filter.
      * @param cancel Cancellation token.
      * @param buildIdxFut Future for create/rebuild index.
      */
     public SchemaIndexCacheVisitorImpl(
         GridCacheContext cctx,
-        @Nullable SchemaIndexCacheFilter rowFilter,
         @Nullable SchemaIndexOperationCancellationToken cancel,
         GridFutureAdapter<Void> buildIdxFut
     ) {
@@ -90,7 +85,6 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
         this.buildIdxFut = buildIdxFut;
 
         this.cancel = cancel;
-        this.rowFilter = rowFilter;
 
         log = cctx.kernalContext().log(getClass());
     }
@@ -122,7 +116,7 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
             GridWorkerFuture<SchemaIndexCacheStat> workerFut = new GridWorkerFuture<>();
 
             GridWorker worker =
-                new SchemaIndexCachePartitionWorker(cctx, locPart, stop, cancel, clo, workerFut, rowFilter, partsCnt);
+                new SchemaIndexCachePartitionWorker(cctx, locPart, stop, cancel, clo, workerFut, partsCnt);
 
             workerFut.setWorker(worker);
             buildIdxCompoundFut.add(workerFut);
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 fdeb89c..d9ad530 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
@@ -1972,7 +1972,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
         else {
             // Otherwise iterate over tables looking for missing indexes.
-            IndexRebuildPartialClosure clo0 = new IndexRebuildPartialClosure();
+            IndexRebuildPartialClosure clo0 = new IndexRebuildPartialClosure(cctx);
 
             for (H2TableDescriptor tblDesc : schemaMgr.tablesForCache(cacheName)) {
                 GridH2Table tbl = tblDesc.table();
@@ -2031,7 +2031,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         SchemaIndexCacheVisitorClosure clo,
         GridFutureAdapter<Void> rebuildIdxFut
     ) {
-        new SchemaIndexCacheVisitorImpl(cctx, null, null, rebuildIdxFut).visit(clo);
+        new SchemaIndexCacheVisitorImpl(cctx, null, rebuildIdxFut).visit(clo);
     }
 
     /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IndexBuildClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IndexBuildClosure.java
deleted file mode 100644
index 065c561..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IndexBuildClosure.java
+++ /dev/null
@@ -1,54 +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.query.h2;
-
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
-import org.apache.ignite.internal.processors.query.h2.opt.H2CacheRow;
-import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
-
-/**
- * Index build closure.
- */
-public class IndexBuildClosure implements SchemaIndexCacheVisitorClosure {
-    /** Row descriptor. */
-    private final GridH2RowDescriptor rowDesc;
-
-    /** Index. */
-    private final GridH2IndexBase idx;
-
-    /**
-     * Constructor.
-     *
-     * @param rowDesc Row descriptor.
-     * @param idx Target index.
-     */
-    public IndexBuildClosure(GridH2RowDescriptor rowDesc, GridH2IndexBase idx) {
-        this.rowDesc = rowDesc;
-        this.idx = idx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void apply(CacheDataRow row) throws IgniteCheckedException {
-        H2CacheRow row0 = rowDesc.createRow(row);
-
-        idx.putx(row0);
-    }
-}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IndexRebuildPartialClosure.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IndexRebuildPartialClosure.java
index 6be91b5..55f8ada 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IndexRebuildPartialClosure.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IndexRebuildPartialClosure.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.IdentityHashMap;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
@@ -35,17 +36,32 @@ public class IndexRebuildPartialClosure implements SchemaIndexCacheVisitorClosur
     /** Indexes. */
     private final Map<GridH2Table, Collection<GridH2IndexBase>> tblIdxs = new IdentityHashMap<>();
 
+    /** Cache context. */
+    private GridCacheContext cctx;
+
+    /**
+     * Constructor.
+     *
+     * @param cctx Cache context.
+     */
+    public IndexRebuildPartialClosure(GridCacheContext cctx) {
+        this.cctx = cctx;
+    }
+
     /** {@inheritDoc} */
     @Override public void apply(CacheDataRow row) throws IgniteCheckedException {
         assert hasIndexes();
 
         for (Map.Entry<GridH2Table, Collection<GridH2IndexBase>> tblIdxEntry : tblIdxs.entrySet()) {
-            GridH2Table tbl = tblIdxEntry.getKey();
+            if (cctx.kernalContext().query().belongsToTable(cctx, tblIdxEntry.getKey().cacheName(),
+                tblIdxEntry.getKey().getName(), row.key(), row.value())) {
+                GridH2Table tbl = tblIdxEntry.getKey();
 
-            H2CacheRow row0 = tbl.rowDescriptor().createRow(row);
+                H2CacheRow row0 = tbl.rowDescriptor().createRow(row);
 
-            for (GridH2IndexBase idx : tblIdxEntry.getValue())
-                idx.putx(row0);
+                for (GridH2IndexBase idx : tblIdxEntry.getValue())
+                    idx.putx(row0);
+            }
         }
     }
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
index 8e10558..6091b4e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
@@ -653,9 +653,11 @@ public class SchemaManager {
 
         try {
             // Populate index with existing cache data.
-            final GridH2RowDescriptor rowDesc = h2Tbl.rowDescriptor();
+            IndexRebuildPartialClosure idxBuild = new IndexRebuildPartialClosure(h2Tbl.cacheContext());
 
-            cacheVisitor.visit(new IndexBuildClosure(rowDesc, h2Idx));
+            idxBuild.addIndex(h2Tbl, h2Idx);
+
+            cacheVisitor.visit(idxBuild);
 
             // At this point index is in consistent state, promote it through H2 SQL statement, so that cached
             // prepared statements are re-built.
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractIndexingCommonTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractIndexingCommonTest.java
index e499ca3..e3986ba 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractIndexingCommonTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/AbstractIndexingCommonTest.java
@@ -161,7 +161,7 @@ public class AbstractIndexingCommonTest extends GridCommonAbstractTest {
             new Thread(() -> {
                 startThread.countDown();
 
-                new SchemaIndexCacheVisitorImpl(cctx, null, null, rebuildIdxFut) {
+                new SchemaIndexCacheVisitorImpl(cctx, null, rebuildIdxFut) {
                     /** {@inheritDoc} */
                     @Override protected void beforeExecute() {
                         String cacheName = cctx.name();
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexFullRebuildTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildTest.java
similarity index 58%
rename from modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexFullRebuildTest.java
rename to modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildTest.java
index 1c5d134..26de2d6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexFullRebuildTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexRebuildTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
@@ -48,20 +49,24 @@ import org.apache.ignite.internal.visor.verify.VisorValidateIndexesJobResult;
 import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTask;
 import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskArg;
 import org.apache.ignite.internal.visor.verify.VisorValidateIndexesTaskResult;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy;
 import org.junit.Test;
 
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
 
 /**
- *
+ * Tesing index full and partial rebuild.
  */
-public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
+public class GridIndexRebuildTest extends GridCommonAbstractTest {
     public static final String FIRST_CACHE = "cache1";
 
     public static final String SECOND_CACHE = "cache2";
 
+    /** */
+    private final ListeningTestLogger listeningLog = new ListeningTestLogger(false, log);
+
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
@@ -122,17 +127,18 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
 
         configuration.setCacheConfiguration(ccfgFirst, ccfgSecond);
 
+        configuration.setGridLogger(listeningLog);
+
         return configuration;
     }
 
     /**
-     * We start several nodes, populate caches, then start replacing values.
-     * After that one node is killed, their index.bin files would be removed.
-     * Finally, we restart the node, index rebuild starting after recovery.
-     * And we checke indexes by "validate indexes" task.
+     * We start several nodes, populate caches, then start replacing values. After that one node is killed, their
+     * index.bin files would be removed. Finally, we restart the node, index rebuild starting after recovery. And we
+     * checke indexes by "validate indexes" task.
      */
     @Test
-    public void test() throws Exception {
+    public void testFullIndexRebuild() throws Exception {
 
         long start = System.currentTimeMillis();
 
@@ -140,10 +146,10 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
 
         grid1.cluster().active(true);
 
-        final int accountCount = 2048;
+        final int accountCnt = 2048;
 
         try (IgniteDataStreamer streamer = grid1.dataStreamer(FIRST_CACHE)) {
-            for (long i = 0; i < accountCount; i++) {
+            for (long i = 0; i < accountCnt; i++) {
                 streamer.addData(i, new Account(i));
             }
 
@@ -151,7 +157,7 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
         }
 
         try (IgniteDataStreamer streamer = grid1.dataStreamer(SECOND_CACHE)) {
-            for (long i = 0; i < accountCount; i++) {
+            for (long i = 0; i < accountCnt; i++) {
                 streamer.addData(i, new Account(i));
             }
 
@@ -171,7 +177,10 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
                     try {
                         cache1.put(i, new Account(i));
 
-                        cache2.put(i, new Account(i));
+                        if (i % 13 == 7)
+                            cache2.put(i, new Account2(i));
+                        else
+                            cache2.put(i, new Account(i));
 
                         i++;
                     }
@@ -182,17 +191,17 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
             }
         }).start();
 
-        File workDirectory = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
+        File workDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false);
 
         long diff = System.currentTimeMillis() - start;
 
         U.sleep(7500 - (diff % 5000));
 
-        IgniteProcessProxy.kill(getTestIgniteInstanceName(3));
+        stopGrid(3);
 
         stop.set(true);
 
-        for (File grp : new File(workDirectory, U.maskForFileName(getTestIgniteInstanceName(3))).listFiles()) {
+        for (File grp : new File(workDir, U.maskForFileName(getTestIgniteInstanceName(3))).listFiles()) {
             new File(grp, "index.bin").delete();
         }
 
@@ -202,19 +211,114 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
 
         U.sleep(3_000);
 
-        ImmutableSet<UUID> nodes = ImmutableSet.of(((IgniteProcessProxy)grid(3)).getId(),
-            ((IgniteProcessProxy)grid(2)).getId());
+        ImmutableSet<UUID> nodes = ImmutableSet.of(grid(2).localNode().id(), grid(3).localNode().id());
+
+        VisorValidateIndexesTaskArg arg = new VisorValidateIndexesTaskArg(null,
+            null, 10000, 1, true, true);
+
+        VisorTaskArgument<VisorValidateIndexesTaskArg> visorTaskArg = new VisorTaskArgument<>(nodes, arg, true);
+
+        ComputeTaskInternalFuture<VisorValidateIndexesTaskResult> exec = grid1.context().task().
+            execute(new VisorValidateIndexesTask(), visorTaskArg);
+
+        VisorValidateIndexesTaskResult res = exec.get();
+
+        Map<UUID, VisorValidateIndexesJobResult> results = res.results();
+
+        boolean hasIssue = false;
+
+        for (VisorValidateIndexesJobResult jobResult : results.values()) {
+            System.err.println(jobResult);
+
+            hasIssue |= jobResult.hasIssues();
+        }
+
+        assertFalse(hasIssue);
+    }
+
+    /**
+     * We start several nodes, populate caches, then start replacing values. After that one node is killed, new index
+     * created. Finally, we restart the node, index rebuild starting after recovery. And we checke indexes by "validate
+     * indexes" task.
+     */
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testPartialIndexRebuild() throws Exception {
+        LogListener lsnr = LogListener
+            .matches("B+Tree is corrupted")
+            .build();
+
+        listeningLog.registerListener(lsnr);
+
+        long start = System.currentTimeMillis();
+
+        IgniteEx grid1 = startGrids(4);
+
+        grid1.cluster().active(true);
+
+        final int accountCnt = 2048;
+
+        try (IgniteDataStreamer streamer = grid1.dataStreamer(SECOND_CACHE)) {
+            for (long i = 0; i < accountCnt; i++)
+                streamer.addData(i, new Account(i));
+
+            streamer.flush();
+        }
+
+        AtomicBoolean stop = new AtomicBoolean();
+
+        IgniteCache<Object, Object> cache2 = grid1.cache(SECOND_CACHE);
+
+        new Thread(new Runnable() {
+            @Override public void run() {
+                long i = 0;
+
+                while (!stop.get()) {
+                    try {
+                        if (i % 13 == 7)
+                            cache2.put(i, new Account2(i));
+                        else
+                            cache2.put(i, new Account(i));
+
+                        i++;
+                    }
+                    catch (Throwable e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+        }).start();
+
+        long diff = System.currentTimeMillis() - start;
+
+        U.sleep(7500 - (diff % 5000));
+
+        stopGrid(3);
+
+        stop.set(true);
+
+        cache2.query(new SqlFieldsQuery("CREATE INDEX idx" +
+            UUID.randomUUID().toString().replaceAll("-", "_") + " on Account (amount)")).getAll();
+
+        startGrid(3);
+
+        awaitPartitionMapExchange();
+
+        U.sleep(3_000);
+
+        ImmutableSet<UUID> nodes = ImmutableSet.of(grid(2).localNode().id(), grid(3).localNode().id());
 
         VisorValidateIndexesTaskArg arg = new VisorValidateIndexesTaskArg(null,
             null, 10000, 1, true, true);
 
-        VisorTaskArgument<VisorValidateIndexesTaskArg> argument = new VisorTaskArgument<>(nodes, arg, true);
+        VisorTaskArgument<VisorValidateIndexesTaskArg> visorTaskArg = new VisorTaskArgument<>(nodes, arg, true);
 
-        ComputeTaskInternalFuture<VisorValidateIndexesTaskResult> execute = grid1.context().task().execute(new VisorValidateIndexesTask(), argument);
+        ComputeTaskInternalFuture<VisorValidateIndexesTaskResult> execute = grid1.context().task().
+            execute(new VisorValidateIndexesTask(), visorTaskArg);
 
-        VisorValidateIndexesTaskResult result = execute.get();
+        VisorValidateIndexesTaskResult res = execute.get();
 
-        Map<UUID, VisorValidateIndexesJobResult> results = result.results();
+        Map<UUID, VisorValidateIndexesJobResult> results = res.results();
 
         boolean hasIssue = false;
 
@@ -225,6 +329,8 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
         }
 
         assertFalse(hasIssue);
+
+        assertFalse("B+Tree is corrupted.", lsnr.check());
     }
 
     /** */
@@ -246,12 +352,8 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
         U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), ig1DbWalPath, false));
     }
 
-    /** {@inheritDoc} */
-    @Override protected boolean isMultiJvm() {
-        return true;
-    }
-
     /** */
+    @SuppressWarnings("unused")
     public class Account {
         /** */
         private Long id;
@@ -289,4 +391,48 @@ public class GridIndexFullRebuildTest extends GridCommonAbstractTest {
             return Objects.hash(id);
         }
     }
+
+    /** */
+    @SuppressWarnings("unused")
+    public static class Account2 {
+        /** */
+        private Long id;
+
+        /** */
+        private String name2;
+
+        /** */
+        private Long Wamount2;
+
+        /** */
+        private Date updateDate2;
+
+        /**
+         * Constructor.
+         *
+         * @param id Account id.
+         */
+        public Account2(Long id) {
+            this.id = id;
+
+            name2 = "Account" + id;
+            Wamount2 = id * 1000;
+            updateDate2 = new Date();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+            Account account = (Account)o;
+            return Objects.equals(id, account.id);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return Objects.hash(id);
+        }
+    }
 }
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 854e06a..992eb03 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
@@ -230,8 +230,8 @@ import org.apache.ignite.internal.processors.query.SqlQueryHistoryFromClientSelf
 import org.apache.ignite.internal.processors.query.SqlQueryHistorySelfTest;
 import org.apache.ignite.internal.processors.query.SqlSchemaSelfTest;
 import org.apache.ignite.internal.processors.query.SqlSystemViewsSelfTest;
-import org.apache.ignite.internal.processors.query.h2.GridIndexFullRebuildTest;
 import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildSelfTest;
+import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildTest;
 import org.apache.ignite.internal.processors.query.h2.H2ResultSetIteratorNullifyOnEndSelfTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlQueryMinMaxTest;
@@ -533,7 +533,7 @@ import org.junit.runners.Suite;
     SqlSystemViewsSelfTest.class,
 
     GridIndexRebuildSelfTest.class,
-    GridIndexFullRebuildTest.class,
+    GridIndexRebuildTest.class,
     CheckIndexesInlineSizeOnNodeJoinMultiJvmTest.class,
 
     SqlTransactionCommandsWithMvccDisabledSelfTest.class,