You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ip...@apache.org on 2019/06/06 09:29:35 UTC

[ignite] branch master updated: IGNITE-11814 Log finish rebuilding all indexes - Fixes #6544.

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

ipavlukhin 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 3374b75  IGNITE-11814 Log finish rebuilding all indexes - Fixes #6544.
3374b75 is described below

commit 3374b7567ab0c125144e94dd8ac07ea6a71c771c
Author: denis-chudov <dc...@gridgain.com>
AuthorDate: Thu Jun 6 12:28:25 2019 +0300

    IGNITE-11814 Log finish rebuilding all indexes - Fixes #6544.
    
    Signed-off-by: ipavlukhin <vo...@gmail.com>
---
 .../GridCacheDatabaseSharedManager.java            |  11 +
 .../internal/util/GridCountDownCallback.java       | 114 +++++++++++
 .../persistence/CleanupRestoredCachesSlowTest.java |  60 +-----
 .../internal/util/GridCountDownCallbackTest.java   |  71 +++++++
 .../testframework/CallbackExecutorLogListener.java |  55 +++++
 .../testframework/MessageOrderLogListener.java     | 226 +++++++++++++++++++++
 .../testframework/MessageOrderLogListenerTest.java | 100 +++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java    |   3 +
 .../ignite/testsuites/IgniteUtilSelfTestSuite.java |   3 +
 .../persistence/RebuildIndexLogMessageTest.java    | 215 ++++++++++++++++++++
 .../IgniteCacheWithIndexingTestSuite.java          |   5 +-
 11 files changed, 809 insertions(+), 54 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index d45cd93..559309c 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -154,6 +154,7 @@ import org.apache.ignite.internal.processors.port.GridPortRecord;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.GridCountDownCallback;
 import org.apache.ignite.internal.util.GridMultiCollectionWrapper;
 import org.apache.ignite.internal.util.GridReadOnlyArrayView;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -1452,6 +1453,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         GridQueryProcessor qryProc = cctx.kernalContext().query();
 
         if (qryProc.moduleEnabled()) {
+            GridCountDownCallback rebuildIndexesCompleteCntr = new GridCountDownCallback(
+                cctx.cacheContexts().size(),
+                () -> log().info("Indexes rebuilding completed for all caches."),
+                1  //need at least 1 index rebuilded to print message about rebuilding completion
+            );
+
             for (final GridCacheContext cacheCtx : (Collection<GridCacheContext>)cctx.cacheContexts()) {
                 if (cacheCtx.startTopologyVersion().equals(fut.initialVersion())) {
                     final int cacheId = cacheCtx.cacheId();
@@ -1485,6 +1492,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                                                 + ", grpName=" + ccfg.getGroupName() + ']', err);
                                     }
                                 }
+
+                                rebuildIndexesCompleteCntr.countDown(true);
                             }
                         });
                     }
@@ -1493,6 +1502,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                             idxRebuildFuts.remove(cacheId, usrFut);
 
                             usrFut.onDone();
+
+                            rebuildIndexesCompleteCntr.countDown(false);
                         }
                     }
                 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridCountDownCallback.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridCountDownCallback.java
