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 177F11813F for ; Sat, 2 Jan 2016 21:22:12 +0000 (UTC) Received: (qmail 3794 invoked by uid 500); 2 Jan 2016 21:22:12 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 3707 invoked by uid 500); 2 Jan 2016 21:22:12 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 3687 invoked by uid 99); 2 Jan 2016 21:22:11 -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; Sat, 02 Jan 2016 21:22:11 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id BE3C1DFE52; Sat, 2 Jan 2016 21:22:11 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Sat, 02 Jan 2016 21:22:13 -0000 Message-Id: <1b98a4a5c7ac4d799d8f67e7623d8d14@git.apache.org> In-Reply-To: <7f4c0c8f0e0c4882a88c3fb8bb52da93@git.apache.org> References: <7f4c0c8f0e0c4882a88c3fb8bb52da93@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [3/3] ignite git commit: IGNITE-2330: Simplified GridFunc. IGNITE-2330: Simplified GridFunc. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/80579253 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/80579253 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/80579253 Branch: refs/heads/master Commit: 80579253febd6389dbb3a84706671cc8083df1f2 Parents: dffcb83 Author: vozerov-gridgain Authored: Sun Jan 3 01:15:24 2016 +0400 Committer: vozerov-gridgain Committed: Sun Jan 3 01:15:24 2016 +0400 ---------------------------------------------------------------------- .../apache/ignite/internal/IgniteKernal.java | 27 +- .../internal/cluster/ClusterGroupAdapter.java | 10 +- .../discovery/GridDiscoveryManager.java | 19 +- .../loadbalancer/GridLoadBalancerManager.java | 8 +- .../processors/cache/GridCacheAdapter.java | 14 +- .../processors/cache/GridCacheContext.java | 3 +- .../cache/GridCacheEvictionManager.java | 5 +- .../processors/cache/GridCacheIoManager.java | 5 +- .../processors/cache/GridCacheIterator.java | 4 +- .../processors/cache/GridCacheKeySet.java | 2 +- .../GridCachePartitionExchangeManager.java | 12 +- .../processors/cache/GridCacheProcessor.java | 9 +- .../processors/cache/GridCacheSwapManager.java | 7 +- .../processors/cache/GridCacheUtils.java | 92 + .../cache/GridCacheValueCollection.java | 5 +- .../processors/cache/IgniteCacheProxy.java | 4 +- .../dht/GridClientPartitionTopology.java | 3 +- .../dht/GridDhtPartitionTopologyImpl.java | 3 +- .../dht/GridDhtTransactionalCacheAdapter.java | 3 +- .../distributed/dht/GridDhtTxLocalAdapter.java | 3 +- .../distributed/dht/GridDhtTxPrepareFuture.java | 5 +- .../dht/atomic/GridDhtAtomicCache.java | 21 +- .../dht/preloader/GridDhtForceKeysFuture.java | 8 +- .../distributed/near/GridNearCacheAdapter.java | 11 +- .../local/atomic/GridLocalAtomicCache.java | 17 +- .../GridCacheAtomicStampedImpl.java | 5 +- .../internal/processors/igfs/IgfsProcessor.java | 8 +- .../dotnet/PlatformDotNetCacheStore.java | 4 +- .../top/GridTopologyCommandHandler.java | 3 +- .../org/apache/ignite/internal/util/F0.java | 325 +--- .../internal/util/GridExecutionStatistics.java | 4 +- .../ignite/internal/util/IgniteUtils.java | 4 +- .../ignite/internal/util/lang/GridFunc.java | 1776 ++++-------------- .../ignite/internal/util/lang/GridTuple3.java | 1 - .../ignite/internal/util/lang/GridTuple4.java | 1 - .../ignite/internal/util/lang/GridTuple5.java | 1 - .../ignite/internal/util/lang/GridTuple6.java | 1 - .../ignite/internal/util/lang/GridTupleV.java | 1 - .../java/org/apache/ignite/lang/IgniteUuid.java | 2 +- .../memory/MemoryEventStorageSpi.java | 1 + .../cache/GridCacheLuceneQueryIndexTest.java | 4 +- .../distributed/GridCacheEventAbstractTest.java | 2 +- .../IpcSharedMemoryCrashDetectionSelfTest.java | 16 +- .../ignite/lang/GridBasicPerformanceTest.java | 10 +- .../ignite/lang/GridFuncPerformanceTest.java | 102 - .../ignite/loadtest/GridLoadTestStatistics.java | 2 +- .../query/h2/opt/GridH2TreeIndex.java | 4 +- 47 files changed, 583 insertions(+), 1994 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 14b5816..7fc0313 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -589,8 +589,16 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { @Override public List getLifecycleBeansFormatted() { LifecycleBean[] beans = cfg.getLifecycleBeans(); - return F.isEmpty(beans) ? Collections.emptyList() : - (List)F.transform(beans, F.string()); + if (F.isEmpty(beans)) + return Collections.emptyList(); + else { + List res = new ArrayList<>(beans.length); + + for (LifecycleBean bean : beans) + res.add(String.valueOf(bean)); + + return res; + } } /** @@ -2269,16 +2277,19 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable { Collection objs = new ArrayList<>(); if (!F.isEmpty(cfg.getLifecycleBeans())) - F.copy(objs, cfg.getLifecycleBeans()); + Collections.addAll(objs, cfg.getLifecycleBeans()); if (!F.isEmpty(cfg.getSegmentationResolvers())) - F.copy(objs, cfg.getSegmentationResolvers()); + Collections.addAll(objs, cfg.getSegmentationResolvers()); - if (cfg.getConnectorConfiguration() != null) - F.copy(objs, cfg.getConnectorConfiguration().getMessageInterceptor(), - cfg.getConnectorConfiguration().getSslContextFactory()); + if (cfg.getConnectorConfiguration() != null) { + objs.add(cfg.getConnectorConfiguration().getMessageInterceptor()); + objs.add(cfg.getConnectorConfiguration().getSslContextFactory()); + } - F.copy(objs, cfg.getMarshaller(), cfg.getGridLogger(), cfg.getMBeanServer()); + objs.add(cfg.getMarshaller()); + objs.add(cfg.getGridLogger()); + objs.add(cfg.getMBeanServer()); return objs; } http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java index 75168a1..a153b83 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/ClusterGroupAdapter.java @@ -51,6 +51,7 @@ import org.apache.ignite.internal.IgniteServicesImpl; import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.executor.GridExecutorService; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; +import org.apache.ignite.internal.util.lang.GridNodePredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.U; @@ -367,8 +368,7 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable { guard(); try { - if (p != null) - ctx.resource().injectGeneric(p); + ctx.resource().injectGeneric(p); return new ClusterGroupAdapter(ctx, subjId, this.p != null ? F.and(p, this.p) : p); } @@ -703,6 +703,7 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable { } /** {@inheritDoc} */ + @SuppressWarnings("unchecked") @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { gridName = U.readString(in); subjId = U.readUuid(in); @@ -919,7 +920,7 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable { ClusterNode node = isOldest ? U.oldest(super.nodes(), null) : U.youngest(super.nodes(), null); - IgnitePredicate p = F.nodeForNodes(node); + IgnitePredicate p = new GridNodePredicate(node); state = new AgeClusterGroupState(node, p, lastTopVer); } @@ -961,8 +962,7 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable { guard(); try { - if (p != null) - ctx.resource().injectGeneric(p); + ctx.resource().injectGeneric(p); return new ClusterGroupAdapter(ctx, this.subjId, new GroupPredicate(this, p)); } http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index 72a2bef..844fd0f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -30,7 +30,6 @@ import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -79,7 +78,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.jobmetrics.GridJobMetrics; import org.apache.ignite.internal.processors.security.SecurityContext; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridBoundedConcurrentOrderedMap; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -1317,13 +1315,26 @@ public class GridDiscoveryManager extends GridManagerAdapter { } /** + * Gets collection of node for given node IDs. + * + * @param ids Ids to include. + * @return Collection with all alive nodes for given IDs. + */ + public Collection nodes(@Nullable Collection ids) { + return F.isEmpty(ids) ? Collections.emptyList() : + F.view( + F.viewReadOnly(ids, U.id2Node(ctx)), + F.notNull()); + } + + /** * Gets collection of node for given node IDs and predicates. * * @param ids Ids to include. * @param p Filter for IDs. * @return Collection with all alive nodes for given IDs. */ - public Collection nodes(@Nullable Collection ids, IgnitePredicate... p) { + public Collection nodes(@Nullable Collection ids, IgnitePredicate p) { return F.isEmpty(ids) ? Collections.emptyList() : F.view( F.viewReadOnly(ids, U.id2Node(ctx), p), @@ -2565,7 +2576,7 @@ public class GridDiscoveryManager extends GridManagerAdapter { nearEnabledCaches = Collections.unmodifiableSet(nearEnabledSet); daemonNodes = Collections.unmodifiableList(new ArrayList<>( - F.view(F.concat(false, loc, rmts), F0.not(daemonFilter)))); + F.view(F.concat(false, loc, rmts), F.not(daemonFilter)))); Map nodeMap = new HashMap<>(allNodes().size() + daemonNodes.size(), 1.0f); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java index 631168b..efe09b0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/loadbalancer/GridLoadBalancerManager.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.managers.loadbalancer; import java.util.Collection; +import java.util.LinkedList; import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -92,7 +93,12 @@ public class GridLoadBalancerManager extends GridManagerAdapter nodes = F.loseList(top, true, exclNodes); + List nodes = new LinkedList<>(); + + for (ClusterNode topNode : top) { + if (!exclNodes.contains(topNode)) + nodes.add(topNode); + } if (nodes.isEmpty()) return null; http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java index 5d4c386..a4cf737 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java @@ -106,7 +106,6 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.lang.GridClosureException; import org.apache.ignite.internal.util.lang.GridTriple; import org.apache.ignite.internal.util.tostring.GridToStringExclude; -import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.C2; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CI2; @@ -2150,12 +2149,7 @@ public abstract class GridCacheAdapter implements IgniteInternalCache>>(keys.size() == 1) { @Nullable @Override public Map> op(IgniteTxLocalAdapter tx) throws IgniteCheckedException { - Map> invokeMap = F.viewAsMap(keys, - new C1>() { - @Override public EntryProcessor apply(K k) { - return entryProcessor; - } - }); + Map> invokeMap = CU.invokeMap(keys, entryProcessor); IgniteInternalFuture fut = tx.invokeAsync(ctx, invokeMap, args); @@ -2222,11 +2216,7 @@ public abstract class GridCacheAdapter implements IgniteInternalCache fut = asyncOp(new AsyncInOp(keys) { @Override public IgniteInternalFuture inOp(IgniteTxLocalAdapter tx) { - Map> invokeMap = F.viewAsMap(keys, new C1>() { - @Override public EntryProcessor apply(K k) { - return entryProcessor; - } - }); + Map> invokeMap = CU.invokeMap(keys, entryProcessor); return tx.invokeAsync(ctx, invokeMap, args); } http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java index c10ebf3..ef49567 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java @@ -85,7 +85,6 @@ import org.apache.ignite.internal.processors.closure.GridClosureProcessor; import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor; import org.apache.ignite.internal.processors.plugin.CachePluginManager; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.lang.GridFunc; import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -1512,7 +1511,7 @@ public class GridCacheContext implements Externalizable { Collection nearNodes = null; if (!F.isEmpty(readers)) { - nearNodes = discovery().nodes(readers, F0.notEqualTo(nearNodeId)); + nearNodes = discovery().nodes(readers, F.notEqualTo(nearNodeId)); if (log.isDebugEnabled()) log.debug("Mapping entry to near nodes [nodes=" + U.nodeIds(nearNodes) + ", entry=" + entry + ']'); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java index 845e204..f542f4b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java @@ -63,7 +63,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridBusyLock; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.GridUnsafe; @@ -1310,7 +1309,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter { try { GridCacheVersion ver = e.version(); - return info.version().equals(ver) && F.isAll(info.filter()); + return info.version().equals(ver) && F.isAll(e, info.filter()); } catch (GridCacheEntryRemovedException ignored) { return false; @@ -1341,7 +1340,7 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter { Collection backups; if (evictSync) - backups = F.view(cctx.dht().topology().nodes(entry.partition(), topVer), F0.notEqualTo(cctx.localNode())); + backups = F.view(cctx.dht().topology().nodes(entry.partition(), topVer), F.notEqualTo(cctx.localNode())); else backups = Collections.emptySet(); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java index 0aa8b1b..4c9cdf2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java @@ -57,7 +57,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPr import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.GridSpinReadWriteLock; import org.apache.ignite.internal.util.typedef.CI1; @@ -731,7 +730,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter { } if (added) { - if (!F.exist(F.nodeIds(nodes), F0.not(F.contains(leftIds)))) { + if (!F.exist(F.nodeIds(nodes), F.not(F.contains(leftIds)))) { if (log.isDebugEnabled()) log.debug("Message will not be sent because all nodes left topology [msg=" + msg + ", nodes=" + U.toShortString(nodes) + ']'); @@ -767,7 +766,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter { U.sleep(retryDelay); } - if (!F.exist(F.nodeIds(nodes), F0.not(F.contains(leftIds)))) { + if (!F.exist(F.nodeIds(nodes), F.not(F.contains(leftIds)))) { if (log.isDebugEnabled()) log.debug("Message will not be sent because all nodes left topology [msg=" + msg + ", nodes=" + U.toShortString(nodes) + ']'); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIterator.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIterator.java index 19da4a8..e4a471c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIterator.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIterator.java @@ -19,6 +19,8 @@ package org.apache.ignite.internal.processors.cache; import java.util.Iterator; import javax.cache.Cache; + +import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridSerializableIterator; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.lang.IgniteClosure; @@ -57,7 +59,7 @@ public class GridCacheIterator implements GridSerializableIterator { ) { this.cctx = cctx; - it = F.iterator0(c, false, filter); + it = F.identityIterator(c, F0.and(filter)); this.trans = trans; } http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheKeySet.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheKeySet.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheKeySet.java index 6d18b7d..af0c704 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheKeySet.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheKeySet.java @@ -66,7 +66,7 @@ public class GridCacheKeySet extends GridSerializableSet { /** {@inheritDoc} */ @Override public Iterator iterator() { - return new GridCacheIterator<>(ctx, map.values(), F.cacheEntry2Key(), filter); + return new GridCacheIterator<>(ctx, map.values(), CU.cacheEntry2Key(), filter); } /** {@inheritDoc} */ http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index a0f7f93..4732597 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -80,6 +80,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.thread.IgniteThread; @@ -1270,9 +1271,16 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana } // After workers line up and before preloading starts we initialize all futures. - if (log.isDebugEnabled()) + if (log.isDebugEnabled()) { + IgnitePredicate p = new IgnitePredicate>() { + @Override public boolean apply(IgniteInternalFuture f) { + return !f.isDone(); + } + }; + log.debug("Before waiting for exchange futures [futs" + - F.view(exchFuts.values(), F.unfinishedFutures()) + ", worker=" + this + ']'); + F.view(exchFuts.values(), p) + ", worker=" + this + ']'); + } // Take next exchange future. exchFut = poll(futQ, timeout, this); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/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 ff02e70..f6fc9d4 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 @@ -96,7 +96,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager; import org.apache.ignite.internal.processors.plugin.CachePluginManager; import org.apache.ignite.internal.processors.query.GridQueryProcessor; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -1160,8 +1159,12 @@ public class GridCacheProcessor extends GridProcessorAdapter { log.debug("Executed onKernalStart() callback for DHT cache: " + dht.name()); } - for (GridCacheManager mgr : F.view(ctx.managers(), F0.notContains(dhtExcludes(ctx)))) - mgr.onKernalStart(); + Collection excluded = dhtExcludes(ctx); + + for (GridCacheManager mgr : ctx.managers()) { + if (!excluded.contains(mgr)) + mgr.onKernalStart(); + } cache.onKernalStart(); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java index 37b5e15..4490f63 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java @@ -42,7 +42,6 @@ import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.GridEmptyCloseableIterator; -import org.apache.ignite.internal.util.GridEmptyIterator; import org.apache.ignite.internal.util.GridWeakIterator; import org.apache.ignite.internal.util.lang.GridCloseableIterator; import org.apache.ignite.internal.util.lang.GridTuple; @@ -1581,7 +1580,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { */ public Iterator> lazySwapIterator() throws IgniteCheckedException { if (!swapEnabled) - return new GridEmptyIterator<>(); + return F.emptyIterator(); return lazyIterator(cctx.gridSwap().rawIterator(spaceName)); } @@ -1672,7 +1671,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { private Iterator> lazyIterator( final GridCloseableIterator> it) { if (it == null) - return new GridEmptyIterator<>(); + return F.emptyIterator(); checkIteratorQueue(); @@ -1743,7 +1742,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter { private Iterator keyIterator( final GridCloseableIterator> it) { if (it == null) - return new GridEmptyIterator<>(); + return F.emptyIterator(); checkIteratorQueue(); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java index 51f6dcd..b7baf7e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java @@ -43,6 +43,8 @@ import javax.cache.configuration.Factory; import javax.cache.expiry.Duration; import javax.cache.expiry.ExpiryPolicy; import javax.cache.integration.CacheWriterException; +import javax.cache.processor.EntryProcessor; + import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -284,6 +286,41 @@ public class GridCacheUtils { } }; + /** */ + private static final IgniteClosure CACHE_ENTRY_KEY = new IgniteClosure() { + @Override public Object apply(Object o) { + return ((Cache.Entry)o).getKey(); + } + + @Override public String toString() { + return "Map entry to key transformer closure."; + } + }; + + /** */ + private static final IgniteClosure CACHE_ENTRY_VAL_GET = new IgniteClosure() { + @SuppressWarnings({"unchecked"}) + @Nullable @Override public Object apply(Object o) { + return ((Cache.Entry)o).getValue(); + } + + @Override public String toString() { + return "Cache entry to get-value transformer closure."; + } + }; + + /** */ + private static final IgnitePredicate CACHE_ENTRY_HAS_PEEK_VAL = new IgnitePredicate() { + @SuppressWarnings({"unchecked"}) + @Override public boolean apply(Object o) { + return ((Cache.Entry)o).getValue() != null; + } + + @Override public String toString() { + return "Cache entry has-peek-value predicate."; + } + }; + /** * Ensure singleton. */ @@ -1863,4 +1900,59 @@ public class GridCacheUtils { return res; } + + /** + * Create invoke map for the given key set. All provided values will be set to the passed entry processor. + * + * @param keys Keys. + * @param entryProc Entry processor. + * @return Invoke map. + */ + @SuppressWarnings("unchecked") + public static Map> invokeMap( + final Set keys, final EntryProcessor entryProc) { + return F.viewAsMap(keys, + new C1>() { + @Override public EntryProcessor apply(K k) { + return entryProc; + } + }); + } + + /** + * Gets closure that returns key for cache entry. The closure internally + * delegates to {@link javax.cache.Cache.Entry#getKey()} method. + * + * @param Key type. + * @return Closure that returns key for an entry. + */ + @SuppressWarnings({"unchecked"}) + public static IgniteClosure, K> cacheEntry2Key() { + return (IgniteClosure, K>)CACHE_ENTRY_KEY; + } + + /** + * Gets closure that returns value for an entry. The closure internally + * delegates to {@link javax.cache.Cache.Entry#get(Object)} method. + * + * @param Key type. + * @param Value type. + * @return Closure that returns value for an entry. + */ + @SuppressWarnings({"unchecked"}) + public static IgniteClosure, V> cacheEntry2Get() { + return (IgniteClosure, V>)CACHE_ENTRY_VAL_GET; + } + + /** + * Gets predicate which returns {@code true} if entry has peek value. + * + * @param Cache key type. + * @param Cache value type. + * @return Predicate which returns {@code true} if entry has peek value. + */ + @SuppressWarnings({"unchecked"}) + public static IgnitePredicate> cacheHasPeekValue() { + return (IgnitePredicate>)CACHE_ENTRY_HAS_PEEK_VAL; + } } http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueCollection.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueCollection.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueCollection.java index 2d1686e..68b54b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueCollection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueCollection.java @@ -27,6 +27,7 @@ import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridSerializableCollection; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -72,8 +73,8 @@ public class GridCacheValueCollection extends GridSerializableCollection( ctx, map.values(), - F.cacheEntry2Get(), - ctx.vararg(F0.and(filter, F.cacheHasPeekValue())) + CU.cacheEntry2Get(), + ctx.vararg(F0.and(filter, CU.cacheHasPeekValue())) ) { { advance(); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java index 27a7587..a6e0ea8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java @@ -69,7 +69,6 @@ import org.apache.ignite.internal.processors.cache.query.CacheQuery; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.GridCloseableIteratorAdapter; -import org.apache.ignite.internal.util.GridEmptyIterator; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.lang.GridClosureException; import org.apache.ignite.internal.util.lang.IgniteOutClosureX; @@ -77,6 +76,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.CX1; +import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; @@ -579,7 +579,7 @@ public class IgniteCacheProxy extends AsyncSupportAdapter>() { @Override public Iterator> iterator() { - return cur != null ? cur.iterator() : new GridEmptyIterator>(); + return cur != null ? cur.iterator() : F.>emptyIterator(); } @Override public List> getAll() { http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java index 8aef5ad..9c18ad0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java @@ -37,7 +37,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; @@ -698,7 +697,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology { // Remove obsolete mappings. if (cur != null) { - for (Integer p : F.view(cur.keySet(), F0.notIn(parts.keySet()))) { + for (Integer p : F.view(cur.keySet(), F.notIn(parts.keySet()))) { Set ids = part2node.get(p); if (ids != null) http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java index a0709c5..cdf0fc2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java @@ -41,7 +41,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridAtomicLong; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; @@ -1055,7 +1054,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology { // Remove obsolete mappings. if (cur != null) { - for (Integer p : F.view(cur.keySet(), F0.notIn(parts.keySet()))) { + for (Integer p : F.view(cur.keySet(), F.notIn(parts.keySet()))) { Set ids = part2node.get(p); if (ids != null) http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java index ae24ed1..2b40436 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java @@ -60,7 +60,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.lang.GridClosureException; @@ -1382,7 +1381,7 @@ public abstract class GridDhtTransactionalCacheAdapter extends GridDhtCach Collection nearNodes = null; if (!F.isEmpty(readers)) { - nearNodes = ctx.discovery().nodes(readers, F0.not(F.idForNodeId(nodeId))); + nearNodes = ctx.discovery().nodes(readers, F.not(F.idForNodeId(nodeId))); if (log.isDebugEnabled()) log.debug("Mapping entry to near nodes [nodes=" + U.toShortString(nearNodes) + ", entry=" + cached + http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java index 534a560..1fe3ae2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java @@ -42,7 +42,6 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanMap; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; @@ -641,7 +640,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter { // Otherwise, during rollback we will not know whether locks need // to be rolled back. // Loose all skipped and previously locked (we cannot reenter locks here). - final Collection passedKeys = skipped != null ? F.view(keys, F0.notIn(skipped)) : keys; + final Collection passedKeys = skipped != null ? F.view(keys, F.notIn(skipped)) : keys; if (log.isDebugEnabled()) log.debug("Lock keys: " + passedKeys); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java index 40399b4..47dafc8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java @@ -60,7 +60,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.dr.GridDrType; import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException; import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -1287,14 +1286,14 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture nearNodes = - cctx.discovery().nodes(readers, F0.not(F.idForNodeId(tx.nearNodeId()))); + cctx.discovery().nodes(readers, F.not(F.idForNodeId(tx.nearNodeId()))); if (log.isDebugEnabled()) log.debug("Mapping entry to near nodes [nodes=" + U.toShortString(nearNodes) + ", entry=" + entry + ']'); // Exclude DHT nodes. - map(entry, F.view(nearNodes, F0.notIn(dhtNodes)), nearMap); + map(entry, F.view(nearNodes, F.notIn(dhtNodes)), nearMap); } else if (log.isDebugEnabled()) log.debug("Entry has no near readers: " + entry); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java index 393413e..e0b108b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java @@ -77,12 +77,10 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSing import org.apache.ignite.internal.processors.cache.dr.GridCacheDrExpirationInfo; import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionConflictContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -412,9 +410,9 @@ public class GridDhtAtomicCache extends GridDhtCacheAdapter { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate... filter) { - A.notNull(key, "key"); + A.notNull(key, "key", val, "val"); - return updateAllAsync0(F0.asMap(key, val), + return updateAllAsync0(F.asMap(key, val), null, null, null, @@ -428,9 +426,9 @@ public class GridDhtAtomicCache extends GridDhtCacheAdapter { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture putAsync0(K key, V val, @Nullable CacheEntryPredicate... filter) { - A.notNull(key, "key"); + A.notNull(key, "key", val, "val"); - return updateAllAsync0(F0.asMap(key, val), + return updateAllAsync0(F.asMap(key, val), null, null, null, @@ -445,7 +443,7 @@ public class GridDhtAtomicCache extends GridDhtCacheAdapter { @Override public V tryPutIfAbsent(K key, V val) throws IgniteCheckedException { A.notNull(key, "key", val, "val"); - return (V)updateAllAsync0(F0.asMap(key, val), + return (V)updateAllAsync0(F.asMap(key, val), null, null, null, @@ -795,18 +793,13 @@ public class GridDhtAtomicCache extends GridDhtCacheAdapter { /** {@inheritDoc} */ @SuppressWarnings("unchecked") @Override public IgniteInternalFuture>> invokeAllAsync(Set keys, - final EntryProcessor entryProcessor, - Object... args) { + EntryProcessor entryProcessor, Object... args) { A.notNull(keys, "keys", entryProcessor, "entryProcessor"); if (keyCheck) validateCacheKeys(keys); - Map invokeMap = F.viewAsMap(keys, new C1() { - @Override public EntryProcessor apply(K k) { - return entryProcessor; - } - }); + Map> invokeMap = CU.invokeMap(keys, entryProcessor); CacheOperationContext opCtx = ctx.operationContextPerCall(); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java index 7970a44..6b03131 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java @@ -43,7 +43,6 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridLeanSet; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -521,8 +520,11 @@ public final class GridDhtForceKeysFuture extends GridCompoundFuture retryKeys = F.view( keys, - F0.notIn(missedKeys), - F0.notIn(F.viewReadOnly(res.forcedInfos(), CU.info2Key()))); + F.and( + F.notIn(missedKeys), + F.notIn(F.viewReadOnly(res.forcedInfos(), CU.info2Key())) + ) + ); if (!retryKeys.isEmpty()) map(retryKeys, F.concat(false, node, exc)); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java index 5bf18d9..5c48dc7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java @@ -64,6 +64,7 @@ import org.apache.ignite.internal.util.typedef.C1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.P1; import org.apache.ignite.internal.util.typedef.internal.A; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteBiPredicate; import org.jetbrains.annotations.NotNull; @@ -416,7 +417,7 @@ public abstract class GridNearCacheAdapter extends GridDistributedCacheAda /** {@inheritDoc} */ @Override public Collection values(CacheEntryPredicate... filter) { - return new GridCacheValueCollection<>(ctx, entrySet(filter), ctx.vararg(F.cacheHasPeekValue())); + return new GridCacheValueCollection<>(ctx, entrySet(filter), ctx.vararg(CU.cacheHasPeekValue())); } /** {@inheritDoc} */ @@ -558,12 +559,12 @@ public abstract class GridNearCacheAdapter extends GridDistributedCacheAda /** {@inheritDoc} */ @NotNull @Override public Iterator> iterator() { return new EntryIterator(nearSet.iterator(), - F.iterator0(dhtSet, false, new P1>() { - @Override public boolean apply(Cache.Entry e) { + F.identityIterator(dhtSet, new P1>() { + @Override + public boolean apply(Cache.Entry e) { try { return GridNearCacheAdapter.super.localPeek(e.getKey(), NEAR_PEEK_MODE, null) == null; - } - catch (IgniteCheckedException ex) { + } catch (IgniteCheckedException ex) { throw new IgniteException(ex); } } http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java index 6130ead..3b60f96 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java @@ -60,7 +60,6 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.local.GridLocalCacheEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.GridUnsafe; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -189,7 +188,7 @@ public class GridLocalAtomicCache extends GridCacheAdapter { @Override public IgniteInternalFuture getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate... filter) { A.notNull(key, "key", val, "val"); - return updateAllAsync0(F0.asMap(key, val), + return updateAllAsync0(F.asMap(key, val), null, null, true, @@ -202,7 +201,7 @@ public class GridLocalAtomicCache extends GridCacheAdapter { @Override public IgniteInternalFuture putAsync0(K key, V val, @Nullable CacheEntryPredicate... filter) { A.notNull(key, "key", val, "val"); - return updateAllAsync0(F0.asMap(key, val), + return updateAllAsync0(F.asMap(key, val), null, null, false, @@ -659,11 +658,7 @@ public class GridLocalAtomicCache extends GridCacheAdapter { if (keyCheck) validateCacheKeys(keys); - Map invokeMap = F.viewAsMap(keys, new C1() { - @Override public EntryProcessor apply(K k) { - return entryProcessor; - } - }); + Map> invokeMap = CU.invokeMap(keys, entryProcessor); CacheOperationContext opCtx = ctx.operationContextPerCall(); @@ -729,11 +724,7 @@ public class GridLocalAtomicCache extends GridCacheAdapter { if (keyCheck) validateCacheKeys(keys); - Map invokeMap = F.viewAsMap(keys, new C1() { - @Override public EntryProcessor apply(K k) { - return entryProcessor; - } - }); + Map> invokeMap = CU.invokeMap(keys, entryProcessor); return updateAllAsync0(null, invokeMap, http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java index f7a82a9..84c4269 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java @@ -30,7 +30,6 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.IgniteInternalCache; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; -import org.apache.ignite.internal.util.F0; import org.apache.ignite.internal.util.tostring.GridToStringBuilder; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -179,8 +178,8 @@ public final class GridCacheAtomicStampedImpl implements GridCacheAtomicSt checkRemoved(); try { - return CU.outTx(internalCompareAndSet(F0.equalTo(expVal), wrapperClosure(newVal), - F0.equalTo(expStamp), wrapperClosure(newStamp)), ctx); + return CU.outTx(internalCompareAndSet(F.equalTo(expVal), wrapperClosure(newVal), + F.equalTo(expStamp), wrapperClosure(newStamp)), ctx); } catch (IgniteCheckedException e) { throw U.convertException(e); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java index 5b8cf86..b313084 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java @@ -45,7 +45,6 @@ import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.ipc.IpcServerEndpoint; import org.apache.ignite.internal.util.typedef.C1; -import org.apache.ignite.internal.util.typedef.CI1; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.U; @@ -128,11 +127,8 @@ public class IgfsProcessor extends IgfsProcessorAdapter { final Map cacheCfgs = new HashMap<>(); - F.forEach(gridCfg.getCacheConfiguration(), new CI1() { - @Override public void apply(CacheConfiguration c) { - cacheCfgs.put(c.getName(), c); - } - }); + for (CacheConfiguration c : gridCfg.getCacheConfiguration()) + cacheCfgs.put(c.getName(), c); Collection attrVals = new ArrayList<>(); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java index 7e65c22..9f0fd3f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java @@ -235,13 +235,13 @@ public class PlatformDotNetCacheStore implements CacheStore, Platfor @Override public Set> entrySet() { return new AbstractSet>() { @Override public Iterator> iterator() { - return F.iterator(entries, new C1, Entry>() { + return F.iteratorReadOnly(entries, new C1, Entry>() { private static final long serialVersionUID = 0L; @Override public Entry apply(Cache.Entry entry) { return new GridMapEntry<>(entry.getKey(), entry.getValue()); } - }, true); + }); } @Override public int size() { http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java index 297785e..5e12199 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java @@ -135,8 +135,7 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter { } else node = F.find(ctx.discovery().allNodes(), null, new P1() { - @Override - public boolean apply(ClusterNode n) { + @Override public boolean apply(ClusterNode n) { return containsIp(n.addresses(), ip); } }); http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/util/F0.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/F0.java b/modules/core/src/main/java/org/apache/ignite/internal/util/F0.java index 7e9e448..b819226 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/F0.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/F0.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.util; -import java.util.Collection; -import java.util.Map; import java.util.Set; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; @@ -27,11 +25,9 @@ import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter; import org.apache.ignite.internal.processors.cache.CacheEntrySerializablePredicate; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheEntryEx; -import org.apache.ignite.internal.util.lang.GridFunc; import org.apache.ignite.internal.util.lang.GridNodePredicate; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.P1; -import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; @@ -42,79 +38,6 @@ import org.jetbrains.annotations.Nullable; */ public class F0 { /** - * Negates given predicates. - *

- * Gets predicate (not peer-deployable) that evaluates to {@code true} if any of given predicates - * evaluates to {@code false}. If all predicates evaluate to {@code true} the - * result predicate will evaluate to {@code false}. - * - * @param p Predicate to negate. - * @param Type of the free variable, i.e. the element the predicate is called on. - * @return Negated predicate (not peer-deployable). - */ - public static IgnitePredicate not(@Nullable final IgnitePredicate... p) { - return F.isAlwaysFalse(p) ? F.alwaysTrue() : F.isAlwaysTrue(p) ? F.alwaysFalse() : new P1() { - @Override public boolean apply(T t) { - return !F.isAll(t, p); - } - }; - } - - /** - * Gets predicate (not peer-deployable) that evaluates to {@code true} if its free variable is not equal - * to {@code target} or both are {@code null}. - * - * @param target Object to compare free variable to. - * @param Type of the free variable, i.e. the element the predicate is called on. - * @return Predicate (not peer-deployable) that evaluates to {@code true} if its free variable is not equal - * to {@code target} or both are {@code null}. - */ - public static IgnitePredicate notEqualTo(@Nullable final T target) { - return new P1() { - @Override public boolean apply(T t) { - return !F.eq(t, target); - } - }; - } - - /** - * Gets predicate (not peer-deployable) that returns {@code true} if its free variable - * is not contained in given collection. - * - * @param c Collection to check for containment. - * @param Type of the free variable for the predicate and type of the - * collection elements. - * @return Predicate (not peer-deployable) that returns {@code true} if its free variable is not - * contained in given collection. - */ - public static IgnitePredicate notIn(@Nullable final Collection c) { - return F.isEmpty(c) ? GridFunc.alwaysTrue() : new P1() { - @Override public boolean apply(T t) { - assert c != null; - - return !c.contains(t); - } - }; - } - - /** - * Gets predicate (not perr-deployable) that evaluates to {@code true} if its free variable is equal - * to {@code target} or both are {@code null}. - * - * @param target Object to compare free variable to. - * @param Type of the free variable, i.e. the element the predicate is called on. - * @return Predicate that evaluates to {@code true} if its free variable is equal to - * {@code target} or both are {@code null}. - */ - public static IgnitePredicate equalTo(@Nullable final T target) { - return new P1() { - @Override public boolean apply(T t) { - return F.eq(t, target); - } - }; - } - - /** * @param p1 Filter1. * @param p2 Filter2. * @return And filter. @@ -133,16 +56,12 @@ public class F0 { if (e1 && e2) return CU.alwaysTrue0(); - if (e1 && !e2) { - assert p2 != null; - + if (e1) { if (p2.length == 1) return p2[0]; } - if (!e1 && e2) { - assert p1 != null; - + if (e2) { if (p1.length == 1) return p1[0]; } @@ -150,16 +69,12 @@ public class F0 { return new CacheEntrySerializablePredicate(new CacheEntryPredicateAdapter() { @Override public boolean apply(GridCacheEntryEx e) { if (!e1) { - assert p1 != null; - for (CacheEntryPredicate p : p1) if (p != null && !p.apply(e)) return false; } if (!e2) { - assert p2 != null; - for (CacheEntryPredicate p : p2) if (p != null && !p.apply(e)) return false; @@ -186,15 +101,11 @@ public class F0 { @Override public void prepareMarshal(GridCacheContext ctx) throws IgniteCheckedException { if (!e1) { - assert p1 != null; - for (CacheEntryPredicate p : p1) p.prepareMarshal(ctx); } if (!e2) { - assert p2 != null; - for (CacheEntryPredicate p : p2) p.prepareMarshal(ctx); } @@ -203,63 +114,6 @@ public class F0 { } /** - * @param p Filter1. - * @param ps Filter2. - * @return And filter. - */ - public static CacheEntryPredicate and0( - @Nullable final CacheEntryPredicate p, - @Nullable final CacheEntryPredicate... ps) { - if (p == null && F.isEmptyOrNulls(ps)) - return CU.alwaysTrue0(); - - if (F.isAlwaysFalse(p) && F.isAlwaysFalse(ps)) - return CU.alwaysFalse0(); - - if (F.isAlwaysTrue(p) && F.isAlwaysTrue(ps)) - return CU.alwaysTrue0(); - - return new CacheEntrySerializablePredicate(new CacheEntryPredicateAdapter() { - @Override public boolean apply(GridCacheEntryEx e) { - assert ps != null; - - if (p != null && !p.apply(e)) - return false; - - for (CacheEntryPredicate p : ps) { - if (p != null && !p.apply(e)) - return false; - } - - return true; - } - - @Override public void entryLocked(boolean locked) { - assert ps != null; - - if (p != null) - p.entryLocked(locked); - - for (CacheEntryPredicate p : ps) { - if (p != null) - p.entryLocked(locked); - } - } - - @Override public void prepareMarshal(GridCacheContext ctx) throws IgniteCheckedException { - assert ps != null; - - if (p != null) - p.prepareMarshal(ctx); - - for (CacheEntryPredicate p : ps) - if (p != null) - p.prepareMarshal(ctx); - } - }); - } - - /** * Get a predicate (non peer-deployable) that evaluates to {@code true} if each of its component predicates * evaluates to {@code true}. The components are evaluated in order they are supplied. * Evaluation will be stopped as soon as first predicate evaluates to {@code false}. @@ -287,16 +141,12 @@ public class F0 { if (e1 && e2) return F.alwaysTrue(); - if (e1 && !e2) { - assert p2 != null; - + if (e1) { if (p2.length == 1) return (IgnitePredicate)p2[0]; } - if (!e1 && e2) { - assert p1 != null; - + if (e2) { if (p1.length == 1) return (IgnitePredicate)p1[0]; } @@ -305,15 +155,11 @@ public class F0 { Set ids = new GridLeanSet<>(); if (!e1) { - assert p1 != null; - for (IgnitePredicate p : p1) ids.addAll(((GridNodePredicate)p).nodeIds()); } if (!e2) { - assert p2 != null; - for (IgnitePredicate p : p2) ids.addAll(((GridNodePredicate)p).nodeIds()); } @@ -325,16 +171,12 @@ public class F0 { return new P1() { @Override public boolean apply(T t) { if (!e1) { - assert p1 != null; - for (IgnitePredicate p : p1) if (p != null && !p.apply(t)) return false; } if (!e2) { - assert p2 != null; - for (IgnitePredicate p : p2) if (p != null && !p.apply(t)) return false; @@ -347,148 +189,6 @@ public class F0 { } /** - * Get a predicate (not peer-deployable) that evaluates to {@code true} if each of its component predicates - * evaluates to {@code true}. The components are evaluated in order they are supplied. - * Evaluation will be stopped as soon as first predicate evaluates to {@code false}. - * Passed in predicates are NOT copied. If no predicates are passed in the returned - * predicate will always evaluate to {@code false}. - * - * @param ps Passed in predicate. If none provided - always-{@code false} predicate is - * returned. - * @param Type of the free variable, i.e. the element the predicate is called on. - * @return Predicate that evaluates to {@code true} if each of its component predicates - * evaluates to {@code true}. - */ - @SuppressWarnings("unchecked") - public static IgnitePredicate and( - @Nullable final IgnitePredicate p, - @Nullable final IgnitePredicate... ps - ) { - if (p == null && F.isEmptyOrNulls(ps)) - return F.alwaysTrue(); - - if (F.isAlwaysFalse(p) && F.isAlwaysFalse(ps)) - return F.alwaysFalse(); - - if (F.isAlwaysTrue(p) && F.isAlwaysTrue(ps)) - return F.alwaysTrue(); - - if (isAllNodePredicates(p) && isAllNodePredicates(ps)) { - assert ps != null; - - Set ids = new GridLeanSet<>(); - - for (IgnitePredicate p0 : ps) { - Collection list = ((GridNodePredicate)p0).nodeIds(); - - if (ids.isEmpty()) - ids.addAll(list); - else - ids.retainAll(list); - } - - Collection list = ((GridNodePredicate)p).nodeIds(); - - if (ids.isEmpty()) - ids.addAll(list); - else - ids.retainAll(list); - - // T must be . - return (IgnitePredicate)new GridNodePredicate(ids); - } - else { - return new P1() { - @Override public boolean apply(T t) { - assert ps != null; - - if (p != null && !p.apply(t)) - return false; - - for (IgnitePredicate p : ps) - if (p != null && !p.apply(t)) - return false; - - return true; - } - }; - } - } - - /** - * Gets predicate (not peer-deployable) that returns {@code true} if its free variable is contained - * in given collection. - * - * @param c Collection to check for containment. - * @param Type of the free variable for the predicate and type of the - * collection elements. - * @return Predicate (not peer-deployable) that returns {@code true} if its free variable is - * contained in given collection. - */ - public static IgnitePredicate in(@Nullable final Collection c) { - return F.isEmpty(c) ? GridFunc.alwaysFalse() : new P1() { - @Override public boolean apply(T t) { - assert c != null; - - return c.contains(t); - } - }; - } - - /** - * Provides predicate (not peer-deployable) which returns {@code true} if it receives an element - * that is contained in the passed in collection. - * - * @param c Collection used for predicate filter. - * @param Element type. - * @return Predicate which returns {@code true} if it receives an element - * that is contained in the passed in collection. - */ - public static IgnitePredicate contains(@Nullable final Collection c) { - return c == null || c.isEmpty() ? GridFunc.alwaysFalse() : new P1() { - @Override public boolean apply(T t) { - return c.contains(t); - } - }; - } - - /** - * Provides predicate (not peer-deployable) which returns {@code true} if it receives an element - * that is not contained in the passed in collection. - * - * @param c Collection used for predicate filter. - * @param Element type. - * @return Predicate which returns {@code true} if it receives an element - * that is not contained in the passed in collection. - */ - public static IgnitePredicate notContains(@Nullable final Collection c) { - return c == null || c.isEmpty() ? GridFunc.alwaysTrue() : new P1() { - @Override public boolean apply(T t) { - return !c.contains(t); - } - }; - } - - /** - * Tests if all passed in predicates are instances of {@link GridNodePredicate} class. - * - * @param ps Collection of predicates to test. - * @return {@code True} if all passed in predicates are instances of {@link GridNodePredicate} class. - */ - public static boolean isAllNodePredicates(@Nullable Iterable> ps) { - if (F.isEmpty(ps)) - return false; - - assert ps != null; - - for (IgnitePredicate p : ps) - if (!(p instanceof GridNodePredicate)) - return false; - - return true; - } - - /** * Tests if all passed in predicates are instances of {@link GridNodePredicate} class. * * @param ps Collection of predicates to test. @@ -498,27 +198,10 @@ public class F0 { if (F.isEmpty(ps)) return false; - assert ps != null; - for (IgnitePredicate p : ps) if (!(p instanceof GridNodePredicate)) return false; return true; } - - /** - * Creates map with given values, adding a strict not-null check for value. - * - * @param key Key. - * @param val Value. - * @param Key's type. - * @param Value's type. - * @return Created map. - */ - public static Map asMap(K key, V val) { - A.notNull(val, "val"); - - return F.asMap(key, val); - } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/modules/core/src/main/java/org/apache/ignite/internal/util/GridExecutionStatistics.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridExecutionStatistics.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridExecutionStatistics.java index a23fe28..fc2e3a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridExecutionStatistics.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridExecutionStatistics.java @@ -71,13 +71,13 @@ public class GridExecutionStatistics { long time = startTime.get().get2(); - AtomicInteger cnt = F.addIfAbsent(cntMap, name, F.newAtomicInt()); + AtomicInteger cnt = F.addIfAbsent(cntMap, name, new AtomicInteger()); assert cnt != null; cnt.incrementAndGet(); - AtomicLong d = F.addIfAbsent(durationMap, name, F.newAtomicLong()); + AtomicLong d = F.addIfAbsent(durationMap, name, new AtomicLong()); assert d != null; http://git-wip-us.apache.org/repos/asf/ignite/blob/80579253/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 480859d..a92ccab 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 @@ -8508,7 +8508,7 @@ public abstract class IgniteUtils { throw new IgniteCheckedException("Addresses can not be resolved [addr=" + addrs + ", hostNames=" + hostNames + ']'); - return F.viewListReadOnly(res, F.identity()); + return Collections.unmodifiableList(res); } /** @@ -8555,7 +8555,7 @@ public abstract class IgniteUtils { res.add(new InetSocketAddress(addr, port)); } - return F.viewListReadOnly(res, F.identity()); + return Collections.unmodifiableList(res); } /**