Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 0036C175DD for ; Thu, 9 Apr 2015 12:14:00 +0000 (UTC) Received: (qmail 49299 invoked by uid 500); 9 Apr 2015 12:13:59 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 49272 invoked by uid 500); 9 Apr 2015 12:13:59 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 49263 invoked by uid 99); 9 Apr 2015 12:13:59 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 09 Apr 2015 12:13:59 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Thu, 09 Apr 2015 12:13:57 +0000 Received: (qmail 48493 invoked by uid 99); 9 Apr 2015 12:13:37 -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; Thu, 09 Apr 2015 12:13:37 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id E21C5DFFCD; Thu, 9 Apr 2015 12:13:36 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.incubator.apache.org Date: Thu, 09 Apr 2015 12:13:52 -0000 Message-Id: In-Reply-To: <54cd8259643248ceba93144c04ce82cd@git.apache.org> References: <54cd8259643248ceba93144c04ce82cd@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [17/50] [abbrv] incubator-ignite git commit: #ignite-180: fix review comments. X-Virus-Checked: Checked by ClamAV on apache.org #ignite-180: fix review comments. Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/672e9b21 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/672e9b21 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/672e9b21 Branch: refs/heads/ignite-688 Commit: 672e9b21132446a7499de28c1392c25068414432 Parents: 2cda69f Author: ivasilinets Authored: Wed Apr 8 16:11:19 2015 +0300 Committer: ivasilinets Committed: Wed Apr 8 16:11:19 2015 +0300 ---------------------------------------------------------------------- .../configuration/CacheConfiguration.java | 20 +++++- .../configuration/CollectionConfiguration.java | 19 ++++++ .../datastructures/DataStructuresProcessor.java | 12 +++- .../GridCacheQueueApiSelfAbstractTest.java | 70 +++++++++++++++++--- .../IgniteCollectionAbstractTest.java | 21 ++++-- ...artitionedOffHeapValuesQueueApiSelfTest.java | 29 ++++++++ ...achePartitionedOffHeapValuesSetSelfTest.java | 30 +++++++++ .../IgniteCacheDataStructuresSelfTestSuite.java | 4 +- 8 files changed, 186 insertions(+), 19 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/672e9b21/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java index c90de92..bd3fac7 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java @@ -154,10 +154,28 @@ public class CacheConfiguration extends MutableConfiguration { return attr != null && !attr; } + + @Override public boolean equals(Object obj) { + if (obj == null) + return false; + + return obj.getClass().equals(this.getClass()); + } }; /** Filter that accepts all nodes. */ - public static final IgnitePredicate ALL_NODES = F.alwaysTrue(); + public static final IgnitePredicate ALL_NODES = new IgnitePredicate() { + @Override public boolean apply(ClusterNode clusterNode) { + return true; + } + + @Override public boolean equals(Object obj) { + if (obj == null) + return false; + + return obj.getClass().equals(this.getClass()); + } + }; /** Default timeout after which long query warning will be printed. */ public static final long DFLT_LONG_QRY_WARN_TIMEOUT = 3000; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/672e9b21/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java index 85daadb..5a1ae0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/CollectionConfiguration.java @@ -18,7 +18,9 @@ package org.apache.ignite.configuration; import org.apache.ignite.cache.*; +import org.apache.ignite.cluster.*; import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.lang.*; import java.io.*; @@ -42,6 +44,9 @@ public class CollectionConfiguration implements Serializable { /** Cache memory mode. */ private CacheMemoryMode memoryMode = ONHEAP_TIERED; + /** Node filter specifying nodes on which this cache should be deployed. */ + private IgnitePredicate nodeFilter; + /** Number of backups. */ private int backups = 0; @@ -110,6 +115,20 @@ public class CollectionConfiguration implements Serializable { } /** + * @return Predicate specifying on which nodes the cache should be started. + */ + public IgnitePredicate getNodeFilter() { + return nodeFilter; + } + + /** + * @param nodeFilter Predicate specifying on which nodes the cache should be started. + */ + public void setNodeFilter(IgnitePredicate nodeFilter) { + this.nodeFilter = nodeFilter; + } + + /** * @return Number of backups. */ public int getBackups() { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/672e9b21/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java index 20baf94..c8e1d93 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java @@ -727,6 +727,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter { ccfg.setMemoryMode(cfg.getMemoryMode()); ccfg.setAtomicityMode(cfg.getAtomicityMode()); ccfg.setOffHeapMaxMemory(cfg.getOffHeapMaxMemory()); + ccfg.setNodeFilter(cfg.getNodeFilter()); ccfg.setWriteSynchronizationMode(FULL_SYNC); ccfg.setAtomicWriteOrderMode(PRIMARY); ccfg.setRebalanceMode(SYNC); @@ -1253,7 +1254,9 @@ public final class DataStructuresProcessor extends GridProcessorAdapter { col.cfg.getMemoryMode() == cfg.getMemoryMode() && col.cfg.getCacheMode() == cfg.getCacheMode() && col.cfg.getBackups() == cfg.getBackups() && - col.cfg.getOffHeapMaxMemory() == cfg.getOffHeapMaxMemory()) + col.cfg.getOffHeapMaxMemory() == cfg.getOffHeapMaxMemory() && + ((col.cfg.getNodeFilter() == null && cfg.getNodeFilter() == null) || + (col.cfg.getNodeFilter() != null && col.cfg.getNodeFilter().equals(cfg.getNodeFilter())))) return col.cacheName; } @@ -1714,6 +1717,9 @@ public final class DataStructuresProcessor extends GridProcessorAdapter { */ static class AddDataCacheProcessor implements EntryProcessor, String>, Externalizable { + /** Cache name prefix. */ + private static final String CACHE_NAME_PREFIX = "datastructures_"; + /** */ private static final long serialVersionUID = 0L; @@ -1744,7 +1750,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter { if (list == null) { list = new ArrayList<>(); - String newName = "datastructeres_" + 0; + String newName = CACHE_NAME_PREFIX + 0; list.add(new CacheCollectionInfo(newName, cfg)); @@ -1758,7 +1764,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter { if (oldName != null) return oldName; - String newName = "datastructeres_" + list.size(); + String newName = CACHE_NAME_PREFIX + list.size(); List newList = new ArrayList<>(list); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/672e9b21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java index 012401d..12397b2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java @@ -19,8 +19,7 @@ package org.apache.ignite.internal.processors.cache.datastructures; import org.apache.ignite.*; import org.apache.ignite.configuration.*; -import org.apache.ignite.internal.processors.cache.*; -import org.apache.ignite.internal.processors.datastructures.*; +import org.apache.ignite.internal.*; import org.apache.ignite.internal.util.typedef.internal.*; import org.apache.ignite.testframework.*; @@ -28,6 +27,8 @@ import java.io.*; import java.util.*; import java.util.concurrent.*; +import static org.apache.ignite.cache.CacheAtomicityMode.*; + /** * Queue basic tests. */ @@ -510,7 +511,7 @@ public abstract class GridCacheQueueApiSelfAbstractTest extends IgniteCollection /** * @throws Exception If failed. */ - public void testReuseHash() throws Exception { + public void testReuseCache() throws Exception { CollectionConfiguration colCfg = collectionConfiguration(); IgniteQueue queue1 = grid(0).queue("Queue1", 0, colCfg); @@ -521,15 +522,66 @@ public abstract class GridCacheQueueApiSelfAbstractTest extends IgniteCollection } /** - * @param queue Ignite queue. - * @return Cache configuration. + * @throws Exception If failed. */ - private CacheConfiguration getQueueCache(IgniteQueue queue) { - GridCacheQueueAdapter delegate = GridTestUtils.getFieldValue(queue, "delegate"); + public void testNotReuseCache() throws Exception { + CollectionConfiguration colCfg1 = collectionConfiguration(); + + CollectionConfiguration colCfg2 = collectionConfiguration(); + + if (colCfg2.getAtomicityMode() == ATOMIC) + colCfg2.setAtomicityMode(TRANSACTIONAL); + else + colCfg2.setAtomicityMode(ATOMIC); + + IgniteQueue queue1 = grid(0).queue("Queue1", 0, colCfg1); + + IgniteQueue queue2 = grid(0).queue("Queue2", 0, colCfg2); + + assertNotSame(getQueueCache(queue1), getQueueCache(queue2)); + } + + /** + * @throws Exception If failed. + */ + public void testFilterNode() throws Exception { + CollectionConfiguration colCfg1 = collectionConfiguration(); + + CollectionConfiguration colCfg2 = collectionConfiguration(); + + colCfg2.setNodeFilter(CacheConfiguration.SERVER_NODES); + + IgniteQueue queue1 = grid(0).queue("Queue1", 0, colCfg1); - GridCacheAdapter cache = GridTestUtils.getFieldValue(delegate, GridCacheQueueAdapter.class, "cache"); + IgniteQueue queue2 = grid(0).queue("Queue2", 0, colCfg2); + + assertNotSame(getQueueCache(queue1), getQueueCache(queue2)); + + colCfg1.setNodeFilter(CacheConfiguration.SERVER_NODES); + + IgniteQueue queue3 = grid(0).queue("Queue3", 0, colCfg1); + + assertEquals(getQueueCache(queue2), getQueueCache(queue3)); + } + + /** + * @throws Exception If failed. + */ + public void testSystemCache() throws Exception { + CollectionConfiguration colCfg = collectionConfiguration(); + + IgniteQueue queue = grid(0).queue("Queue1", 0, colCfg); + + final CacheConfiguration ccfg = getQueueCache(queue); + + GridTestUtils.assertThrows(log, new Callable() { + @Override public Object call() throws Exception { + grid(0).cache(ccfg.getName()); + return null; + } + }, IllegalStateException.class, "Failed to get cache because it is a system cache"); - return cache.configuration(); + assertNotNull(((IgniteKernal)grid(0)).internalCache(ccfg.getName())); } /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/672e9b21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java index 3ef9ebd..5d1e16e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCollectionAbstractTest.java @@ -17,19 +17,18 @@ package org.apache.ignite.internal.processors.cache.datastructures; +import org.apache.ignite.*; import org.apache.ignite.cache.*; import org.apache.ignite.configuration.*; +import org.apache.ignite.internal.processors.cache.*; +import org.apache.ignite.internal.processors.datastructures.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; +import org.apache.ignite.testframework.*; import org.apache.ignite.testframework.junits.common.*; -import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*; -import static org.apache.ignite.cache.CacheAtomicityMode.*; -import static org.apache.ignite.cache.CacheMemoryMode.*; import static org.apache.ignite.cache.CacheMode.*; -import static org.apache.ignite.cache.CacheRebalanceMode.*; -import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*; /** * @@ -116,4 +115,16 @@ public abstract class IgniteCollectionAbstractTest extends GridCommonAbstractTes @Override protected void afterTestsStopped() throws Exception { stopAllGrids(); } + + /** + * @param queue Ignite queue. + * @return Cache configuration. + */ + protected CacheConfiguration getQueueCache(IgniteQueue queue) { + GridCacheQueueAdapter delegate = GridTestUtils.getFieldValue(queue, "delegate"); + + GridCacheAdapter cache = GridTestUtils.getFieldValue(delegate, GridCacheQueueAdapter.class, "cache"); + + return cache.configuration(); + } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/672e9b21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java new file mode 100644 index 0000000..1cffd46 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java @@ -0,0 +1,29 @@ +/* + * 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.datastructures.partitioned; + +import org.apache.ignite.cache.*; + +/** + * Queue tests with partitioned cache. + */ +public class GridCachePartitionedOffHeapValuesQueueApiSelfTest extends GridCachePartitionedQueueApiSelfTest { + /** {@inheritDoc} */ + @Override protected CacheMemoryMode collectionMemoryMode() { + return CacheMemoryMode.OFFHEAP_VALUES; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/672e9b21/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesSetSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesSetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesSetSelfTest.java new file mode 100644 index 0000000..7c6041f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesSetSelfTest.java @@ -0,0 +1,30 @@ +/* + * 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.datastructures.partitioned; + +import org.apache.ignite.cache.*; + +/** + * Set tests with off-heap values cache memory mode. + */ +public class GridCachePartitionedOffHeapValuesSetSelfTest extends GridCachePartitionedSetSelfTest { + /** {@inheritDoc} */ + @Override protected CacheMemoryMode collectionMemoryMode() { + return CacheMemoryMode.OFFHEAP_VALUES; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/672e9b21/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java index 7ae9553..d17b3db 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java @@ -61,17 +61,19 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite { suite.addTest(new TestSuite(GridCachePartitionedSequenceApiSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedSequenceMultiNodeSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedQueueApiSelfTest.class)); + suite.addTest(new TestSuite(GridCachePartitionedOffHeapValuesQueueApiSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAtomicQueueApiSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAtomicOffheapQueueApiSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedQueueMultiNodeSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAtomicQueueMultiNodeSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAtomicOffheapQueueMultiNodeSelfTest.class)); - // TODO: IGNITE-80. + // TODO: IGNITE-180. //suite.addTest(new TestSuite(GridCachePartitionedQueueCreateMultiNodeSelfTest.class)); //suite.addTest(new TestSuite(GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.class)); //suite.addTest(new TestSuite(GridCachePartitionedAtomicOffheapQueueCreateMultiNodeSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedSetSelfTest.class)); + suite.addTest(new TestSuite(GridCachePartitionedOffHeapValuesSetSelfTest.class)); suite.addTest(new TestSuite(IgnitePartitionedSetNoBackupsSelfTest.class)); suite.addTest(new TestSuite(GridCachePartitionedAtomicSetSelfTest.class)); suite.addTest(new TestSuite(IgnitePartitionedCountDownLatchSelfTest.class));