new file mode 100644
index 0000000..97a73ed
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridCountDownCallback.java
@@ -0,0 +1,114 @@
+/*
+ * 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.util;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Allows to execute callback when a set of operations will be completed. Also has an execution counter,
+ * which allows to block callback execution in case if it is below given threshold. By default, threshold
+ * is 0 and nothing blocks callback execution.
+ *
+ * <p>
+ * <b>Sample usage:</b>:
+ * <pre>{@code
+ *     GridCountDownCallback countDownCb = new GridCountDownCallback(
+ *         n,                     //internal counter is initiated with n
+ *         () -> doSomething()    //callback
+ *     );
+ *
+ *     //each call of countDown() decrements internal counter
+ *     //doSomething() will be executed after counter reaches 0
+ *     for (int i = 0; i < n; i++)
+ *         new Thread(() -> countDownCb.countDown()).start();
+ * }</pre>
+ *
+ * <p>
+ * <b>Usage with execution threshold:</b>:
+ * <pre>{@code
+ *     GridCountDownCallback countDownCb = new GridCountDownCallback(
+ *         n,                     //internal counter is initiated with n
+ *         () -> doSomething(),   //callback
+ *         n/2                    //execution threshold is initiated with n/2
+ *     );
+ *
+ *     //a half of calls of countDown() increase execution counter, so it reaches threshold and callback executes.
+ *     //doSomething() will be executed after n threads will perform countDown()
+ *     for (int i = 0; i < n; i++)
+ *         new Thread(() -> countDownCb.countDown(n % 2 == 0)).start();
+ * }</pre>
+ */
+public class GridCountDownCallback {
+    /** */
+    private final AtomicInteger cntr;
+
+    /** */
+    private final int executionThreshold;
+
+    /** */
+    private final AtomicInteger executionCntr = new AtomicInteger(0);
+
+    /** */
+    private final Runnable cb;
+
+    /**
+     * Constructor.
+     *
+     * @param initCnt count of invocations of {@link #countDown}.
+     * @param cb callback which will be executed after <code>initialCount</code>
+     * invocations of {@link #countDown}.
+     * @param executionThreshold minimal count of really performed operations to execute callback.
+     */
+    public GridCountDownCallback(int initCnt, Runnable cb, int executionThreshold) {
+        cntr = new AtomicInteger(initCnt);
+
+        this.executionThreshold = executionThreshold;
+
+        this.cb = cb;
+    }
+
+    /**
+     * Constructor. Execution threshold is set to 0.
+     *
+     * @param initCnt count of invocations of {@link #countDown}.
+     * @param cb callback which will be executed after <code>initialCount</code>
+     * invocations of {@link #countDown}.
+     */
+    public GridCountDownCallback(int initCnt, Runnable cb) {
+        this(initCnt, cb, 0);
+    }
+
+    /**
+     * Decrements the internal counter. If counter becomes 0, callback will be executed.
+     *
+     * @param doIncreaseExecutionCounter whether to increase execution counter
+     */
+    public void countDown(boolean doIncreaseExecutionCounter) {
+        if (doIncreaseExecutionCounter)
+            executionCntr.incrementAndGet();
+
+        if (cntr.decrementAndGet() == 0 && executionCntr.get() >= executionThreshold)
+            cb.run();
+    }
+
+    /**
+     * Decrements the internal counter. If counter becomes 0, callback will be executed.
+     */
+    public void countDown() {
+        countDown(true);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CleanupRestoredCachesSlowTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CleanupRestoredCachesSlowTest.java
index fc08073..bbf8b66 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CleanupRestoredCachesSlowTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/CleanupRestoredCachesSlowTest.java
@@ -20,10 +20,6 @@ import java.io.File;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.file.OpenOption;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.LinkedHashSet;
-import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
@@ -44,6 +40,7 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStor
 import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.testframework.ListeningTestLogger;
 import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.MessageOrderLogListener;
 import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
@@ -112,60 +109,13 @@ public class CleanupRestoredCachesSlowTest extends GridCommonAbstractTest implem
         }
     }
 
