You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/12/30 15:32:37 UTC

[1/5] ignite git commit: IGNITE-2263: DANGEROUS COMMIT! Simplified iterators and views.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2263 2f1fbb955 -> c4cae9d3c


IGNITE-2263: DANGEROUS COMMIT! Simplified iterators and views.


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

Branch: refs/heads/ignite-2263
Commit: 64f6e32826e4715423174bd9de53433cad46e16f
Parents: 2f1fbb9
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Dec 30 17:23:08 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 30 17:23:08 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  15 +-
 .../processors/cache/GridCacheIterator.java     |   4 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   7 +-
 .../ignite/internal/util/lang/GridFunc.java     | 256 +++++++++++--------
 .../java/org/apache/ignite/lang/IgniteUuid.java |   2 +-
 .../ignite/lang/GridFuncPerformanceTest.java    | 102 --------
 6 files changed, 178 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/64f6e328/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..befb9ab 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
@@ -1317,13 +1317,26 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
+     * Gets collection of node for given node IDs.
+     *
+     * @param ids Ids to include.
+     * @return Collection with all alive nodes for given IDs.
+     */
+    public Collection<ClusterNode> nodes(@Nullable Collection<UUID> ids) {
+        return F.isEmpty(ids) ? Collections.<ClusterNode>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<ClusterNode> nodes(@Nullable Collection<UUID> ids, IgnitePredicate<UUID>... p) {
+    public Collection<ClusterNode> nodes(@Nullable Collection<UUID> ids, IgnitePredicate<UUID> p) {
         return F.isEmpty(ids) ? Collections.<ClusterNode>emptyList() :
             F.view(
                 F.viewReadOnly(ids, U.id2Node(ctx), p),

http://git-wip-us.apache.org/repos/asf/ignite/blob/64f6e328/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 4644e59..48f5fe2 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<K, V, T> implements GridSerializableIterator<T> {
     ) {
         this.cctx = cctx;
 
-        it = F.identityIterator(c, false, filter);
+        it = F.identityIterator(c, false, F0.and(filter));
 
         this.trans = trans;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/64f6e328/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..89fd29f 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
@@ -521,8 +521,11 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
             if (!cctx.rebalanceEnabled()) {
                 Collection<KeyCacheObject> retryKeys = F.view(
                     keys,
-                    F0.notIn(missedKeys),
-                    F0.notIn(F.viewReadOnly(res.forcedInfos(), CU.<KeyCacheObject, V>info2Key())));
+                    F.and(
+                        F0.notIn(missedKeys),
+                        F0.notIn(F.viewReadOnly(res.forcedInfos(), CU.<KeyCacheObject, V>info2Key()))
+                    )
+                );
 
                 if (!retryKeys.isEmpty())
                     map(retryKeys, F.concat(false, node, exc));

http://git-wip-us.apache.org/repos/asf/ignite/blob/64f6e328/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index e1d7b82..1bf7c22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -1285,6 +1285,19 @@ public class GridFunc {
      * @return Number of elements in the collection for which all given predicates
      *      evaluates to {@code true}. If no predicates is provided - all elements are counted.
      */
+    public static <T> int size(@Nullable Collection<? extends T> c, IgnitePredicate<? super T> p) {
+        return c == null || c.isEmpty() ? 0 : isAlwaysTrue(p) ? c.size() : size(c.iterator(), p);
+    }
+
+    /**
+     * Gets size of the given collection with provided optional predicates.
+     *
+     * @param c Collection to size.
+     * @param p Optional predicates that filters out elements from count.
+     * @param <T> Type of the iterator.
+     * @return Number of elements in the collection for which all given predicates
+     *      evaluates to {@code true}. If no predicates is provided - all elements are counted.
+     */
     public static <T> int size(@Nullable Collection<? extends T> c, @Nullable IgnitePredicate<? super T>[] p) {
         return c == null || c.isEmpty() ? 0 : isEmpty(p) || isAlwaysTrue(p) ? c.size() : size(c.iterator(), p);
     }
@@ -1312,6 +1325,32 @@ public class GridFunc {
     }
 
     /**
+     * Gets size of the given iterator with provided optional predicate. Iterator
+     * will be traversed to get the count.
+     *
+     * @param it Iterator to size.
+     * @param p Optional predicates that filters out elements from count.
+     * @param <T> Type of the iterator.
+     * @return Number of elements in the iterator for which all given predicates
+     *      evaluates to {@code true}. If no predicates is provided - all elements are counted.
+     */
+    public static <T> int size(@Nullable Iterator<? extends T> it, IgnitePredicate<? super T> p) {
+        if (it == null)
+            return 0;
+
+        int n = 0;
+
+        if (!isAlwaysFalse(p)) {
+            while (it.hasNext()) {
+                if (p.apply(it.next()))
+                    n++;
+            }
+        }
+
+        return n;
+    }
+
+    /**
      * Gets size of the given iterator with provided optional predicates. Iterator
      * will be traversed to get the count.
      *
@@ -1338,24 +1377,18 @@ public class GridFunc {
     }
 
     /**
-     * Creates write-through light-weight view on given collection with provided predicates. Resulting
-     * collection will only "have" elements for which all provided predicates, if any, evaluate
-     * to {@code true}. Note that only wrapping collection will be created and no duplication of
-     * data will occur. Also note that if array of given predicates is not empty then method
-     * {@code size()} uses full iteration through the collection.
+     * Creates write-through light-weight view on given collection.
      *
      * @param c Input collection that serves as a base for the view.
      * @param p Optional predicates. If predicates are not provided - all elements will be in the view.
      * @param <T> Type of the collection.
      * @return Light-weight view on given collection with provided predicate.
      */
-    @SafeVarargs
-    public static <T> Collection<T> view(@Nullable final Collection<T> c,
-        @Nullable final IgnitePredicate<? super T>... p) {
+    public static <T> Collection<T> view(@Nullable final Collection<T> c, final IgnitePredicate<? super T> p) {
         if (isEmpty(c) || isAlwaysFalse(p))
             return Collections.emptyList();
 
-        return isEmpty(p) || isAlwaysTrue(p) ? c : new GridSerializableCollection<T>() {
+        return isAlwaysTrue(p) ? c : new GridSerializableCollection<T>() {
             // Pass through (will fail for readonly).
             @Override public boolean add(T e) {
                 return isAll(e, p) && c.add(e);
@@ -1371,7 +1404,39 @@ public class GridFunc {
             }
 
             @Override public boolean isEmpty() {
-                return F.isEmpty(p) ? c.isEmpty() : !iterator().hasNext();
+                return !iterator().hasNext();
+            }
+        };
+    }
+
+    /**
+     * Creates read-only light-weight view on given collection with transformation.
+     *
+     * @param c Input collection that serves as a base for the view.
+     * @param trans Transformation closure.
+     * @param <T1> Type of the collection.
+     * @return Light-weight view on given collection with provided predicate.
+     */
+    @SuppressWarnings("RedundantTypeArguments")
+    public static <T1, T2> Collection<T2> viewReadOnly(@Nullable final Collection<? extends T1> c,
+        final IgniteClosure<? super T1, T2> trans) {
+        A.notNull(trans, "trans");
+
+        if (isEmpty(c))
+            return Collections.emptyList();
+
+        return new GridSerializableCollection<T2>() {
+            @NotNull
+            @Override public Iterator<T2> iterator() {
+                return F.<T1, T2>iterator(c, trans, true);
+            }
+
+            @Override public int size() {
+                return c.size();
+            }
+
+            @Override public boolean isEmpty() {
+                return c.isEmpty();
             }
         };
     }
@@ -1391,9 +1456,8 @@ public class GridFunc {
      * @return Light-weight view on given collection with provided predicate.
      */
     @SuppressWarnings("RedundantTypeArguments")
-    @SafeVarargs
     public static <T1, T2> Collection<T2> viewReadOnly(@Nullable final Collection<? extends T1> c,
-        final IgniteClosure<? super T1, T2> trans, @Nullable final IgnitePredicate<? super T1>... p) {
+        final IgniteClosure<? super T1, T2> trans, @Nullable final IgnitePredicate<? super T1> p) {
         A.notNull(trans, "trans");
 
         if (isEmpty(c) || isAlwaysFalse(p))
@@ -1406,11 +1470,11 @@ public class GridFunc {
             }
 
             @Override public int size() {
-                return F.isEmpty(p) ? c.size() : F.size(iterator());
+                return F.size(iterator());
             }
 
             @Override public boolean isEmpty() {
-                return F.isEmpty(p) ? c.isEmpty() : !iterator().hasNext();
+                return !iterator().hasNext();
             }
         };
     }
@@ -1423,17 +1487,17 @@ public class GridFunc {
      * uses full iteration through the entry set.
      *
      * @param m Input map that serves as a base for the view.
-     * @param p Optional predicates. If predicates are not provided - all will be in the view.
+     * @param p Predicate.
      * @param <K> Type of the key.
      * @param <V> Type of the value.
      * @return Light-weight view on given map with provided predicate.
      */
     public static <K0, K extends K0, V0, V extends V0> Map<K, V> view(@Nullable final Map<K, V> m,
-        @Nullable final IgnitePredicate<? super K>... p) {
+        final IgnitePredicate<? super K> p) {
         if (isEmpty(m) || isAlwaysFalse(p))
             return Collections.emptyMap();
 
-        return isEmpty(p) || isAlwaysTrue(p) ? m : new GridSerializableMap<K, V>() {
+        return isAlwaysTrue(p) ? m : new GridSerializableMap<K, V>() {
             /** */
             private static final long serialVersionUID = 5531745605372387948L;
 
@@ -1506,36 +1570,26 @@ public class GridFunc {
      *
      * @param m Input map that serves as a base for the view.
      * @param trans Transformer for map value transformation.
-     * @param p Optional predicates. If predicates are not provided - all will be in the view.
      * @param <K> Type of the key.
      * @param <V> Type of the input map value.
      * @param <V1> Type of the output map value.
      * @return Light-weight view on given map with provided predicate and transformer.
      */
     public static <K0, K extends K0, V0, V extends V0, V1> Map<K, V1> viewReadOnly(@Nullable final Map<K, V> m,
-        final IgniteClosure<V, V1> trans, @Nullable final IgnitePredicate<? super K>... p) {
+        final IgniteClosure<V, V1> trans) {
         A.notNull(trans, "trans");
 
-        if (isEmpty(m) || isAlwaysFalse(p))
+        if (isEmpty(m))
             return Collections.emptyMap();
 
-        final boolean hasPred = p != null && p.length > 0;
-
         return new GridSerializableMap<K, V1>() {
-            /** Entry predicate. */
-            private IgnitePredicate<Entry<K, V>> ep = new P1<Map.Entry<K, V>>() {
-                @Override public boolean apply(Entry<K, V> e) {
-                    return isAll(e.getKey(), p);
-                }
-            };
-
             @NotNull
             @Override public Set<Entry<K, V1>> entrySet() {
                 return new GridSerializableSet<Map.Entry<K, V1>>() {
                     @NotNull
                     @Override public Iterator<Entry<K, V1>> iterator() {
                         return new Iterator<Entry<K, V1>>() {
-                            private Iterator<Entry<K, V>> it = identityIterator(m.entrySet(), true, ep);
+                            private Iterator<Entry<K, V>> it = identityIterator(m.entrySet(), true);
 
                             @Override public boolean hasNext() {
                                 return it.hasNext();
@@ -1566,7 +1620,7 @@ public class GridFunc {
                     }
 
                     @Override public int size() {
-                        return hasPred ? F.size(m.keySet(), p) : m.size();
+                        return m.size();
                     }
 
                     @SuppressWarnings({"unchecked"})
@@ -1576,27 +1630,25 @@ public class GridFunc {
 
                     @SuppressWarnings({"unchecked"})
                     @Override public boolean contains(Object o) {
-                        return F.isAll((Map.Entry<K, V>)o, ep) && m.entrySet().contains(o);
+                        return m.entrySet().contains(o);
                     }
 
                     @Override public boolean isEmpty() {
-                        return hasPred ? !iterator().hasNext() : m.isEmpty();
+                        return m.isEmpty();
                     }
                 };
             }
 
             @Override public boolean isEmpty() {
-                return hasPred ? entrySet().isEmpty() : m.isEmpty();
+                return m.isEmpty();
             }
 
             @SuppressWarnings({"unchecked"})
             @Nullable @Override public V1 get(Object key) {
-                if (isAll((K) key, p)) {
-                    V v = m.get(key);
+                V v = m.get(key);
 
-                    if (v != null)
-                        return trans.apply(v);
-                }
+                if (v != null)
+                    return trans.apply(v);
 
                 return null;
             }
@@ -1611,48 +1663,34 @@ public class GridFunc {
 
             @SuppressWarnings({"unchecked"})
             @Override public boolean containsKey(Object key) {
-                return isAll((K)key, p) && m.containsKey(key);
+                return m.containsKey(key);
             }
         };
     }
 
     /**
-     * Read-only map view of a collection. Resulting map is a lightweight view of an input collection,
-     * with filtered elements of an input collection as keys, and closure execution results
-     * as values. The map will only contain keys for which all provided predicates, if any, evaluate
-     * to {@code true}. Note that only wrapping map will be created and no duplication of data will occur.
-     * Also note that if array of given predicates is not empty then method {@code size()}
-     * uses full iteration through the entry set.
+     * Read-only map view of a collection.
      *
      * @param c Input collection.
      * @param mapClo Mapping closure, that maps key to value.
-     * @param p Optional predicates to filter input collection. If predicates are not provided - all
-     *          elements will be in the view.
      * @param <K> Key type.
      * @param <V> Value type.
      * @return Light-weight view on given map with provided predicates and mapping.
      */
     @SuppressWarnings("TypeMayBeWeakened")
     public static <K0, K extends K0, V0, V extends V0> Map<K, V> viewAsMap(@Nullable final Set<K> c,
-        final IgniteClosure<? super K, V> mapClo, @Nullable final IgnitePredicate<? super K>... p) {
+        final IgniteClosure<? super K, V> mapClo) {
         A.notNull(mapClo, "trans");
 
-        if (isEmpty(c) || isAlwaysFalse(p))
+        if (isEmpty(c))
             return Collections.emptyMap();
 
         return new GridSerializableMap<K, V>() {
-            /** Entry predicate. */
-            private IgnitePredicate<K> ep = new P1<K>() {
-                @Override public boolean apply(K e) {
-                    return isAll(e, p);
-                }
-            };
-
             @NotNull @Override public Set<Entry<K, V>> entrySet() {
                 return new GridSerializableSet<Entry<K, V>>() {
                     @NotNull @Override public Iterator<Entry<K, V>> iterator() {
                         return new Iterator<Entry<K, V>>() {
-                            private Iterator<K> it = identityIterator(c, true, ep);
+                            private Iterator<K> it = identityIterator(c, true);
 
                             @Override public boolean hasNext() {
                                 return it.hasNext();
@@ -1685,7 +1723,7 @@ public class GridFunc {
                     }
 
                     @Override public int size() {
-                        return F.size(c, p);
+                        return F.size(c);
                     }
 
                     @Override public boolean remove(Object o) {
@@ -1702,6 +1740,7 @@ public class GridFunc {
                 return entrySet().isEmpty();
             }
 
+            @SuppressWarnings("unchecked")
             @Nullable @Override public V get(Object key) {
                 if (containsKey(key))
                     return mapClo.apply((K)key);
@@ -1717,8 +1756,9 @@ public class GridFunc {
                 throw new UnsupportedOperationException("Remove is not supported for readonly collection view.");
             }
 
+            @SuppressWarnings("SuspiciousMethodCalls")
             @Override public boolean containsKey(Object key) {
-                return isAll((K)key, p) && c.contains(key);
+                return c.contains(key);
             }
         };
     }
@@ -1873,6 +1913,20 @@ public class GridFunc {
     }
 
     /**
+     * Creates and returns iterator from given collection.
+     *
+     * @param c Input collection.
+     * @param readOnly If {@code true}, then resulting iterator will not allow modifications
+     *      to the underlying collection.
+     * @param <T> Type of the collection elements.
+     * @return Iterator from given collection and optional filtering predicate.
+     */
+    @SuppressWarnings({"unchecked"})
+    public static <T> GridIterator<T> identityIterator(Iterable<? extends T> c, boolean readOnly) {
+        return F.iterator(c, IDENTITY, readOnly);
+    }
+
+    /**
      * Creates and returns iterator from given collection and optional filtering predicates.
      * Returned iterator will only have elements for which all given predicates evaluates to
      * {@code true} (if provided). Note that this method will not create new collection but
@@ -1882,17 +1936,35 @@ public class GridFunc {
      * @param c Input collection.
      * @param readOnly If {@code true}, then resulting iterator will not allow modifications
      *      to the underlying collection.
-     * @param p Optional filtering predicates.
+     * @param p Optional filtering predicate.
      * @param <T> Type of the collection elements.
      * @return Iterator from given collection and optional filtering predicate.
      */
     @SuppressWarnings({"unchecked"})
     public static <T> GridIterator<T> identityIterator(Iterable<? extends T> c, boolean readOnly,
-        IgnitePredicate<? super T>... p) {
+        IgnitePredicate<? super T> p) {
         return F.iterator(c, IDENTITY, readOnly, p);
     }
 
     /**
+     * Creates and returns transforming iterator from given collection.
+     *
+     * @param c Input collection.
+     * @param trans Transforming closure to convert from T1 to T2.
+     * @param readOnly If {@code true}, then resulting iterator will not allow modifications
+     *      to the underlying collection.
+     * @param <T1> Type of the collection elements.
+     * @param <T2> Type of returned elements.
+     * @return Iterator from given collection and optional filtering predicate.
+     */
+    public static <T1, T2> GridIterator<T2> iterator(final Iterable<? extends T1> c,
+        final IgniteClosure<? super T1, T2> trans, final boolean readOnly) {
+        A.notNull(c, "c", trans, "trans");
+
+        return  iterator(c.iterator(), trans, readOnly);
+    }
+
+    /**
      * Creates and returns transforming iterator from given collection and optional
      * filtering predicates. Returned iterator will only have elements for which all
      * given predicates evaluates to {@code true} ( if provided). Note that this method
@@ -1903,15 +1975,14 @@ public class GridFunc {
      * @param trans Transforming closure to convert from T1 to T2.
      * @param readOnly If {@code true}, then resulting iterator will not allow modifications
      *      to the underlying collection.
-     * @param p Optional filtering predicates.
+     * @param p Filtering predicate.
      * @param <T1> Type of the collection elements.
      * @param <T2> Type of returned elements.
      * @return Iterator from given collection and optional filtering predicate.
      */
     public static <T1, T2> GridIterator<T2> iterator(final Iterable<? extends T1> c,
-        final IgniteClosure<? super T1, T2> trans, final boolean readOnly,
-        @Nullable final IgnitePredicate<? super T1>... p) {
-        A.notNull(c, "c", trans, "trans");
+        final IgniteClosure<? super T1, T2> trans, final boolean readOnly, final IgnitePredicate<? super T1> p) {
+        A.notNull(c, "c", trans, "trans", p, "p");
 
         if (isAlwaysFalse(p))
             return F.emptyIterator();
@@ -1930,53 +2001,36 @@ public class GridFunc {
             private Iterator<? extends T1> iter = c.iterator();
 
             @Override public boolean hasNextX() {
-                if (isEmpty(p))
-                    return iter.hasNext();
+                if (!moved)
+                    return more;
                 else {
-                    if (!moved)
-                        return more;
-                    else {
-                        more = false;
-
-                        while (iter.hasNext()) {
-                            elem = iter.next();
-
-                            boolean isAll = true;
+                    more = false;
 
-                            for (IgnitePredicate<? super T1> r : p)
-                                if (r != null && !r.apply(elem)) {
-                                    isAll = false;
+                    while (iter.hasNext()) {
+                        elem = iter.next();
 
-                                    break;
-                                }
+                        if (p.apply(elem)) {
+                            more = true;
+                            moved = false;
 
-                            if (isAll) {
-                                more = true;
-                                moved = false;
-
-                                return true;
-                            }
+                            return true;
                         }
+                    }
 
-                        elem = null; // Give to GC.
+                    elem = null; // Give to GC.
 
-                        return false;
-                    }
+                    return false;
                 }
             }
 
             @Nullable @Override public T2 nextX() {
-                if (isEmpty(p))
-                    return trans.apply(iter.next());
-                else {
-                    if (hasNext()) {
-                        moved = true;
+                if (hasNext()) {
+                    moved = true;
 
-                        return trans.apply(elem);
-                    }
-                    else
-                        throw new NoSuchElementException();
+                    return trans.apply(elem);
                 }
+                else
+                    throw new NoSuchElementException();
             }
 
             @Override public void removeX() {
@@ -1995,7 +2049,7 @@ public class GridFunc {
      *      to the underlying collection.
      * @return Iterator from given iterator and optional filtering predicate.
      */
-    public static <T1, T2> Iterator<T2> iterator(final Iterator<? extends T1> c,
+    public static <T1, T2> GridIterator<T2> iterator(final Iterator<? extends T1> c,
         final IgniteClosure<? super T1, T2> trans, final boolean readOnly) {
         A.notNull(c, "c", trans, "trans");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/64f6e328/modules/core/src/main/java/org/apache/ignite/lang/IgniteUuid.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteUuid.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteUuid.java
index 5c6bb9a..7904671 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteUuid.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteUuid.java
@@ -186,7 +186,7 @@ public final class IgniteUuid implements Comparable<IgniteUuid>, Iterable<Ignite
 
     /** {@inheritDoc} */
     @Override public GridIterator<IgniteUuid> iterator() {
-        return F.iterator(Collections.singleton(this), F.<IgniteUuid>identity(), true);
+        return F.identityIterator(Collections.singleton(this), true);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/64f6e328/modules/core/src/test/java/org/apache/ignite/lang/GridFuncPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/lang/GridFuncPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/lang/GridFuncPerformanceTest.java
deleted file mode 100644
index 5afd75f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/lang/GridFuncPerformanceTest.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.lang;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import org.apache.ignite.internal.util.lang.GridIterator;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.testframework.junits.common.GridCommonTest;
-
-/**
- * GridFunc performance test.
- */
-@GridCommonTest(group = "Lang")
-public class GridFuncPerformanceTest extends GridCommonAbstractTest {
-    /**
-     *  Creates test.
-     */
-    public GridFuncPerformanceTest() {
-        super(/*start grid*/false);
-    }
-
-    /**
-     *
-     */
-    public void testTransformingIteratorPerformance() {
-        // Warmup.
-        testBody();
-        testBody();
-        testBody();
-
-        long r1 = testBody();
-        long r2 = testBody();
-        long r3 = testBody();
-
-        double r = (r1 + r2 + r3) / 3.f;
-
-        System.out.println("Average result is: " + Math.round(r) + "msec.");
-    }
-
-    /**
-     *
-     * @return Duration of the test.
-     */
-    @SuppressWarnings({"UnusedDeclaration"})
-    private long testBody() {
-        int MAX = 20000000;
-
-        Collection<Integer> l = new ArrayList<>(MAX);
-
-        for (int i = 0; i < MAX / 10; i++)
-            l.add(i);
-
-        IgniteClosure<Integer, Integer> c = new IgniteClosure<Integer, Integer>() {
-            @Override public Integer apply(Integer e) {
-                return e;
-            }
-        };
-
-        IgnitePredicate<Integer> p1 = new IgnitePredicate<Integer>() {
-            @Override public boolean apply(Integer e) {
-                return e % 2 == 0;
-            }
-        };
-        IgnitePredicate<Integer> p2 = new IgnitePredicate<Integer>() {
-            @Override public boolean apply(Integer e) {
-                return e % 2 != 0;
-            }
-        };
-
-        GridIterator<Integer> iter = F.iterator(l, c, true, p1, p2);
-
-        long n = 0;
-
-        long start = System.currentTimeMillis();
-
-        for (Integer i : iter)
-            n += i;
-
-        long duration = System.currentTimeMillis() - start;
-
-        System.out.println("Duration: " + duration + "msec.");
-
-        return duration;
-    }
-}
\ No newline at end of file


[5/5] ignite git commit: IGNITE-2263: Removed "lose" method. GridFunc has no warnings for now.

Posted by vo...@apache.org.
IGNITE-2263: Removed "lose" method. GridFunc has no warnings for now.


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

Branch: refs/heads/ignite-2263
Commit: c4cae9d3cdaf64c02385dcee8b73309458f2d31b
Parents: 9214e3c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Dec 30 17:33:17 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 30 17:33:17 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/util/lang/GridFunc.java | 17 ++++++++---------
 1 file changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4cae9d3/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index f30abcc..b5e7da9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -825,11 +825,11 @@ public class GridFunc {
      * @param c Input collection.
      * @param cp If {@code true} method creates new collection without modifying the input one,
      *      otherwise does <tt>in-place</tt> modifications.
-     * @param p Predicates to filter by. If no predicates provided - no elements are lost.
+     * @param p Predicate.
      * @param <T> Type of collections.
      * @return Collection of remaining elements.
      */
-    public static <T> Collection<T> lose(Collection<T> c, boolean cp, @Nullable IgnitePredicate<? super T>... p) {
+    public static <T> Collection<T> lose(Collection<T> c, boolean cp, IgnitePredicate<? super T> p) {
         A.notNull(c, "c");
 
         Collection<T> res;
@@ -837,19 +837,18 @@ public class GridFunc {
         if (!cp) {
             res = c;
 
-            if (isEmpty(p))
-                res.clear();
-            else if (!isAlwaysFalse(p))
-                for (Iterator<T> iter = res.iterator(); iter.hasNext();)
-                    if (isAll(iter.next(), p))
+            if (!isAlwaysFalse(p)) {
+                for (Iterator<T> iter = res.iterator(); iter.hasNext(); )
+                    if (p.apply(iter.next()))
                         iter.remove();
+            }
         }
         else {
             res = new LinkedList<>();
 
-            if (!isEmpty(p) && !isAlwaysTrue(p))
+            if (!isAlwaysTrue(p))
                 for (T t : c)
-                    if (!isAll(t, p))
+                    if (!p.apply(t))
                         res.add(t);
         }
 


[3/5] ignite git commit: IGNITE-2263: Removed "loseList" method.

Posted by vo...@apache.org.
IGNITE-2263: Removed "loseList" method.


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

Branch: refs/heads/ignite-2263
Commit: 4b7de5433ff2f8a6695f504182d9b81a398dd537
Parents: da168c6
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Dec 30 17:27:29 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 30 17:27:29 2015 +0300

----------------------------------------------------------------------
 .../loadbalancer/GridLoadBalancerManager.java   |  8 ++++-
 .../ignite/internal/util/lang/GridFunc.java     | 35 --------------------
 2 files changed, 7 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4b7de543/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<LoadBalancingSpi
                 if (F.isEmpty(exclNodes))
                     return GridLoadBalancerManager.this.getBalancedNode(ses, top, job);
 
-                List<ClusterNode> nodes = F.loseList(top, true, exclNodes);
+                List<ClusterNode> 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/4b7de543/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index a40f188..b15bebe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -857,41 +857,6 @@ public class GridFunc {
     }
 
     /**
-     * Loses all elements in input list that are contained in {@code filter} collection.
-     *
-     * @param c Input list.
-     * @param cp If {@code true} method creates new list not modifying input,
-     *      otherwise does <tt>in-place</tt> modifications.
-     * @param filter Filter collection. If {@code filter} collection is empty or
-     *      {@code null} - no elements are lost.
-     * @param <T> Type of list.
-     * @return List of remaining elements
-     */
-    // TODO: REMOVE!
-    public static <T> List<T> loseList(List<T> c, boolean cp, @Nullable Collection<? super T> filter) {
-        A.notNull(c, "c");
-
-        List<T> res;
-
-        if (!cp) {
-            res = c;
-
-            if (filter != null)
-                res.removeAll(filter);
-        }
-        else {
-            res = new LinkedList<>();
-
-            for (T t : c) {
-                if (filter == null || !filter.contains(t))
-                    res.add(t);
-            }
-        }
-
-        return res;
-    }
-
-    /**
      * Gets closure which converts node to node ID.
      *
      * @return Closure which converts node to node ID.


[4/5] ignite git commit: IGNITE-2263: Removed "isAll(Predicate)" method.

Posted by vo...@apache.org.
IGNITE-2263: Removed "isAll(Predicate)" method.


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

Branch: refs/heads/ignite-2263
Commit: 9214e3c5f711e625ab41bcbd3298a1493565354c
Parents: 4b7de54
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Dec 30 17:30:18 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 30 17:30:18 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/lang/GridFunc.java     | 26 ++++++--------------
 1 file changed, 7 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9214e3c5/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index b15bebe..f30abcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -1349,7 +1349,7 @@ public class GridFunc {
         return isAlwaysTrue(p) ? c : new GridSerializableCollection<T>() {
             // Pass through (will fail for readonly).
             @Override public boolean add(T e) {
-                return isAll(e, p) && c.add(e);
+                return p.apply(e) && c.add(e);
             }
 
             @NotNull
@@ -1462,7 +1462,7 @@ public class GridFunc {
             /** Entry predicate. */
             private IgnitePredicate<Entry<K, V>> ep = new P1<Map.Entry<K, V>>() {
                 @Override public boolean apply(Entry<K, V> e) {
-                    return isAll(e.getKey(), p);
+                    return p.apply(e.getKey());
                 }
             };
 
@@ -1480,12 +1480,12 @@ public class GridFunc {
 
                     @SuppressWarnings({"unchecked"})
                     @Override public boolean remove(Object o) {
-                        return F.isAll((Map.Entry<K, V>)o, ep) && m.entrySet().remove(o);
+                        return ep.apply((Map.Entry<K, V>)o) && m.entrySet().remove(o);
                     }
 
                     @SuppressWarnings({"unchecked"})
                     @Override public boolean contains(Object o) {
-                        return F.isAll((Map.Entry<K, V>)o, ep) && m.entrySet().contains(o);
+                        return ep.apply((Map.Entry<K, V>)o) && m.entrySet().contains(o);
                     }
 
                     @Override public boolean isEmpty() {
@@ -1500,13 +1500,13 @@ public class GridFunc {
 
             @SuppressWarnings({"unchecked"})
             @Nullable @Override public V get(Object key) {
-                return isAll((K)key, p) ? m.get(key) : null;
+                return p.apply((K)key) ? m.get(key) : null;
             }
 
             @Nullable @Override public V put(K key, V val) {
                 V oldVal = get(key);
 
-                if (isAll(key, p))
+                if (p.apply(key))
                     m.put(key, val);
 
                 return oldVal;
@@ -1514,7 +1514,7 @@ public class GridFunc {
 
             @SuppressWarnings({"unchecked"})
             @Override public boolean containsKey(Object key) {
-                return isAll((K)key, p) && m.containsKey(key);
+                return p.apply((K)key) && m.containsKey(key);
             }
         };
     }
@@ -2618,18 +2618,6 @@ public class GridFunc {
     }
 
     /**
-     * Shortcut for {@link #isAll(Object, org.apache.ignite.lang.IgnitePredicate[])} method with only single predicate.
-     *
-     * @param t Value to test.
-     * @param p Predicate.
-     * @return {@code True} if test is passed.
-     */
-    // TODO: To be removed!
-    public static <T> boolean isAll(@Nullable T t, @Nullable IgnitePredicate<? super T> p) {
-        return p == null || p.apply(t);
-    }
-
-    /**
      * Tests if all provided predicates evaluate to {@code true} for given value. Note that
      * evaluation will be short-circuit when first predicate evaluated to {@code false} is found.
      *


[2/5] ignite git commit: IGNITE-2263: Removed "isNull" predicate => not used.

Posted by vo...@apache.org.
IGNITE-2263: Removed "isNull" predicate => not used.


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

Branch: refs/heads/ignite-2263
Commit: da168c6d8a8abbb67dede6a9b645d050ea8bef4e
Parents: 64f6e32
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Dec 30 17:23:45 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 30 17:23:45 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/lang/GridFunc.java       | 18 ------------------
 1 file changed, 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/da168c6d/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index 1bf7c22..a40f188 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -144,13 +144,6 @@ public class GridFunc {
     };
 
     /** */
-    public static final IgnitePredicate<Object> IS_NULL = new P1<Object>() {
-        @Override public boolean apply(Object o) {
-            return o == null;
-        }
-    };
-
-    /** */
     public static final IgnitePredicate<Object> IS_NOT_NULL = new P1<Object>() {
         @Override public boolean apply(Object o) {
             return o != null;
@@ -2143,17 +2136,6 @@ public class GridFunc {
     }
 
     /**
-     * Gets predicate that evaluates to {@code true} if its free variable is {@code null}.
-     *
-     * @param <T> 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 {@code null}.
-     */
-    @SuppressWarnings("unchecked")
-    public static <T> IgnitePredicate<T> isNull() {
-        return (IgnitePredicate<T>)IS_NULL;
-    }
-
-    /**
      * Gets predicate that evaluates to {@code true} if its free variable is not {@code null}.
      *
      * @param <T> Type of the free variable, i.e. the element the predicate is called on.