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 37A381879B for ; Fri, 12 Jun 2015 11:17:29 +0000 (UTC) Received: (qmail 5529 invoked by uid 500); 12 Jun 2015 11:17:28 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 5499 invoked by uid 500); 12 Jun 2015 11:17:28 -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 5490 invoked by uid 99); 12 Jun 2015 11:17:28 -0000 Received: from Unknown (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 12 Jun 2015 11:17:28 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 4484F18221F for ; Fri, 12 Jun 2015 11:17:28 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.771 X-Spam-Level: * X-Spam-Status: No, score=1.771 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, T_RP_MATCHES_RCVD=-0.01, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-eu-west.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id bbM9kJHMUOyh for ; Fri, 12 Jun 2015 11:17:14 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with SMTP id AB67C255D2 for ; Fri, 12 Jun 2015 11:16:58 +0000 (UTC) Received: (qmail 2165 invoked by uid 99); 12 Jun 2015 11:16:56 -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; Fri, 12 Jun 2015 11:16:56 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id AF806E1808; Fri, 12 Jun 2015 11:16:56 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vozerov@apache.org To: commits@ignite.incubator.apache.org Date: Fri, 12 Jun 2015 11:17:04 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [09/24] incubator-ignite git commit: ignite-sprint-6: merge from ignite-471-2 ignite-sprint-6: merge from ignite-471-2 Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/37c56f69 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/37c56f69 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/37c56f69 Branch: refs/heads/ignite-gg-10411 Commit: 37c56f69749e994090e1df192b12617e795699f5 Parents: 8020594 Author: Denis Magda Authored: Thu Jun 11 15:48:14 2015 +0300 Committer: Denis Magda Committed: Thu Jun 11 15:48:14 2015 +0300 ---------------------------------------------------------------------- .../ignite/internal/GridPluginContext.java | 6 ++ .../internal/MarshallerContextAdapter.java | 30 ++++++- .../ignite/internal/MarshallerContextImpl.java | 10 +-- .../internal/interop/InteropIgnition.java | 2 + .../processors/cache/GridCacheMessage.java | 51 ----------- .../processors/cache/GridCacheProcessor.java | 10 +-- .../cacheobject/IgniteCacheObjectProcessor.java | 9 +- .../IgniteCacheObjectProcessorImpl.java | 12 ++- .../datastreamer/DataStreamerImpl.java | 8 +- .../portable/GridPortableInputStream.java | 10 +++ .../processors/query/GridQueryProcessor.java | 2 +- .../ignite/internal/util/IgniteUtils.java | 3 + .../internal/visor/VisorMultiNodeTask.java | 2 +- .../ignite/marshaller/MarshallerContext.java | 8 ++ .../org/apache/ignite/plugin/PluginContext.java | 6 ++ modules/core/src/test/config/tests.properties | 2 +- .../internal/GridDiscoveryEventSelfTest.java | 6 +- ...ridFailFastNodeFailureDetectionSelfTest.java | 2 + .../GridFailoverTaskWithPredicateSelfTest.java | 3 - .../GridJobMasterLeaveAwareSelfTest.java | 2 - .../internal/GridJobStealingSelfTest.java | 3 - ...ectionLocalJobMultipleArgumentsSelfTest.java | 2 - .../GridTaskExecutionContextSelfTest.java | 9 -- .../IgniteComputeEmptyClusterGroupTest.java | 3 - .../IgniteComputeTopologyExceptionTest.java | 9 -- .../GridDiscoveryManagerAliveCacheSelfTest.java | 5 ++ .../cache/GridCacheAbstractSelfTest.java | 3 - .../cache/GridCacheAffinityRoutingSelfTest.java | 4 +- .../cache/GridCacheDeploymentSelfTest.java | 3 - .../cache/GridCacheEntryMemorySizeSelfTest.java | 91 +++++++++++--------- .../cache/GridCacheMemoryModeSelfTest.java | 2 - .../GridCacheReferenceCleanupSelfTest.java | 3 - .../cache/IgniteCacheAbstractTest.java | 3 - .../IgniteCacheEntryListenerAbstractTest.java | 14 +-- ...cheAtomicReferenceMultiNodeAbstractTest.java | 11 --- ...GridCacheQueueMultiNodeAbstractSelfTest.java | 2 - ...CacheQueueRotativeMultiNodeAbstractTest.java | 10 --- .../GridCacheSetAbstractSelfTest.java | 9 -- .../GridCacheAbstractJobExecutionTest.java | 3 - .../GridCachePreloadLifecycleAbstractTest.java | 2 - ...heAbstractTransformWriteThroughSelfTest.java | 3 - .../near/GridCacheNearEvictionSelfTest.java | 3 - ...idCachePartitionedHitsAndMissesSelfTest.java | 3 - .../GridCacheSwapScanQueryAbstractSelfTest.java | 3 - ...ridCacheContinuousQueryAbstractSelfTest.java | 2 - .../closure/GridClosureProcessorSelfTest.java | 29 +++---- .../continuous/GridEventConsumeSelfTest.java | 2 - .../DataStreamProcessorSelfTest.java | 44 ++++++---- .../internal/util/nio/GridNioSelfTest.java | 13 ++- .../internal/util/nio/GridNioSslSelfTest.java | 2 + .../tostring/GridToStringBuilderSelfTest.java | 4 +- .../marshaller/MarshallerContextTestImpl.java | 11 ++- .../ignite/messaging/GridMessagingSelfTest.java | 3 - .../GridP2PContinuousDeploymentSelfTest.java | 2 + .../p2p/GridP2PLocalDeploymentSelfTest.java | 6 +- .../p2p/GridP2PRemoteClassLoadersSelfTest.java | 31 +++---- .../spi/GridTcpSpiForwardingSelfTest.java | 3 - .../ignite/testframework/GridTestUtils.java | 14 +++ .../config/GridTestProperties.java | 14 ++- .../junits/IgniteTestResources.java | 16 ++-- .../ignite/testsuites/IgniteBasicTestSuite.java | 29 +++++-- .../IgniteCacheFailoverTestSuite.java | 14 ++- ...niteCacheP2pUnmarshallingErrorTestSuite.java | 20 ++++- .../ignite/testsuites/IgniteCacheTestSuite.java | 28 ++++-- .../testsuites/IgniteKernalSelfTestSuite.java | 14 ++- .../IgniteMarshallerSelfTestSuite.java | 28 ++++-- .../testsuites/IgniteUtilSelfTestSuite.java | 18 +++- .../tests/p2p/P2PTestTaskExternalPath1.java | 10 +-- .../tests/p2p/P2PTestTaskExternalPath2.java | 8 +- modules/indexing/pom.xml | 16 ++++ .../cache/GridCacheCrossCacheQuerySelfTest.java | 2 - .../cache/GridCacheOffHeapSelfTest.java | 1 - ...idCacheReduceQueryMultithreadedSelfTest.java | 10 --- .../processors/cache/GridCacheSwapSelfTest.java | 3 - .../IgniteCacheAbstractFieldsQuerySelfTest.java | 13 +-- .../cache/IgniteCacheAbstractQuerySelfTest.java | 2 - ...hePartitionedQueryMultiThreadedSelfTest.java | 40 ++++----- .../IgniteCacheQueryMultiThreadedSelfTest.java | 1 - ...dCacheAbstractReduceFieldsQuerySelfTest.java | 1 - .../query/h2/sql/GridQueryParsingTest.java | 5 +- 80 files changed, 452 insertions(+), 394 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/GridPluginContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginContext.java index af03670..7abe05b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginContext.java @@ -20,6 +20,7 @@ package org.apache.ignite.internal; import org.apache.ignite.*; import org.apache.ignite.cluster.*; import org.apache.ignite.configuration.*; +import org.apache.ignite.marshaller.*; import org.apache.ignite.plugin.*; import org.apache.ignite.spi.*; @@ -55,6 +56,11 @@ public class GridPluginContext implements PluginContext { } /** {@inheritDoc} */ + @Override public MarshallerContext marshallerContext() { + return ctx.marshallerContext(); + } + + /** {@inheritDoc} */ @Override public Collection nodes() { return ctx.discovery().allNodes(); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java index 21f2264..2ec9825 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java @@ -41,6 +41,9 @@ public abstract class MarshallerContextAdapter implements MarshallerContext { /** */ private final ConcurrentMap map = new ConcurrentHashMap8<>(); + /** */ + private final Set registeredSystemTypes = new HashSet<>(); + /** * Initializes context. */ @@ -107,7 +110,17 @@ public abstract class MarshallerContextAdapter implements MarshallerContext { String clsName = line.trim(); - map.put(clsName.hashCode(), clsName); + int typeId = clsName.hashCode(); + + String oldClsName; + + if ((oldClsName = map.put(typeId, clsName)) != null) { + if (!oldClsName.equals(clsName)) + throw new IgniteException("Duplicate type ID [id=" + typeId + ", clsName=" + clsName + + ", oldClsName=" + oldClsName + ']'); + } + + registeredSystemTypes.add(clsName); } } } @@ -116,12 +129,17 @@ public abstract class MarshallerContextAdapter implements MarshallerContext { @Override public boolean registerClass(int id, Class cls) throws IgniteCheckedException { boolean registered = true; - if (!map.containsKey(id)) { + String clsName = map.get(id); + + if (clsName == null) { registered = registerClassName(id, cls.getName()); if (registered) map.putIfAbsent(id, cls.getName()); } + else if (!clsName.equals(cls.getName())) + throw new IgniteCheckedException("Duplicate ID [id=" + id + ", oldCls=" + clsName + + ", newCls=" + cls.getName()); return registered; } @@ -133,7 +151,8 @@ public abstract class MarshallerContextAdapter implements MarshallerContext { if (clsName == null) { clsName = className(id); - assert clsName != null : id; + if (clsName == null) + throw new ClassNotFoundException("Unknown type ID: " + id); String old = map.putIfAbsent(id, clsName); @@ -144,6 +163,11 @@ public abstract class MarshallerContextAdapter implements MarshallerContext { return U.forName(clsName, ldr); } + /** {@inheritDoc} */ + @Override public boolean isSystemType(String typeName) { + return registeredSystemTypes.contains(typeName); + } + /** * Registers class name. * http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java index e614408..4b0251d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java @@ -90,13 +90,9 @@ public class MarshallerContextImpl extends MarshallerContextAdapter { try { old = cache0.tryPutIfAbsent(id, clsName); - if (old != null && !old.equals(clsName)) { - U.quietAndWarn(log, "Type ID collision detected, may affect performance " + - "(set idMapper property on marshaller to fix) [id=" + id + ", clsName1=" + clsName + - "clsName2=" + old + ']'); - - return false; - } + if (old != null && !old.equals(clsName)) + throw new IgniteCheckedException("Type ID collision detected [id=" + id + ", clsName1=" + clsName + + ", clsName2=" + old + ']'); failedCnt = 0; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java index d8cc276..96639cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/interop/InteropIgnition.java @@ -53,6 +53,8 @@ public class InteropIgnition { if (gridName != null) cfg.setGridName(gridName); + else + gridName = cfg.getGridName(); InteropBootstrap bootstrap = bootstrap(factoryId); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java index 5432c90..8ebce35 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java @@ -566,57 +566,6 @@ public abstract class GridCacheMessage implements Message { return col; } - /** - * @param map Map to marshal. - * @param ctx Context. - * @return Marshalled map. - * @throws IgniteCheckedException If failed. - */ - @SuppressWarnings("TypeMayBeWeakened") // Don't weaken type to clearly see that it's linked hash map. - @Nullable protected final LinkedHashMap marshalBooleanLinkedMap( - @Nullable LinkedHashMap map, GridCacheSharedContext ctx) throws IgniteCheckedException { - assert ctx != null; - - if (map == null) - return null; - - LinkedHashMap byteMap = U.newLinkedHashMap(map.size()); - - for (Map.Entry e : map.entrySet()) { - if (ctx.deploymentEnabled()) - prepareObject(e.getKey(), ctx); - - byteMap.put(CU.marshal(ctx, e.getKey()), e.getValue()); - } - - return byteMap; - } - - /** - * @param byteMap Map to unmarshal. - * @param ctx Context. - * @param ldr Loader. - * @return Unmarshalled map. - * @throws IgniteCheckedException If failed. - */ - @Nullable protected final LinkedHashMap unmarshalBooleanLinkedMap( - @Nullable Map byteMap, GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { - assert ldr != null; - assert ctx != null; - - if (byteMap == null) - return null; - - LinkedHashMap map = U.newLinkedHashMap(byteMap.size()); - - Marshaller marsh = ctx.marshaller(); - - for (Map.Entry e : byteMap.entrySet()) - map.put(marsh.unmarshal(e.getKey(), ldr), e.getValue()); - - return map; - } - /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { writer.setBuffer(buf); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 5582ba7..9b16388 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -265,7 +265,7 @@ public class GridCacheProcessor extends GridProcessorAdapter { // Suppress warning if at least one ATOMIC cache found. perf.add("Enable ATOMIC mode if not using transactions (set 'atomicityMode' to ATOMIC)", - cfg.getAtomicityMode() == ATOMIC); + cfg.getAtomicityMode() == ATOMIC); // Suppress warning if at least one non-FULL_SYNC mode found. perf.add("Disable fully synchronous writes (set 'writeSynchronizationMode' to PRIMARY_SYNC or FULL_ASYNC)", @@ -784,8 +784,6 @@ public class GridCacheProcessor extends GridProcessorAdapter { for (GridCacheAdapter cache : caches.values()) onKernalStart(cache); - boolean utilityCacheStarted = false; - // Wait for caches in SYNC preload mode. for (CacheConfiguration cfg : ctx.config().getCacheConfiguration()) { GridCacheAdapter cache = caches.get(maskNull(cfg.getName())); @@ -796,11 +794,8 @@ public class GridCacheProcessor extends GridProcessorAdapter { (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0)) { cache.preloader().syncFuture().get(); - if (CU.isUtilityCache(cache.name())) { + if (CU.isUtilityCache(cache.name())) ctx.cacheObjects().onUtilityCacheStarted(); - - utilityCacheStarted = true; - } } } } @@ -808,7 +803,6 @@ public class GridCacheProcessor extends GridProcessorAdapter { assert caches.containsKey(CU.MARSH_CACHE_NAME) : "Marshaller cache should be started"; assert caches.containsKey(CU.UTILITY_CACHE_NAME) : "Utility cache should be started"; - assert utilityCacheStarted; } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java index f8e5a60..dc0d1e5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java @@ -76,10 +76,9 @@ public interface IgniteCacheObjectProcessor extends GridProcessor { /** * Checks whether given class is portable. * - * @param cls Class. - * @return {@code true} If the class was registered as portable. + * @return {@code true} If portable objects are enabled. */ - public boolean isPortableClass(Class cls); + public boolean isPortableEnabled(); /** * @param obj Portable object to get field from. @@ -115,9 +114,11 @@ public interface IgniteCacheObjectProcessor extends GridProcessor { public Object unmarshal(CacheObjectContext ctx, byte[] bytes, ClassLoader clsLdr) throws IgniteCheckedException; /** + * @param ccfg Cache configuration. * @return Cache object context. + * @throws IgniteCheckedException If failed. */ - public CacheObjectContext contextForCache(CacheConfiguration ccfg); + public CacheObjectContext contextForCache(CacheConfiguration ccfg) throws IgniteCheckedException; /** * @param ctx Cache context. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java index 45fc121..3e59b10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java @@ -185,7 +185,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme } /** {@inheritDoc} */ - @Override public CacheObjectContext contextForCache(CacheConfiguration ccfg) { + @Override public CacheObjectContext contextForCache(CacheConfiguration ccfg) throws IgniteCheckedException { assert ccfg != null; CacheMemoryMode memMode = ccfg.getMemoryMode(); @@ -194,10 +194,14 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme GridQueryProcessor.isEnabled(ccfg) || !ccfg.isCopyOnRead(); - return new CacheObjectContext(ctx, + CacheObjectContext res = new CacheObjectContext(ctx, ccfg.getAffinityMapper() != null ? ccfg.getAffinityMapper() : new GridCacheDefaultAffinityKeyMapper(), - ccfg.isCopyOnRead() && memMode == ONHEAP_TIERED, + ccfg.isCopyOnRead() && memMode != OFFHEAP_VALUES, storeVal); + + ctx.resource().injectGeneric(res.defaultAffMapper()); + + return res; } /** {@inheritDoc} */ @@ -229,7 +233,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme } /** {@inheritDoc} */ - @Override public boolean isPortableClass(Class cls) { + @Override public boolean isPortableEnabled() { return false; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java index ed8e573..26b0568 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java @@ -183,7 +183,13 @@ public class DataStreamerImpl implements IgniteDataStreamer, Delayed CacheConfiguration ccfg = ctx.cache().cacheConfiguration(cacheName); - this.cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg); + try { + this.cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg); + } + catch (IgniteCheckedException e) { + throw new IgniteException("Failed to initialize cache context.", e); + } + this.cacheName = cacheName; this.flushQ = flushQ; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java index f630d01..501517b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/portable/GridPortableInputStream.java @@ -37,6 +37,16 @@ public interface GridPortableInputStream extends GridPortableStream { public byte[] readByteArray(int cnt); /** + * Reads {@code cnt} of bytes into byte array. + * + * @param arr Expected item count. + * @param off offset + * @param cnt number of bytes to read. + * @return actual length read. + */ + public int read(byte[] arr, int off, int cnt); + + /** * Read boolean value. * * @return Boolean value. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java ---------------------------------------------------------------------- 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 e187713..fed90d5 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 @@ -135,7 +135,7 @@ public class GridQueryProcessor extends GridProcessorAdapter { TypeId typeId; - if (valCls == null || ctx.cacheObjects().isPortableClass(valCls)) { + if (valCls == null || ctx.cacheObjects().isPortableEnabled()) { processPortableMeta(meta, desc); typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType())); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java index 9016b10..a303abc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java @@ -7872,6 +7872,9 @@ public abstract class IgniteUtils { if (cls != null) return cls; + if (ldr == null) + ldr = gridClassLoader; + ConcurrentMap ldrMap = classCache.get(ldr); if (ldrMap == null) { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java index 7295760..9eda849 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java @@ -39,7 +39,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.*; public abstract class VisorMultiNodeTask implements ComputeTask, R> { /** Auto-injected grid instance. */ @IgniteInstanceResource - protected IgniteEx ignite; + protected transient IgniteEx ignite; /** Debug flag. */ protected boolean debug; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java index f0e248a..dc131e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java +++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java @@ -43,4 +43,12 @@ public interface MarshallerContext { * @throws IgniteCheckedException In case of any other error. */ public Class getClass(int id, ClassLoader ldr) throws ClassNotFoundException, IgniteCheckedException; + + /** + * Checks whether the given type is a system one - JDK class or Ignite class. + * + * @param typeName Type name. + * @return {@code true} if the type is a system one, {@code false} otherwise. + */ + public boolean isSystemType(String typeName); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/main/java/org/apache/ignite/plugin/PluginContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/PluginContext.java b/modules/core/src/main/java/org/apache/ignite/plugin/PluginContext.java index 6a4f20b..b519bd1 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/PluginContext.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/PluginContext.java @@ -20,6 +20,7 @@ package org.apache.ignite.plugin; import org.apache.ignite.*; import org.apache.ignite.cluster.*; import org.apache.ignite.configuration.*; +import org.apache.ignite.marshaller.*; import org.apache.ignite.spi.*; import org.apache.ignite.spi.discovery.DiscoverySpi; @@ -40,6 +41,11 @@ public interface PluginContext { public Ignite grid(); /** + * @return Marshaller context. + */ + public MarshallerContext marshallerContext(); + + /** * Gets a collection of all grid nodes. Remote nodes are discovered via underlying * {@link DiscoverySpi} implementation used. * http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/config/tests.properties ---------------------------------------------------------------------- diff --git a/modules/core/src/test/config/tests.properties b/modules/core/src/test/config/tests.properties index 781b0f1..a9c6b77 100644 --- a/modules/core/src/test/config/tests.properties +++ b/modules/core/src/test/config/tests.properties @@ -129,7 +129,7 @@ ssh.username=uname ssh.password=passwd # SSL tests keystore. -ssl.keystore.path=modules/clients/src/test/keystore/server.jks +ssl.keystore.path=@{IGNITE_HOME}/modules/clients/src/test/keystore/server.jks ssl.keystore.password=123456 # Hadoop home directory. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java index d90bafe..3981b7f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java @@ -100,7 +100,7 @@ public class GridDiscoveryEventSelfTest extends GridCommonAbstractTest { @Override public boolean apply(Event evt) { assert evt.type() == EVT_NODE_JOINED; - evts.put(cnt.getAndIncrement(), ((DiscoveryEvent) evt).topologyNodes()); + evts.put(cnt.getAndIncrement(), ((DiscoveryEvent)evt).topologyNodes()); latch.countDown(); @@ -147,6 +147,8 @@ public class GridDiscoveryEventSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testLeaveSequenceEvents() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-932"); + try { Ignite g0 = startGrid(0); @@ -213,6 +215,8 @@ public class GridDiscoveryEventSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testMixedSequenceEvents() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-932"); + try { Ignite g0 = startGrid(0); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java index a116d54..992d7bf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java @@ -66,6 +66,8 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract * @throws Exception If failed. */ public void testFailFast() throws Exception { + fail("https://issues.apache.org/jira/browse/IGNITE-933"); + startGridsMultiThreaded(5); final CountDownLatch failLatch = new CountDownLatch(4); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/GridFailoverTaskWithPredicateSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridFailoverTaskWithPredicateSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridFailoverTaskWithPredicateSelfTest.java index eb99ce1..d14ed0d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridFailoverTaskWithPredicateSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridFailoverTaskWithPredicateSelfTest.java @@ -22,7 +22,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.compute.*; import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.failover.*; import org.apache.ignite.spi.failover.always.*; @@ -77,8 +76,6 @@ public class GridFailoverTaskWithPredicateSelfTest extends GridCommonAbstractTes } }); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java index 3af9b56..55fc2f7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java @@ -26,7 +26,6 @@ import org.apache.ignite.internal.managers.communication.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.internal.util.typedef.internal.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.plugin.extensions.communication.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.*; @@ -90,7 +89,6 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest { cfg.setDiscoverySpi(discoSpi); cfg.setCommunicationSpi(new CommunicationSpi()); - cfg.setMarshaller(new OptimizedMarshaller(false)); CacheConfiguration ccfg = defaultCacheConfiguration(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java index 0980010..ea04f35 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobStealingSelfTest.java @@ -23,7 +23,6 @@ import org.apache.ignite.compute.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.collision.jobstealing.*; import org.apache.ignite.spi.failover.jobstealing.*; @@ -290,8 +289,6 @@ public class GridJobStealingSelfTest extends GridCommonAbstractTest { cfg.setCollisionSpi(colSpi); cfg.setFailoverSpi(failSpi); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionLocalJobMultipleArgumentsSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionLocalJobMultipleArgumentsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionLocalJobMultipleArgumentsSelfTest.java index 79fcdbc..3bdc9cf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionLocalJobMultipleArgumentsSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionLocalJobMultipleArgumentsSelfTest.java @@ -56,8 +56,6 @@ public class GridProjectionLocalJobMultipleArgumentsSelfTest extends GridCommonA @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setMarshaller(new OptimizedMarshaller(false)); - CacheConfiguration cache = defaultCacheConfiguration(); cache.setCacheMode(PARTITIONED); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionContextSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionContextSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionContextSelfTest.java index d16aa1f..3dcc8f8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionContextSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskExecutionContextSelfTest.java @@ -40,15 +40,6 @@ public class GridTaskExecutionContextSelfTest extends GridCommonAbstractTest { private static final AtomicInteger CNT = new AtomicInteger(); /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setMarshaller(new OptimizedMarshaller(false)); - - return cfg; - } - - /** {@inheritDoc} */ @Override protected void beforeTestsStarted() throws Exception { startGridsMultiThreaded(2); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeEmptyClusterGroupTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeEmptyClusterGroupTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeEmptyClusterGroupTest.java index 4061881..610bec7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeEmptyClusterGroupTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeEmptyClusterGroupTest.java @@ -22,7 +22,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.compute.*; import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -51,8 +50,6 @@ public class IgniteComputeEmptyClusterGroupTest extends GridCommonAbstractTest { cfg.setDiscoverySpi(discoSpi); - cfg.setMarshaller(new OptimizedMarshaller(false)); - CacheConfiguration ccfg = defaultCacheConfiguration(); ccfg.setCacheMode(PARTITIONED); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java index 2db8062..14882ff 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteComputeTopologyExceptionTest.java @@ -44,15 +44,6 @@ public class IgniteComputeTopologyExceptionTest extends GridCommonAbstractTest { stopAllGrids(); } - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setMarshaller(new OptimizedMarshaller(false)); - - return cfg; - } - /** * @throws Exception If failed. */ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java index f2afb07..ba61890 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java @@ -78,6 +78,11 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe }; /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 10 * 60 * 1000; //10 minutes. + } + + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java index eeb9f45..3f14f49 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java @@ -27,7 +27,6 @@ import org.apache.ignite.internal.util.lang.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.internal.util.typedef.internal.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -204,8 +203,6 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest { cfg.setCacheConfiguration(cacheConfiguration(gridName)); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java index 5efe270..78ecf08 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java @@ -89,8 +89,6 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest { cfg.setCacheConfiguration(); } - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } @@ -171,7 +169,7 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest { /** * Test key. */ - private static class AffinityTestKey { + protected static class AffinityTestKey { /** Affinity key. */ @AffinityKeyMapped private final int affKey; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java index 9d4298d..f46a463 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java @@ -23,7 +23,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.internal.util.typedef.internal.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -87,8 +86,6 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest { cfg.setConnectorConfiguration(null); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java index 625a429..981ba78 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java @@ -48,59 +48,28 @@ public class GridCacheEntryMemorySizeSelfTest extends GridCommonAbstractTest { private static final int NULL_REF_SIZE = 1; /** Entry overhead. */ - private static final int ENTRY_OVERHEAD; + private static int ENTRY_OVERHEAD; /** Replicated entry overhead. */ - private static final int REPLICATED_ENTRY_OVERHEAD; + private static int REPLICATED_ENTRY_OVERHEAD; /** DHT entry overhead. */ - private static final int DHT_ENTRY_OVERHEAD; + private static int DHT_ENTRY_OVERHEAD; /** Near entry overhead. */ - private static final int NEAR_ENTRY_OVERHEAD; + private static int NEAR_ENTRY_OVERHEAD; /** Reader size. */ - private static final int READER_SIZE = 24; + private static int READER_SIZE = 24; /** Key size in bytes. */ - private static final int KEY_SIZE; + private static int KEY_SIZE; /** 1KB value size in bytes. */ - private static final int ONE_KB_VAL_SIZE; + private static int ONE_KB_VAL_SIZE; /** 2KB value size in bytes. */ - private static final int TWO_KB_VAL_SIZE; - - /** - * - */ - static { - try { - ENTRY_OVERHEAD = U.staticField(GridCacheMapEntry.class, "SIZE_OVERHEAD"); - DHT_ENTRY_OVERHEAD = U.staticField(GridDhtCacheEntry.class, "DHT_SIZE_OVERHEAD"); - NEAR_ENTRY_OVERHEAD = U.staticField(GridNearCacheEntry.class, "NEAR_SIZE_OVERHEAD"); - REPLICATED_ENTRY_OVERHEAD = DHT_ENTRY_OVERHEAD; - - Marshaller marsh = new OptimizedMarshaller(); - - marsh.setContext(new MarshallerContext() { - @Override public boolean registerClass(int id, Class cls) { - return true; - } - - @Override public Class getClass(int id, ClassLoader ldr) { - throw new UnsupportedOperationException(); - } - }); - - KEY_SIZE = marsh.marshal(1).length; - ONE_KB_VAL_SIZE = marsh.marshal(new Value(new byte[1024])).length; - TWO_KB_VAL_SIZE = marsh.marshal(new Value(new byte[2048])).length; - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } - } + private static int TWO_KB_VAL_SIZE; /** Cache mode. */ private CacheMode mode; @@ -133,6 +102,50 @@ public class GridCacheEntryMemorySizeSelfTest extends GridCommonAbstractTest { return cfg; } + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + try { + ENTRY_OVERHEAD = U.staticField(GridCacheMapEntry.class, "SIZE_OVERHEAD"); + DHT_ENTRY_OVERHEAD = U.staticField(GridDhtCacheEntry.class, "DHT_SIZE_OVERHEAD"); + NEAR_ENTRY_OVERHEAD = U.staticField(GridNearCacheEntry.class, "NEAR_SIZE_OVERHEAD"); + REPLICATED_ENTRY_OVERHEAD = DHT_ENTRY_OVERHEAD; + + Marshaller marsh = createMarshaller(); + + KEY_SIZE = marsh.marshal(1).length; + ONE_KB_VAL_SIZE = marsh.marshal(new Value(new byte[1024])).length; + TWO_KB_VAL_SIZE = marsh.marshal(new Value(new byte[2048])).length; + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + + /** + * Creates an instance of Marshaller that is used by caches during the test run. + * + * @return + */ + protected Marshaller createMarshaller() throws IgniteCheckedException { + Marshaller marsh = new OptimizedMarshaller(); + + marsh.setContext(new MarshallerContext() { + @Override public boolean registerClass(int id, Class cls) { + return true; + } + + @Override public Class getClass(int id, ClassLoader ldr) { + throw new UnsupportedOperationException(); + } + + @Override public boolean isSystemType(String typeName) { + return false; + } + }); + + return marsh; + } + /** @throws Exception If failed. */ public void testLocal() throws Exception { mode = LOCAL; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java index b4e523e..5629a96 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java @@ -22,7 +22,6 @@ import org.apache.ignite.cache.*; import org.apache.ignite.cache.eviction.lru.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -77,7 +76,6 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest { cfg.setSwapSpaceSpi(new FileSwapSpaceSpi()); cfg.setCacheConfiguration(cacheConfiguration()); - cfg.setMarshaller(new OptimizedMarshaller(false)); return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java index 35de371..7133ba8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java @@ -22,7 +22,6 @@ import org.apache.ignite.cache.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -69,8 +68,6 @@ public class GridCacheReferenceCleanupSelfTest extends GridCommonAbstractTest { cfg.setCacheConfiguration(cacheCfg); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java index 1a62d4b..62a5b20 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java @@ -22,7 +22,6 @@ import org.apache.ignite.cache.*; import org.apache.ignite.cache.store.*; import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -95,8 +94,6 @@ public abstract class IgniteCacheAbstractTest extends GridCommonAbstractTest { cfg.setCacheConfiguration(cacheConfiguration(gridName)); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java index bb449e0..aa24eab 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java @@ -695,11 +695,11 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb if (i % 2 == 0) cache.put(key, i + 1); else - cache.invoke(key, new SetValueProcessor(i + 1)); + cache.invoke(key, new EntrySetValueProcessor(i + 1)); } // Invoke processor does not update value, should not trigger event. - assertEquals(String.valueOf(UPDATES), cache.invoke(key, new ToStringProcessor())); + assertEquals(String.valueOf(UPDATES), cache.invoke(key, new EntryToStringProcessor())); assertFalse(cache.putIfAbsent(key, -1)); @@ -1122,7 +1122,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb /** * */ - protected static class ToStringProcessor implements EntryProcessor { + protected static class EntryToStringProcessor implements EntryProcessor { /** {@inheritDoc} */ @Override public String process(MutableEntry e, Object... arguments) throws EntryProcessorException { @@ -1131,21 +1131,21 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb /** {@inheritDoc} */ @Override public String toString() { - return S.toString(ToStringProcessor.class, this); + return S.toString(EntryToStringProcessor.class, this); } } /** * */ - protected static class SetValueProcessor implements EntryProcessor { + protected static class EntrySetValueProcessor implements EntryProcessor { /** */ private Integer val; /** * @param val Value to set. */ - public SetValueProcessor(Integer val) { + public EntrySetValueProcessor(Integer val) { this.val = val; } @@ -1159,7 +1159,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb /** {@inheritDoc} */ @Override public String toString() { - return S.toString(SetValueProcessor.class, this); + return S.toString(EntrySetValueProcessor.class, this); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java index 412ad62..88cd623 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java @@ -18,9 +18,7 @@ package org.apache.ignite.internal.processors.cache.datastructures; import org.apache.ignite.*; -import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import java.util.*; @@ -36,15 +34,6 @@ public abstract class GridCacheAtomicReferenceMultiNodeAbstractTest extends Igni return GRID_CNT; } - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setMarshaller(new OptimizedMarshaller(false)); - - return cfg; - } - /** * JUnit. * http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java index b39017e..33fe127 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java @@ -25,7 +25,6 @@ import org.apache.ignite.internal.util.tostring.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.internal.util.typedef.internal.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.resources.*; import org.apache.ignite.testframework.*; @@ -95,7 +94,6 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends IgniteColl cfg.setSystemThreadPoolSize(RETRIES * 2); - cfg.setMarshaller(new OptimizedMarshaller(false)); cfg.setConnectorConfiguration(null); return cfg; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java index f45d860..3997ebc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java @@ -23,7 +23,6 @@ import org.apache.ignite.internal.util.tostring.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.internal.util.typedef.internal.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.resources.*; import org.apache.ignite.testframework.*; @@ -71,15 +70,6 @@ public abstract class GridCacheQueueRotativeMultiNodeAbstractTest extends Ignite assert G.allGrids().isEmpty(); } - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setMarshaller(new OptimizedMarshaller(false)); - - return cfg; - } - /** * JUnit. * http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java index 22af54a..6383f91 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java @@ -48,15 +48,6 @@ public abstract class GridCacheSetAbstractSelfTest extends IgniteCollectionAbstr } /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - cfg.setMarshaller(new OptimizedMarshaller(false)); - - return cfg; - } - - /** {@inheritDoc} */ @Override protected CollectionConfiguration collectionConfiguration() { CollectionConfiguration colCfg = super.collectionConfiguration(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java index 0db5e95..f8a15bc 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java @@ -23,7 +23,6 @@ import org.apache.ignite.configuration.*; import org.apache.ignite.internal.*; import org.apache.ignite.internal.processors.cache.*; import org.apache.ignite.internal.util.typedef.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -60,8 +59,6 @@ public abstract class GridCacheAbstractJobExecutionTest extends GridCommonAbstra cfg.setDiscoverySpi(disco); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadLifecycleAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadLifecycleAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadLifecycleAbstractTest.java index 9818176..e670a0a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadLifecycleAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePreloadLifecycleAbstractTest.java @@ -21,7 +21,6 @@ import org.apache.ignite.cache.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.internal.*; import org.apache.ignite.lifecycle.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -80,7 +79,6 @@ public abstract class GridCachePreloadLifecycleAbstractTest extends GridCommonAb c.setDeploymentMode(DeploymentMode.SHARED); c.setNetworkTimeout(10000); c.setConnectorConfiguration(null); - c.setMarshaller(new OptimizedMarshaller(false)); // c.setPeerClassLoadingLocalClassPathExclude(GridCachePreloadLifecycleAbstractTest.class.getName(), // MyValue.class.getName()); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractTransformWriteThroughSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractTransformWriteThroughSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractTransformWriteThroughSelfTest.java index 5c1d674..62003e3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractTransformWriteThroughSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAbstractTransformWriteThroughSelfTest.java @@ -21,7 +21,6 @@ import org.apache.ignite.*; import org.apache.ignite.cache.affinity.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.processors.cache.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -99,8 +98,6 @@ public abstract class GridCacheAbstractTransformWriteThroughSelfTest extends Gri @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setMarshaller(new OptimizedMarshaller(false)); - TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); discoSpi.setIpFinder(IP_FINDER); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java index ec3a3e4..28ef0a3 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionSelfTest.java @@ -23,7 +23,6 @@ import org.apache.ignite.configuration.*; import org.apache.ignite.internal.processors.cache.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -72,8 +71,6 @@ public class GridCacheNearEvictionSelfTest extends GridCommonAbstractTest { c.setDiscoverySpi(disco); - c.setMarshaller(new OptimizedMarshaller(false)); - return c; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java index f5eb189..6138022 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedHitsAndMissesSelfTest.java @@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.near; import org.apache.ignite.*; import org.apache.ignite.cache.*; import org.apache.ignite.configuration.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -51,8 +50,6 @@ public class GridCachePartitionedHitsAndMissesSelfTest extends GridCommonAbstrac @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setMarshaller(new OptimizedMarshaller(false)); - // DiscoverySpi TcpDiscoverySpi disco = new TcpDiscoverySpi(); disco.setIpFinder(IP_FINDER); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java index 6ccfbc2..8896840 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/GridCacheSwapScanQueryAbstractSelfTest.java @@ -22,7 +22,6 @@ import org.apache.ignite.configuration.*; import org.apache.ignite.internal.*; import org.apache.ignite.internal.processors.cache.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*; @@ -56,8 +55,6 @@ public abstract class GridCacheSwapScanQueryAbstractSelfTest extends GridCommonA @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setMarshaller(new OptimizedMarshaller(false)); - TcpDiscoverySpi disco = new TcpDiscoverySpi(); disco.setIpFinder(ipFinder); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java index 378d5a3..4681071 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java @@ -99,8 +99,6 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo cfg.setDiscoverySpi(disco); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java index 0b178ff..381db74 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java @@ -23,7 +23,6 @@ import org.apache.ignite.compute.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.util.typedef.*; import org.apache.ignite.lang.*; -import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -58,8 +57,6 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(gridName); - cfg.setMarshaller(new OptimizedMarshaller(false)); - TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); discoSpi.setIpFinder(ipFinder); @@ -95,7 +92,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { /** * Test runnable job. */ - private static class TestRunnable implements IgniteRunnable { + private static class ClosureTestRunnable implements IgniteRunnable { /** */ @IgniteInstanceResource private Ignite ignite; @@ -130,7 +127,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { /** * Test callable job. */ - private static class TestCallable extends AbstractTestCallable { + private static class ClosureTestCallable extends AbstractTestCallable { /** {@inheritDoc} */ @Override public Integer call() { log.info("Callable job executed on node: " + ignite.cluster().localNode().id()); @@ -144,11 +141,11 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { /** * Test callable job which throws class not found exception. */ - private static class TestCallableError extends AbstractTestCallable implements Externalizable { + private static class ClosureTestCallableError extends AbstractTestCallable implements Externalizable { /** * */ - public TestCallableError() { + public ClosureTestCallableError() { // No-op. } @@ -234,7 +231,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { * @return Future object. */ private ComputeTaskFuture runAsync(int idx, - Collection jobs, + Collection jobs, @Nullable IgnitePredicate p) { assert idx >= 0 && idx < NODES_CNT; @@ -302,7 +299,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { * @param p Optional node predicate. * @return Future object. */ - private ComputeTaskFuture> callAsync(int idx, Collection jobs, + private ComputeTaskFuture> callAsync(int idx, Collection jobs, @Nullable IgnitePredicate p) { assert idx >= 0 && idx < NODES_CNT; assert !F.isEmpty(jobs); @@ -334,7 +331,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testRunAsyncSingle() throws Exception { - IgniteRunnable job = new TestRunnable(); + IgniteRunnable job = new ClosureTestRunnable(); ComputeTaskFuture fut = broadcast(0, job, null); @@ -360,7 +357,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testRunAsyncMultiple() throws Exception { - Collection jobs = F.asList(new TestRunnable(), new TestRunnable()); + Collection jobs = F.asList(new ClosureTestRunnable(), new ClosureTestRunnable()); ComputeTaskFuture fut = runAsync(0, jobs, null); @@ -374,7 +371,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testCallAsyncSingle() throws Exception { - IgniteCallable job = new TestCallable(); + IgniteCallable job = new ClosureTestCallable(); ComputeTaskFuture> fut1 = broadcast(0, job, null); @@ -404,7 +401,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { public void testCallAsyncErrorNoFailover() throws Exception { IgniteCompute comp = compute(grid(0).cluster().forPredicate(F.notEqualTo(grid(0).localNode()))).withAsync(); - comp.withNoFailover().call(new TestCallableError()); + comp.withNoFailover().call(new ClosureTestCallableError()); ComputeTaskFuture fut = comp.future(); @@ -422,7 +419,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testWithName() throws Exception { - grid(0).compute().withName("TestTaskName").call(new TestCallable()); + grid(0).compute().withName("TestTaskName").call(new ClosureTestCallable()); } /** @@ -460,7 +457,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testCallAsyncMultiple() throws Exception { - Collection jobs = F.asList(new TestCallable(), new TestCallable()); + Collection jobs = F.asList(new ClosureTestCallable(), new ClosureTestCallable()); ComputeTaskFuture> fut = callAsync(0, jobs, null); @@ -479,7 +476,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest { * @throws Exception If failed. */ public void testReduceAsync() throws Exception { - Collection jobs = F.asList(new TestCallable(), new TestCallable()); + Collection jobs = F.asList(new ClosureTestCallable(), new ClosureTestCallable()); IgniteCompute comp = grid(0).compute().withAsync(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java index 9ffef4b..5ce2efd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java @@ -85,8 +85,6 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest { if (include) cfg.setUserAttributes(F.asMap("include", true)); - cfg.setMarshaller(new OptimizedMarshaller(false)); - return cfg; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java index 81751f2..80bf9e0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java @@ -91,8 +91,6 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest { cfg.setIncludeProperties(); - cfg.setMarshaller(new OptimizedMarshaller(false)); - if (useCache) { CacheConfiguration cc = defaultCacheConfiguration(); @@ -893,18 +891,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest { try (IgniteDataStreamer ldr = ignite.dataStreamer(null)) { ldr.allowOverwrite(true); - ldr.receiver(new StreamReceiver() { - @Override - public void receive(IgniteCache cache, - Collection> entries) { - for (Map.Entry e : entries) { - assertTrue(e.getKey() instanceof String); - assertTrue(e.getValue() instanceof TestObject); - - cache.put(e.getKey(), new TestObject(e.getValue().val + 1)); - } - } - }); + ldr.receiver(getStreamReceiver()); for (int i = 0; i < 100; i++) ldr.addData(String.valueOf(i), new TestObject(i)); @@ -927,14 +914,14 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest { /** * */ - private static class TestObject { + public static class TestObject { /** Value. */ - private final int val; + public final int val; /** * @param val Value. */ - private TestObject(int val) { + public TestObject(int val) { this.val = val; } @@ -958,6 +945,13 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest { } /** + * @return Stream receiver. + */ + protected StreamReceiver getStreamReceiver() { + return new TestDataReceiver(); + } + + /** * */ @SuppressWarnings("PublicInnerClass") @@ -977,4 +971,20 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest { storeMap.remove(key); } } + + /** + * + */ + private static class TestDataReceiver implements StreamReceiver { + /** {@inheritDoc} */ + @Override public void receive(IgniteCache cache, + Collection> entries) { + for (Map.Entry e : entries) { + assertTrue(e.getKey() instanceof String); + assertTrue(e.getValue() instanceof TestObject); + + cache.put(e.getKey(), new TestObject(e.getValue().val + 1)); + } + } + } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/37c56f69/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java index bdf9929..c81ed56 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java @@ -644,7 +644,7 @@ public class GridNioSelfTest extends GridCommonAbstractTest { try { client = createClient(U.getLocalHost(), PORT, U.getLocalHost()); - MessageWithId msg = new MessageWithId(); + MessageWithId msg = new MessageWithId(idProvider.getAndIncrement()); byte[] data = serializeMessage(msg); @@ -746,7 +746,7 @@ public class GridNioSelfTest extends GridCommonAbstractTest { client = createClient(U.getLocalHost(), PORT, U.getLocalHost()); while (cntr.getAndIncrement() < MSG_CNT * THREAD_CNT) { - MessageWithId msg = new MessageWithId(); + MessageWithId msg = new MessageWithId(idProvider.getAndIncrement()); byte[] data = serializeMessage(msg); @@ -1408,7 +1408,14 @@ public class GridNioSelfTest extends GridCommonAbstractTest { */ private static class MessageWithId implements Serializable { /** */ - private final int id = idProvider.getAndIncrement(); + private final int id; + + /** + * @param id Message ID. + */ + public MessageWithId(int id) { + this.id = id; + } /** */ @SuppressWarnings({"unused"})