-    /**
-     * Checks the order of the messages in the log.
-     */
-    private static class MessageOrderLogListener extends LogListener {
-        /** */
-        private final LinkedHashSet<String> matchedMessages = new LinkedHashSet<>();
-
-        /** */
-        private final List<String> matchesList;
-
-        /** */
-        private final boolean doAddDuplicates;
-
-        /** */
-        MessageOrderLogListener(List<String> matchesList, boolean doAddDuplicates) {
-            this.matchesList = matchesList;
-
-            this.doAddDuplicates = doAddDuplicates;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean check() {
-            List<String> list = new ArrayList<>(matchedMessages);
-
-            return list.equals(matchesList);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void reset() {
-            matchedMessages.clear();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void accept(String s) {
-            for (String match : matchesList)
-                if (s.matches(match)) {
-                    if (doAddDuplicates || !matchedMessages.contains(match))
-                        matchedMessages.add(match);
-
-                    break;
-                }
-        }
-    }
-
     /** */
     private static final String CACHE_NAME = "myCache";
 
     /** */
     private final LogListener logLsnr = new MessageOrderLogListener(
-        Arrays.asList(
-            "Cache stores cleanup started asynchronously",
-            "Cleanup cache stores .*? cleanFiles=true\\]"
-        ),
-        false
+        "Cache stores cleanup started asynchronously",
+        "Cleanup cache stores .*? cleanFiles=true\\]"
     );
 
     /** {@inheritDoc} */
@@ -197,6 +147,8 @@ public class CleanupRestoredCachesSlowTest extends GridCommonAbstractTest implem
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
         cleanPersistenceDir();
     }
 
@@ -205,6 +157,8 @@ public class CleanupRestoredCachesSlowTest extends GridCommonAbstractTest implem
         stopAllGrids();
 
         cleanPersistenceDir();
+
+        super.afterTest();
     }
 
     /**
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/GridCountDownCallbackTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/GridCountDownCallbackTest.java
new file mode 100644
index 0000000..aea9ed1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/GridCountDownCallbackTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.util;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ *
+ */
+public class GridCountDownCallbackTest {
+    /** */
+    @Test
+    public void testCountDownCallback() throws InterruptedException {
+        AtomicInteger cntr = new AtomicInteger(0);
+        AtomicInteger performedCntr = new AtomicInteger(0);
+
+        AtomicBoolean res = new AtomicBoolean();
+
+        int countsTillCb = 30;
+
+        GridCountDownCallback cb = new GridCountDownCallback(
+            countsTillCb,
+            () -> res.set(cntr.get() == countsTillCb && performedCntr.get() == countsTillCb / 5),
+            0
+        );
+
+        ExecutorService es = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
+
+        for (int i = 1; i < 100; i++) {
+            final int fi = i;
+
+            es.submit(() -> {
+                synchronized (es) {
+                    cntr.incrementAndGet();
+
+                    if (fi % 5 == 0)
+                        performedCntr.incrementAndGet();
+
+                    cb.countDown(fi % 5 == 0);
+                }
+            });
+        }
+
+        es.shutdown();
+
+        es.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS);
+
+        assertTrue(res.get());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/CallbackExecutorLogListener.java b/modules/core/src/test/java/org/apache/ignite/testframework/CallbackExecutorLogListener.java
new file mode 100644
index 0000000..6d41f3d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/CallbackExecutorLogListener.java
@@ -0,0 +1,55 @@
+/*
+ * 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.testframework;
+
+/**
+ * Allows to listen a log and run the callback when the message that matches the regexp is detected.
+ */
+public class CallbackExecutorLogListener extends LogListener {
+    /** */
+    private final String expectedMessage;
+
+    /** */
+    private final Runnable cb;
+
+    /**
+     * Default constructor.
+     *
+     * @param expectedMessage regexp for message that triggers the callback
+     * @param cb callback
+     */
+    public CallbackExecutorLogListener(String expectedMessage, Runnable cb) {
+        this.expectedMessage = expectedMessage;
+        this.cb = cb;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean check() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        /* No-op */
+    }
+
+    /** {@inheritDoc} */
+    @Override public void accept(String s) {
+        if (s.matches(expectedMessage))
+            cb.run();
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/MessageOrderLogListener.java b/modules/core/src/test/java/org/apache/ignite/testframework/MessageOrderLogListener.java
new file mode 100644
index 0000000..e8ea6ef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/MessageOrderLogListener.java
@@ -0,0 +1,226 @@
+/*
+ * 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.testframework;
+
+import java.util.Iterator;
+import org.apache.ignite.internal.util.GridConcurrentLinkedHashSet;
+
+/**
+ * This log listener allows to check the order or presence of messages in log. Messages are matched via regexps.
+ * Also allows to unify various messages into groups to check messages only within group.
+ * Groups can be ordered or non-ordered. In non-ordered groups, message order will not be checked, only
+ * will be checked the presence of given messages.
+ * <p>
+ * Message groups can also be unified into higher-level groups, that can also be ordered or non-ordered and in general
+ * work like message groups.
+ */
+public class MessageOrderLogListener extends LogListener {
+    /** */
+    private final MessageGroup matchesGrp;
+
+    /**
+     * Constructor accepting array of messages.
+     *
+     * @param messages array of messages that will be unified into ordered group.
+     */
+    public MessageOrderLogListener(String... messages) {
+        this(new MessageGroup(true) {{
+            for (String m : messages)
+                add(m);
+        }});
+    }
+
+    /**
+     * Constructor accepting message group
+     *
+     * @param matchesGrp group
+     */
+    public MessageOrderLogListener(MessageGroup matchesGrp) {
+        this.matchesGrp = matchesGrp;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean check() {
+        return matchesGrp.check();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        matchesGrp.reset();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void accept(String s) {
+        matchesGrp.accept(s);
+    }
+
+    /**
+     * Allows to unify messages into groups and groups into higher-level groups.
+     */
+    public static class MessageGroup extends LogListener {
+        /** */
+        private final boolean ordered;
+
+        /** */
+        private final String containedStr;
+
+        /** */
+        private final GridConcurrentLinkedHashSet<MessageGroup> groups;
+
+        /** */
+        private final GridConcurrentLinkedHashSet<MessageGroup> matched = new GridConcurrentLinkedHashSet<>();
+
+        /** */
+        private volatile boolean checked;
+
+        /** */
+        private volatile String actualStr;
+
+        /** */
+        private volatile int lastAcceptedIdx;
+
+        /**
+         * Default constructor.
+         *
+         * @param ordered whether to check order of messages in this group.
+         */
+        public MessageGroup(boolean ordered) {
+            this(ordered, null);
+        }
+
+        /** */
+        private MessageGroup(boolean ordered, String s) {
+            this.ordered = ordered;
+
+            containedStr = s;
+
+            groups = s == null ? new GridConcurrentLinkedHashSet<>() : null;
+        }
+
+        /**
+         * Adds a message regexp to group.
+         *
+         * @param s message regexp
+         * @return this for chaining
+         */
+        public MessageGroup add(String s) {
+            return add(new MessageGroup(false, s));
+        }
+
+        /**
+         * Adds another message group to this group.
+         *
+         * @param grp group
+         * @return this for chaining
+         */
+        public MessageGroup add(MessageGroup grp) {
+            groups.add(grp);
+
+            return this;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean check() {
+            if (checked)
+                return true;
+
+            if (groups != null) {
+                for (MessageGroup group : groups) {
+                    if (!matched.contains(group) || !group.check())
+                        return false;
+                }
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void reset() {
+            checked = false;
+
+            actualStr = null;
+
+            lastAcceptedIdx = 0;
+
+            if (groups != null) {
+                for (MessageGroup group : groups)
+                    group.reset();
+            }
+
+            for (Iterator<MessageGroup> iter = matched.iterator(); iter.hasNext();) {
+                iter.next();
+
+                iter.remove();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void accept(String s) {
+            internalAccept(s);
+        }
+
+        /**
+         * Tries to accept given string. If fails, tries to accept it using internal message groups.
+         *
+         * @param s message
+         * @return whether accepted
+         */
+        private boolean internalAccept(String s) {
+            if (checked)
+                return false;
+            else if (containedStr != null && s.matches(containedStr)) {
+                checked = true;
+
+                actualStr = s;
+
+                return true;
+            }
+            else {
+                if (groups != null) {
+                    int i = 0;
+
+                    for (MessageGroup group : groups) {
+                        if (i < lastAcceptedIdx && ordered) {
+                            i++;
+
+                            continue;
+                        }
+
+                        if (group.internalAccept(s)) {
+                            matched.add(group);
+
+                            lastAcceptedIdx = i;
+
+                            return true;
+                        }
+
+                        i++;
+                    }
+                }
+
+                return false;
+            }
+        }
+
+        /** */
+        public String getActualStr() {
+            return actualStr;
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/MessageOrderLogListenerTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/MessageOrderLogListenerTest.java
new file mode 100644
index 0000000..80dc7bc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/MessageOrderLogListenerTest.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.testframework;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for {@link MessageOrderLogListener}
+ */
+public class MessageOrderLogListenerTest {
+    /** */
+    @Test
+    public void testMessageOrderLogListener() {
+        MessageOrderLogListener lsnr = new MessageOrderLogListener("a", "b");
+
+        lsnr.accept("a");
+        lsnr.accept("b");
+
+        assertTrue(lsnr.check());
+
+        lsnr.reset();
+
+        lsnr.accept("b");
+        lsnr.accept("a");
+
+        assertFalse(lsnr.check());
+
+        lsnr.reset();
+
+        lsnr.accept("b");
+        lsnr.accept("a");
+        lsnr.accept("b");
+
+        assertFalse(lsnr.check());
+
+        lsnr = new MessageOrderLogListener(new MessageOrderLogListener.MessageGroup(true)
+            .add(new MessageOrderLogListener.MessageGroup(false).add("a").add("b"))
+            .add(new MessageOrderLogListener.MessageGroup(true).add("c").add("d"))
+        );
+
+        lsnr.accept("b");
+        lsnr.accept("a");
+        lsnr.accept("c");
+        lsnr.accept("d");
+
+        assertTrue(lsnr.check());
+
+        lsnr.reset();
+
+        lsnr.accept("b");
+        lsnr.accept("a");
+        lsnr.accept("d");
+        lsnr.accept("c");
+
+        assertFalse(lsnr.check());
+
+        lsnr.reset();
+
+        lsnr.accept("b");
+        lsnr.accept("c");
+        lsnr.accept("a");
+        lsnr.accept("d");
+
+        assertFalse(lsnr.check());
+
+        lsnr = new MessageOrderLogListener(new MessageOrderLogListener.MessageGroup(true)
+            .add(
+                new MessageOrderLogListener.MessageGroup(false)
+                    .add(new MessageOrderLogListener.MessageGroup(true).add("a").add("b"))
+                    .add(new MessageOrderLogListener.MessageGroup(true).add("c").add("d"))
+            )
+            .add("e")
+        );
+
+        lsnr.accept("c");
+        lsnr.accept("d");
+        lsnr.accept("a");
+        lsnr.accept("b");
+        lsnr.accept("e");
+
+        assertTrue(lsnr.check());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index c9ee1f3..30be078 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -96,6 +96,7 @@ import org.apache.ignite.plugin.security.SecurityPermissionSetBuilderTest;
 import org.apache.ignite.spi.GridSpiLocalHostInjectionTest;
 import org.apache.ignite.startup.properties.NotStringSystemPropertyTest;
 import org.apache.ignite.testframework.test.ConfigVariationsExecutionTest;
+import org.apache.ignite.testframework.MessageOrderLogListenerTest;
 import org.apache.ignite.testframework.test.ConfigVariationsTestSuiteBuilderTest;
 import org.apache.ignite.testframework.test.ListeningTestLoggerTest;
 import org.apache.ignite.testframework.test.ParametersTest;
@@ -215,6 +216,8 @@ import org.junit.runners.Suite;
 
     ListeningTestLoggerTest.class,
 
+    MessageOrderLogListenerTest.class,
+
     CacheLocalGetSerializationTest.class,
 
     PluginNodeValidationTest.class,
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
index d620363..a355480 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.commandline.CommandHandlerParsingTest;
 import org.apache.ignite.internal.pagemem.impl.PageIdUtilsSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilsSelfTest;
 import org.apache.ignite.internal.util.GridArraysSelfTest;
+import org.apache.ignite.internal.util.GridCountDownCallbackTest;
 import org.apache.ignite.internal.util.IgniteDevOnlyLogTest;
 import org.apache.ignite.internal.util.IgniteExceptionRegistrySelfTest;
 import org.apache.ignite.internal.util.IgniteUtilsSelfTest;
@@ -126,6 +127,8 @@ import org.junit.runners.Suite;
 
     // control.sh
     CommandHandlerParsingTest.class,
+
+    GridCountDownCallbackTest.class
 })
 public class IgniteUtilSelfTestSuite {
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/RebuildIndexLogMessageTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/RebuildIndexLogMessageTest.java
new file mode 100644
index 0000000..7980cf3
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/RebuildIndexLogMessageTest.java
@@ -0,0 +1,215 @@
+/*
+ * 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.persistence;
+
+import java.io.File;
+import java.io.Serializable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.CallbackExecutorLogListener;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.MessageOrderLogListener;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_BASELINE_AUTO_ADJUST_ENABLED;
+
+/**
+ *
+ */
+@WithSystemProperty(key = IGNITE_BASELINE_AUTO_ADJUST_ENABLED, value = "false")
+public class RebuildIndexLogMessageTest extends GridCommonAbstractTest implements Serializable {
+    /** */
+    private static final String CACHE_NAME_A = "testCacheA";
+
+    /** */
+    private static final String CACHE_NAME_B = "testCacheB";
+
+    /** */
+    private final CountDownLatch checkLatch = new CountDownLatch(1);
+
+    /** */
+    private final LogListener logLsnr = new MessageOrderLogListener(
+        new MessageOrderLogListener.MessageGroup(true)
+            //message group for all caches
+            .add(
+                new MessageOrderLogListener.MessageGroup(false)
+                    .add(
+                        //message group for testCacheA
+                        new MessageOrderLogListener.MessageGroup(true)
+                            .add("Started indexes rebuilding for cache \\[name=testCacheA.*")
+                            .add("((Finished indexes rebuilding)|(Failed to rebuild indexes)) for cache " +
+                                "\\[name=testCacheA.*")
+                    )
+                    .add(
+                        //message group for testCacheB
+                        new MessageOrderLogListener.MessageGroup(true)
+                            .add("Started indexes rebuilding for cache \\[name=testCacheB.*")
+                            .add("((Finished indexes rebuilding)|(Failed to rebuild indexes)) for cache " +
+                                "\\[name=testCacheB.*")
+                    )
+            )
+            //message that appears after rebuilding is completed for all caches
+            .add("Indexes rebuilding completed for all caches.")
+    );
+
+    /** */
+    private final LogListener latchLsnr =
+        new CallbackExecutorLogListener("Indexes rebuilding completed for all caches.", checkLatch::countDown);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        ListeningTestLogger testLog =
+            new ListeningTestLogger(false, super.getConfiguration(igniteInstanceName).getGridLogger());
+
+        testLog.registerListener(logLsnr);
+        testLog.registerListener(latchLsnr);
+
+        return super.getConfiguration(igniteInstanceName)
+            .setGridLogger(testLog)
+            .setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDefaultDataRegionConfiguration(
+                        new DataRegionConfiguration()
+                            .setPersistenceEnabled(true)
+                    )
+            )
+            .setCacheConfiguration(
+                new CacheConfiguration<>()
+                    .setName(CACHE_NAME_A)
+                    .setBackups(0)
+                    .setIndexedTypes(Integer.class, Person.class)
+                    .setAffinity(new RendezvousAffinityFunction(false, 8)),
+                new CacheConfiguration<>()
+                    .setName(CACHE_NAME_B)
+                    .setBackups(0)
+                    .setIndexedTypes(Integer.class, Person.class)
+                    .setAffinity(new RendezvousAffinityFunction(false, 8))
+            );
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+
+        super.afterTest();
+    }
+
+    /** */
+    @Test
+    public void testRebuildIndexLogMessage() throws Exception {
+        IgniteEx ignite = startGrids(1);
+
+        String gridName = ignite.name();
+
+        ignite.cluster().active(true);
+
+        IgniteCache<Integer, Person> cacheA = ignite.getOrCreateCache(CACHE_NAME_A);
+        IgniteCache<Integer, Person> cacheB = ignite.getOrCreateCache(CACHE_NAME_B);
+
+        IgniteInternalCache<Integer, Person> cacheAEx = ignite.cachex(CACHE_NAME_A);
+        IgniteInternalCache<Integer, Person> cacheBEx = ignite.cachex(CACHE_NAME_B);
+
+        for (int i = 0; i < 100; i++)
+            cacheA.put(i, new Person(i, i));
+
+        for (int i = 0; i < 100; i++)
+            cacheB.put(i, new Person(i, i));
+
+        forceCheckpoint();
+
+        File cacheAWorkDir = ((FilePageStoreManager)cacheAEx.context().shared().pageStore())
+            .cacheWorkDir(cacheAEx.configuration());
+        File cacheBWorkDir = ((FilePageStoreManager)cacheBEx.context().shared().pageStore())
+            .cacheWorkDir(cacheBEx.configuration());
+
+        File idxPathA = cacheAWorkDir.toPath().resolve("index.bin").toFile();
+        File idxPathB = cacheBWorkDir.toPath().resolve("index.bin").toFile();
+
+        stopAllGrids();
+
+        assertTrue(U.delete(idxPathA));
+        assertTrue(U.delete(idxPathB));
+
+        ignite = startGrid(getConfiguration(gridName));
+
+        ignite.cluster().active(true);
+
+        cacheA = ignite.getOrCreateCache(CACHE_NAME_A);
+        cacheB = ignite.getOrCreateCache(CACHE_NAME_B);
+
+        cacheA.put(1000, new Person(1000, 1));
+        cacheB.put(1000, new Person(1000, 1));
+
+        checkLatch.await(60, TimeUnit.SECONDS);
+
+        if (checkLatch.getCount() > 0)
+            throw new TimeoutException("Test timed out: cannot detect log message about completion of indexes rebuilding");
+
+        assertTrue(logLsnr.check());
+    }
+
+    /** */
+    private static class Person implements Serializable {
+        /** */
+        @QuerySqlField(index = true)
+        private int id;
+
+        /** */
+        @QuerySqlField(index = true)
+        private int age;
+
+        /**
+         * @param id Id.
+         * @param age Age.
+         */
+        Person(int id, int age) {
+            this.id = id;
+            this.age = age;
+        }
+
+        /**
+         * @return Age/
+         */
+        public int age() {
+            return age;
+        }
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
index a7dc9da..864a6fb 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheConfigurationPrimi
 import org.apache.ignite.internal.processors.cache.IgniteCacheGroupsSqlTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheStarvationOnRebalanceTest;
 import org.apache.ignite.internal.processors.cache.IgniteClientReconnectQueriesTest;
+import org.apache.ignite.internal.processors.cache.persistence.RebuildIndexLogMessageTest;
 import org.apache.ignite.internal.processors.cache.ttl.CacheTtlAtomicLocalSelfTest;
 import org.apache.ignite.internal.processors.cache.ttl.CacheTtlAtomicPartitionedSelfTest;
 import org.apache.ignite.internal.processors.cache.ttl.CacheTtlTransactionalLocalSelfTest;
@@ -89,7 +90,9 @@ import org.junit.runners.Suite;
 
     ClusterReadOnlyModeSqlTest.class,
 
-    CacheGroupMetricsMBeanWithIndexTest.class
+    CacheGroupMetricsMBeanWithIndexTest.class,
+
+    RebuildIndexLogMessageTest.class
 })
 public class IgniteCacheWithIndexingTestSuite {
 }