You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/10/04 12:10:43 UTC

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

Repository: ignite
Updated Branches:
  refs/heads/master 62f3c4c52 -> 20033c732


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

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


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

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

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


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

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