Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id DC30C200C8B for ; Mon, 22 May 2017 17:12:35 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id DAEF6160BA5; Mon, 22 May 2017 15:12:35 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id CC4FE160BE4 for ; Mon, 22 May 2017 17:12:33 +0200 (CEST) Received: (qmail 53033 invoked by uid 500); 22 May 2017 15:12:32 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 51531 invoked by uid 99); 22 May 2017 15:12:30 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 22 May 2017 15:12:30 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id D0DA9DFA33; Mon, 22 May 2017 15:12:30 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Mon, 22 May 2017 15:13:16 -0000 Message-Id: <7a615eee1063435eba3c53b1e8f203b9@git.apache.org> In-Reply-To: <9fa8b040ac584bb7a9fddc61f3a941b7@git.apache.org> References: <9fa8b040ac584bb7a9fddc61f3a941b7@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [48/50] [abbrv] ignite git commit: ignite-5075 archived-at: Mon, 22 May 2017 15:12:36 -0000 http://git-wip-us.apache.org/repos/asf/ignite/blob/7ae45b4b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/serializer/RecordV1Serializer.java ---------------------------------------------------------------------- diff --git a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/serializer/RecordV1Serializer.java b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/serializer/RecordV1Serializer.java index f67f617..cd9c41e 100644 --- a/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/serializer/RecordV1Serializer.java +++ b/modules/pds/src/main/java/org/apache/ignite/internal/processors/cache/database/wal/serializer/RecordV1Serializer.java @@ -222,7 +222,7 @@ public class RecordV1Serializer implements RecordSerializer { buf.putInt(walPtr.length()); } - putCacheStates(buf, cpRec.cacheStates()); + putCacheStates(buf, cpRec.cacheGroupStates()); buf.put(cpRec.end() ? (byte)1 : 0); @@ -730,7 +730,7 @@ public class RecordV1Serializer implements RecordSerializer { CheckpointRecord cpRec = new CheckpointRecord(new UUID(msb, lsb), walPtr, end); - cpRec.cacheStates(states); + cpRec.cacheGroupStates(states); res = cpRec; @@ -1234,7 +1234,7 @@ public class RecordV1Serializer implements RecordSerializer { assert cpRec.checkpointMark() == null || cpRec.checkpointMark() instanceof FileWALPointer : "Invalid WAL record: " + cpRec; - int cacheStatesSize = cacheStatesSize(cpRec.cacheStates()); + int cacheStatesSize = cacheStatesSize(cpRec.cacheGroupStates()); FileWALPointer walPtr = (FileWALPointer)cpRec.checkpointMark(); http://git-wip-us.apache.org/repos/asf/ignite/blob/7ae45b4b/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheGroupsTest.java ---------------------------------------------------------------------- diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheGroupsTest.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheGroupsTest.java new file mode 100644 index 0000000..b184375 --- /dev/null +++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/IgnitePersistentStoreCacheGroupsTest.java @@ -0,0 +1,161 @@ +/* + * 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.cache.database; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.MemoryConfiguration; +import org.apache.ignite.configuration.PersistenceConfiguration; +import org.apache.ignite.internal.processors.cache.database.wal.FileWriteAheadLogManager; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; + +/** + * + */ +public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final String GROUP1 = "grp1"; + + /** */ + private static final String GROUP2 = "grp2"; + + /** */ + private CacheConfiguration[] ccfgs; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + System.setProperty(FileWriteAheadLogManager.IGNITE_PDS_WAL_MODE, "LOG_ONLY"); + + GridTestUtils.deleteDbFiles(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + System.clearProperty(FileWriteAheadLogManager.IGNITE_PDS_WAL_MODE); + + GridTestUtils.deleteDbFiles(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + cfg.setConsistentId(gridName); + + MemoryConfiguration memCfg = new MemoryConfiguration(); + memCfg.setPageSize(1024); + memCfg.setDefaultMemoryPolicySize(10 * 1024 * 1024); + + cfg.setMemoryConfiguration(memCfg); + + cfg.setPersistenceConfiguration(new PersistenceConfiguration()); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + if (ccfgs != null) { + cfg.setCacheConfiguration(ccfgs); + + ccfgs = null; + } + + return cfg; + } + + /** + * @throws Exception If failed. + */ + public void testNodeRestart() throws Exception { + ccfgs = new CacheConfiguration[5]; + + ccfgs[0] = cacheConfiguration(GROUP1, "c1", PARTITIONED, ATOMIC, 1); + ccfgs[1] = cacheConfiguration(GROUP1, "c2", PARTITIONED, TRANSACTIONAL, 1); + ccfgs[2] = cacheConfiguration(GROUP2, "c3", PARTITIONED, ATOMIC, 1); + ccfgs[3] = cacheConfiguration(GROUP2, "c4", PARTITIONED, TRANSACTIONAL, 1); + ccfgs[4] = cacheConfiguration(null, "c5", PARTITIONED, ATOMIC, 1); + + String[] caches = {"c1", "c2", "c3", "c5", "c5"}; + + Ignite node = startGrid(0); + + for (String cacheName : caches) { + IgniteCache cache = node.cache(cacheName); + + for (int i = 0; i < 10; i++) { + cache.put(i, cacheName + i); + + assertEquals(cacheName + i, cache.get(i)); + } + } + + stopGrid(0); + + node = startGrid(0); + + for (String cacheName : caches) { + IgniteCache cache = node.cache(cacheName); + + for (int i = 0; i < 10; i++) + assertEquals(cacheName + i, cache.get(i)); + } + } + + /** + * @param grpName Cache group name. + * @param name Cache name. + * @param cacheMode Cache mode. + * @param atomicityMode Atomicity mode. + * @param backups Backups number. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration( + String grpName, + String name, + CacheMode cacheMode, + CacheAtomicityMode atomicityMode, + int backups + ) { + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setName(name); + ccfg.setGroupName(grpName); + ccfg.setAtomicityMode(atomicityMode); + ccfg.setBackups(backups); + ccfg.setCacheMode(cacheMode); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + + return ccfg; + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/7ae45b4b/modules/pds/src/test/java/org/apache/ignite/cache/database/pagemem/NoOpPageStoreManager.java ---------------------------------------------------------------------- diff --git a/modules/pds/src/test/java/org/apache/ignite/cache/database/pagemem/NoOpPageStoreManager.java b/modules/pds/src/test/java/org/apache/ignite/cache/database/pagemem/NoOpPageStoreManager.java index 9d5fe69..4b344bc 100644 --- a/modules/pds/src/test/java/org/apache/ignite/cache/database/pagemem/NoOpPageStoreManager.java +++ b/modules/pds/src/test/java/org/apache/ignite/cache/database/pagemem/NoOpPageStoreManager.java @@ -19,7 +19,7 @@ package org.apache.ignite.cache.database.pagemem; import java.nio.ByteBuffer; import java.util.Collections; -import java.util.Set; +import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; @@ -29,7 +29,8 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.pagemem.FullPageId; import org.apache.ignite.internal.pagemem.PageIdUtils; import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager; -import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; +import org.apache.ignite.internal.processors.cache.CacheGroupInfrastructure; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgniteFuture; @@ -52,17 +53,17 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager { } /** {@inheritDoc} */ - @Override public void initializeForCache(CacheConfiguration ccfg) throws IgniteCheckedException { + @Override public void initializeForCache(CacheGroupDescriptor grpDesc, CacheConfiguration ccfg) throws IgniteCheckedException { // No-op. } /** {@inheritDoc} */ - @Override public void shutdownForCache(GridCacheContext cacheCtx, boolean destroy) throws IgniteCheckedException { + @Override public void shutdownForCacheGroup(CacheGroupInfrastructure grp, boolean destroy) throws IgniteCheckedException { // No-op. } /** {@inheritDoc} */ - @Override public void onPartitionCreated(int cacheId, int partId) throws IgniteCheckedException { + @Override public void onPartitionCreated(int grpId, int partId) throws IgniteCheckedException { // No-op. } @@ -170,17 +171,12 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager { } /** {@inheritDoc} */ - @Override public Set savedCacheNames() { - return Collections.emptySet(); + @Override public Map readCacheConfigurations() throws IgniteCheckedException { + return Collections.emptyMap(); } /** {@inheritDoc} */ - @Override public CacheConfiguration readConfiguration(String cacheName) { - return null; - } - - /** {@inheritDoc} */ - @Override public boolean hasIndexStore(int cacheId) { + @Override public boolean hasIndexStore(int grpId) { return false; }