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 2017/03/21 09:17:10 UTC

[01/14] ignite git commit: IGNITE-2552 EvictionPolicies refactored, logic changed

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 99bb87ae9 -> 92beda22f


IGNITE-2552 EvictionPolicies refactored, logic changed


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

Branch: refs/heads/ignite-2.0
Commit: 19566fbc0a683e5a72cf91ee0f8de987c524c01d
Parents: d4b87f4
Author: voipp <al...@gmail.com>
Authored: Wed Feb 15 11:54:48 2017 +0300
Committer: agura <ag...@apache.org>
Committed: Fri Mar 10 14:50:25 2017 +0300

----------------------------------------------------------------------
 .../cache/eviction/AbstractEvictionPolicy.java  | 210 +++++++++++++++++++
 .../ignite/cache/eviction/EvictionPolicy.java   |   2 -
 .../cache/eviction/fifo/FifoEvictionPolicy.java | 158 +-------------
 .../cache/eviction/lru/LruEvictionPolicy.java   | 159 ++------------
 .../eviction/random/RandomEvictionPolicy.java   | 124 -----------
 .../random/RandomEvictionPolicyMBean.java       |  42 ----
 .../eviction/sorted/SortedEvictionPolicy.java   | 160 ++------------
 .../internal/visor/util/VisorTaskUtils.java     |   8 +-
 .../resources/META-INF/classnames.properties    |   1 -
 ...idCacheConfigurationConsistencySelfTest.java |   6 +-
 .../cache/eviction/EvictionAbstractTest.java    |   4 +-
 11 files changed, 264 insertions(+), 610 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java
new file mode 100644
index 0000000..eca58e6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.eviction;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.jsr166.LongAdder8;
+
+/**
+ * Common functionality implementation for eviction policies
+ */
+public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K, V> {
+    /** Memory size occupied by elements in container. */
+    private final LongAdder8 memSize = new LongAdder8();
+
+    /** Max memory size occupied by elements in container. */
+    private volatile long maxMemSize;
+
+    /** Maximum elements in container. */
+    private volatile int max;
+
+    /** Batch size. */
+    private volatile int batchSize = 1;
+
+    /**
+     * Shrinks backed container to maximum allowed size.
+     */
+    protected void shrink() {
+        long maxMem = this.maxMemSize;
+
+        if (maxMem > 0) {
+            long startMemSize = memSize.longValue();
+
+            if (startMemSize >= maxMem)
+                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem; ) {
+                    int size = shrink0();
+
+                    if (size == -1)
+                        break;
+
+                    i += size;
+                }
+        }
+
+        int max = this.max;
+
+        if (max > 0) {
+            int startSize = getCurrentSize();
+
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
+                for (int i = max; i < startSize && getCurrentSize() > max; i++)
+                    if (shrink0() == -1)
+                        break;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onEntryAccessed(boolean rmv, EvictableEntry<K, V> entry) {
+        if (!rmv) {
+            if (!entry.isCached())
+                return;
+
+            if (touch(entry))
+                shrink();
+        }
+        else {
+            Object node = entry.removeMeta();
+
+            if (node != null) {
+                removeMeta(node);
+
+                addToMemorySize(-entry.size());
+            }
+
+        }
+    }
+
+    /**
+     * @param x Changing memory size by adding the value.
+     */
+    protected void addToMemorySize(int x) {
+        memSize.add(x);
+    }
+
+    /**
+     *
+     * @return Size of the container.
+     */
+    protected abstract int getCurrentSize();
+
+    /**
+     *
+     * @return Size of the memory which was shrinked0.
+     */
+    protected abstract int shrink0();
+
+    /**
+     *
+     * @param meta Meta-information shipped to an entry.
+     * @return {@code True} if meta was successfully removed from the container.
+     */
+    protected abstract boolean removeMeta(Object meta);
+
+    /**
+     * @param entry Entry to touch.
+     * @return {@code True} if container has been changed by this call.
+     */
+    protected abstract boolean touch(EvictableEntry<K, V> entry);
+
+    /**
+     * Sets maximum allowed cache size in bytes.
+     */
+    public void setMaxMemorySize(long maxMemSize) {
+        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
+
+        this.maxMemSize = maxMemSize;
+    }
+
+    /**
+     * Gets maximum allowed cache size in bytes.
+     *
+     * @return maximum allowed cache size in bytes.
+     */
+    public long getMaxMemorySize() {
+        return maxMemSize;
+    }
+
+    /**
+     * Gets current queue size in bytes.
+     *
+     * @return current queue size in bytes.
+     */
+    public long getCurrentMemorySize() {
+        return memSize.longValue();
+    }
+
+    /**
+     * Sets maximum allowed size of cache before entry will start getting evicted.
+     *
+     * @param max Maximum allowed size of cache before entry will start getting evicted.
+     */
+    public void setMaxSize(int max) {
+        A.ensure(max >= 0, "max >= 0");
+
+        this.max = max;
+    }
+
+    /**
+     * Gets maximum allowed size of cache before entry will start getting evicted.
+     *
+     * @return Maximum allowed size of cache before entry will start getting evicted.
+     */
+    public int getMaxSize() {
+        return max;
+    }
+
+    /**
+     * Sets batch size.
+     *
+     * @param batchSize Batch size.
+     */
+    public void setBatchSize(int batchSize) {
+        A.ensure(batchSize > 0, "batchSize > 0");
+
+        this.batchSize = batchSize;
+    }
+
+    /**
+     * Gets batch size.
+     *
+     * @return batch size.
+     */
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public LongAdder8 getMemSize() {
+        return memSize;
+    }
+
+    /** {@inheritDoc} */
+    public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(max);
+        out.writeInt(batchSize);
+        out.writeLong(maxMemSize);
+    }
+    /** {@inheritDoc} */
+    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        max = in.readInt();
+        batchSize = in.readInt();
+        maxMemSize = in.readLong();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
index a77514a..6125b50 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
@@ -19,7 +19,6 @@ package org.apache.ignite.cache.eviction;
 
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
-import org.apache.ignite.cache.eviction.random.RandomEvictionPolicy;
 import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy;
 
 /**
@@ -31,7 +30,6 @@ import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy;
  * Ignite comes with following eviction policies out-of-the-box:
  * <ul>
  * <li>{@link LruEvictionPolicy}</li>
- * <li>{@link RandomEvictionPolicy}</li>
  * <li>{@link FifoEvictionPolicy}</li>
  * <li>{@link SortedEvictionPolicy}</li>
  * </ul>

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
index 4d1c3f1..dd7ba0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
@@ -18,20 +18,13 @@
 package org.apache.ignite.cache.eviction.fifo;
 
 import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
 import java.util.Collection;
 import java.util.Collections;
+import org.apache.ignite.cache.eviction.AbstractEvictionPolicy;
 import org.apache.ignite.cache.eviction.EvictableEntry;
-import org.apache.ignite.cache.eviction.EvictionPolicy;
-import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jsr166.ConcurrentLinkedDeque8;
 import org.jsr166.ConcurrentLinkedDeque8.Node;
-import org.jsr166.LongAdder8;
-
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE;
 
 /**
  * Eviction policy based on {@code First In First Out (FIFO)} algorithm and supports batch eviction.
@@ -51,22 +44,10 @@ import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE
  * table-like data structures. The {@code FIFO} ordering information is
  * maintained by attaching ordering metadata to cache entries.
  */
-public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvictionPolicyMBean, Externalizable {
+public class FifoEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements FifoEvictionPolicyMBean, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Maximum size. */
-    private volatile int max = DFLT_CACHE_SIZE;
-
-    /** Batch size. */
-    private volatile int batchSize = 1;
-
-    /** Max memory size. */
-    private volatile long maxMemSize;
-
-    /** Memory size. */
-    private final LongAdder8 memSize = new LongAdder8();
-
     /** FIFO queue. */
     private final ConcurrentLinkedDeque8<EvictableEntry<K, V>> queue =
         new ConcurrentLinkedDeque8<>();
@@ -84,9 +65,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     public FifoEvictionPolicy(int max) {
-        A.ensure(max >= 0, "max >= 0");
-
-        this.max = max;
+        setMaxSize(max);
     }
 
     /**
@@ -96,43 +75,8 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * @param batchSize Batch size.
      */
     public FifoEvictionPolicy(int max, int batchSize) {
-        A.ensure(max >= 0, "max >= 0");
-        A.ensure(batchSize > 0, "batchSize > 0");
-
-        this.max = max;
-        this.batchSize = batchSize;
-    }
-
-    /**
-     * Gets maximum allowed size of cache before entry will start getting evicted.
-     *
-     * @return Maximum allowed size of cache before entry will start getting evicted.
-     */
-    @Override public int getMaxSize() {
-        return max;
-    }
-
-    /**
-     * Sets maximum allowed size of cache before entry will start getting evicted.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    @Override public void setMaxSize(int max) {
-        A.ensure(max >= 0, "max >= 0");
-
-        this.max = max;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getBatchSize() {
-        return batchSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setBatchSize(int batchSize) {
-        A.ensure(batchSize > 0, "batchSize > 0");
-
-        this.batchSize = batchSize;
+        setMaxSize(max);
+        setBatchSize(batchSize);
     }
 
     /** {@inheritDoc} */
@@ -140,23 +84,6 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
         return queue.size();
     }
 
-    /** {@inheritDoc} */
-    @Override public long getMaxMemorySize() {
-        return maxMemSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setMaxMemorySize(long maxMemSize) {
-        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
-
-        this.maxMemSize = maxMemSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getCurrentMemorySize() {
-        return memSize.longValue();
-    }
-
     /**
      * Gets read-only view on internal {@code FIFO} queue in proper order.
      *
@@ -167,31 +94,15 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
     }
 
     /** {@inheritDoc} */
-    @Override public void onEntryAccessed(boolean rmv, EvictableEntry<K, V> entry) {
-        if (!rmv) {
-            if (!entry.isCached())
-                return;
-
-            // Shrink only if queue was changed.
-            if (touch(entry))
-                shrink();
-        }
-        else {
-            Node<EvictableEntry<K, V>> node = entry.removeMeta();
-
-            if (node != null) {
-                queue.unlinkx(node);
-
-                memSize.add(-entry.size());
-            }
-        }
+    @Override protected boolean removeMeta(Object meta) {
+        return queue.unlinkx((Node<EvictableEntry<K, V>>)meta);
     }
 
     /**
      * @param entry Entry to touch.
      * @return {@code True} if queue has been changed by this call.
      */
-    private boolean touch(EvictableEntry<K, V> entry) {
+    protected boolean touch(EvictableEntry<K, V> entry) {
         Node<EvictableEntry<K, V>> node = entry.meta();
 
         // Entry has not been enqueued yet.
@@ -214,7 +125,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
                         return false;
                     }
 
-                    memSize.add(entry.size());
+                    addToMemorySize(entry.size());
 
                     return true;
                 }
@@ -229,44 +140,11 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
     }
 
     /**
-     * Shrinks FIFO queue to maximum allowed size.
-     */
-    private void shrink() {
-        long maxMem = this.maxMemSize;
-
-        if (maxMem > 0) {
-            long startMemSize = memSize.longValue();
-
-            if (startMemSize >= maxMem)
-                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
-                    int size = shrink0();
-
-                    if (size == -1)
-                        break;
-
-                    i += size;
-                }
-        }
-
-        int max = this.max;
-
-        if (max > 0) {
-            int startSize = queue.sizex();
-
-            // Shrink only if queue is full.
-            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
-                for (int i = max; i < startSize && queue.sizex() > max; i++)
-                    if (shrink0() == -1)
-                        break;
-        }
-    }
-
-    /**
      * Tries to remove one item from queue.
      *
      * @return number of bytes that was free. {@code -1} if queue is empty.
      */
-    private int shrink0() {
+    @Override protected int shrink0() {
         EvictableEntry<K, V> entry = queue.poll();
 
         if (entry == null)
@@ -279,7 +157,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
         if (meta != null) {
             size = entry.size();
 
-            memSize.add(-size);
+            addToMemorySize(-size);
 
             if (!entry.evict())
                 touch(entry);
@@ -289,20 +167,6 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
     }
 
     /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeInt(max);
-        out.writeInt(batchSize);
-        out.writeLong(maxMemSize);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        max = in.readInt();
-        batchSize = in.readInt();
-        maxMemSize = in.readLong();
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(FifoEvictionPolicy.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
index f321764..caa4b3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
@@ -18,20 +18,13 @@
 package org.apache.ignite.cache.eviction.lru;
 
 import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
 import java.util.Collection;
 import java.util.Collections;
+import org.apache.ignite.cache.eviction.AbstractEvictionPolicy;
 import org.apache.ignite.cache.eviction.EvictableEntry;
-import org.apache.ignite.cache.eviction.EvictionPolicy;
-import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jsr166.ConcurrentLinkedDeque8;
 import org.jsr166.ConcurrentLinkedDeque8.Node;
-import org.jsr166.LongAdder8;
-
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE;
 
 /**
  * Eviction policy based on {@code Least Recently Used (LRU)} algorithm and supports batch eviction.
@@ -50,22 +43,10 @@ import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE
  * This implementation is very efficient since it is lock-free and does not create any additional table-like
  * data structures. The {@code LRU} ordering information is maintained by attaching ordering metadata to cache entries.
  */
-public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictionPolicyMBean, Externalizable {
+public class LruEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements LruEvictionPolicyMBean, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Maximum size. */
-    private volatile int max = DFLT_CACHE_SIZE;
-
-    /** Batch size. */
-    private volatile int batchSize = 1;
-
-    /** Max memory size. */
-    private volatile long maxMemSize;
-
-    /** Memory size. */
-    private final LongAdder8 memSize = new LongAdder8();
-
     /** Queue. */
     private final ConcurrentLinkedDeque8<EvictableEntry<K, V>> queue =
         new ConcurrentLinkedDeque8<>();
@@ -83,41 +64,7 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     public LruEvictionPolicy(int max) {
-        A.ensure(max >= 0, "max >= 0");
-
-        this.max = max;
-    }
-
-    /**
-     * Gets maximum allowed size of cache before entry will start getting evicted.
-     *
-     * @return Maximum allowed size of cache before entry will start getting evicted.
-     */
-    @Override public int getMaxSize() {
-        return max;
-    }
-
-    /**
-     * Sets maximum allowed size of cache before entry will start getting evicted.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    @Override public void setMaxSize(int max) {
-        A.ensure(max >= 0, "max >= 0");
-
-        this.max = max;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getBatchSize() {
-        return batchSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setBatchSize(int batchSize) {
-        A.ensure(batchSize > 0, "batchSize > 0");
-
-        this.batchSize = batchSize;
+        setMaxSize(max);
     }
 
     /** {@inheritDoc} */
@@ -125,23 +72,6 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
         return queue.size();
     }
 
-    /** {@inheritDoc} */
-    @Override public long getMaxMemorySize() {
-        return maxMemSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setMaxMemorySize(long maxMemSize) {
-        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
-
-        this.maxMemSize = maxMemSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getCurrentMemorySize() {
-        return memSize.longValue();
-    }
-
     /**
      * Gets read-only view on internal {@code FIFO} queue in proper order.
      *
@@ -152,30 +82,15 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
     }
 
     /** {@inheritDoc} */
-    @Override public void onEntryAccessed(boolean rmv, EvictableEntry<K, V> entry) {
-        if (!rmv) {
-            if (!entry.isCached())
-                return;
-
-            if (touch(entry))
-                shrink();
-        }
-        else {
-            Node<EvictableEntry<K, V>> node = entry.removeMeta();
-
-            if (node != null) {
-                queue.unlinkx(node);
-
-                memSize.add(-entry.size());
-            }
-        }
+    @Override protected boolean removeMeta(Object meta) {
+        return queue.unlinkx((Node<EvictableEntry<K, V>>)meta);
     }
 
     /**
      * @param entry Entry to touch.
      * @return {@code True} if new node has been added to queue by this call.
      */
-    private boolean touch(EvictableEntry<K, V> entry) {
+    @Override protected boolean touch(EvictableEntry<K, V> entry) {
         Node<EvictableEntry<K, V>> node = entry.meta();
 
         // Entry has not been enqueued yet.
@@ -185,7 +100,7 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
 
                 if (entry.putMetaIfAbsent(node) != null) {
                     // Was concurrently added, need to clear it from queue.
-                    queue.unlinkx(node);
+                    removeMeta(node);
 
                     // Queue has not been changed.
                     return false;
@@ -193,12 +108,12 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
                 else if (node.item() != null) {
                     if (!entry.isCached()) {
                         // Was concurrently evicted, need to clear it from queue.
-                        queue.unlinkx(node);
+                        removeMeta(node);
 
                         return false;
                     }
 
-                    memSize.add(entry.size());
+                    addToMemorySize(entry.size());
 
                     return true;
                 }
@@ -207,13 +122,13 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
                     return false;
             }
         }
-        else if (queue.unlinkx(node)) {
+        else if (removeMeta(node)) {
             // Move node to tail.
             Node<EvictableEntry<K, V>> newNode = queue.offerLastx(entry);
 
             if (!entry.replaceMeta(node, newNode))
                 // Was concurrently added, need to clear it from queue.
-                queue.unlinkx(newNode);
+                removeMeta(newNode);
         }
 
         // Entry is already in queue.
@@ -221,43 +136,11 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
     }
 
     /**
-     * Shrinks queue to maximum allowed size.
-     */
-    private void shrink() {
-        long maxMem = this.maxMemSize;
-
-        if (maxMem > 0) {
-            long startMemSize = memSize.longValue();
-
-            if (startMemSize >= maxMem)
-                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
-                    int size = shrink0();
-
-                    if (size == -1)
-                        break;
-
-                    i += size;
-                }
-        }
-
-        int max = this.max;
-
-        if (max > 0) {
-            int startSize = queue.sizex();
-
-            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
-                for (int i = max; i < startSize && queue.sizex() > max; i++)
-                    if (shrink0() == -1)
-                        break;
-        }
-    }
-
-    /**
      * Tries to remove one item from queue.
      *
      * @return number of bytes that was free. {@code -1} if queue is empty.
      */
-    private int shrink0() {
+    @Override protected int shrink0() {
         EvictableEntry<K, V> entry = queue.poll();
 
         if (entry == null)
@@ -270,7 +153,7 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
         if (meta != null) {
             size = entry.size();
 
-            memSize.add(-size);
+            addToMemorySize(-size);
 
             if (!entry.evict())
                 touch(entry);
@@ -280,21 +163,7 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
     }
 
     /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeInt(max);
-        out.writeInt(batchSize);
-        out.writeLong(maxMemSize);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        max = in.readInt();
-        batchSize = in.readInt();
-        maxMemSize = in.readLong();
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(LruEvictionPolicy.class, this, "size", queue.sizex());
+        return S.toString(LruEvictionPolicy.class, this, "size", getCurrentSize());
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
deleted file mode 100644
index f77551d..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
+++ /dev/null
@@ -1,124 +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.cache.eviction.random;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import javax.cache.Cache;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.eviction.EvictableEntry;
-import org.apache.ignite.cache.eviction.EvictionPolicy;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE;
-
-/**
- * Cache eviction policy which will select random cache entry for eviction if cache
- * size exceeds the {@link #getMaxSize()} parameter. This implementation is
- * extremely light weight, lock-free, and does not create any data structures to maintain
- * any order for eviction.
- * <p>
- * Random eviction will provide the best performance over any key queue in which every
- * key has the same probability of being accessed.
- *
- * @deprecated This eviction policy implementation doesn't support near cache
- * and doesn't work on client nodes. Also it seems that random eviction doesn't make sense.
- */
-@Deprecated
-public class RandomEvictionPolicy<K, V> implements EvictionPolicy<K, V>, RandomEvictionPolicyMBean, Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Maximum size. */
-    private volatile int max = DFLT_CACHE_SIZE;
-
-    /**
-     * Constructs random eviction policy with all defaults.
-     */
-    public RandomEvictionPolicy() {
-        // No-op.
-    }
-
-    /**
-     * Constructs random eviction policy with maximum size.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    public RandomEvictionPolicy(int max) {
-        A.ensure(max > 0, "max > 0");
-
-        this.max = max;
-    }
-
-    /**
-     * Gets maximum allowed size of cache before entry will start getting evicted.
-     *
-     * @return Maximum allowed size of cache before entry will start getting evicted.
-     */
-    @Override public int getMaxSize() {
-        return max;
-    }
-
-    /**
-     * Sets maximum allowed size of cache before entry will start getting evicted.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    @Override public void setMaxSize(int max) {
-        A.ensure(max > 0, "max > 0");
-
-        this.max = max;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void onEntryAccessed(boolean rmv, EvictableEntry<K, V> entry) {
-        if (!entry.isCached())
-            return;
-
-        IgniteCache<K, V> cache = entry.unwrap(IgniteCache.class);
-
-        int size = cache.localSize(CachePeekMode.ONHEAP);
-
-        for (int i = max; i < size; i++) {
-            Cache.Entry<K, V> e = cache.randomEntry();
-
-            if (e != null)
-                e.unwrap(EvictableEntry.class).evict();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeInt(max);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        max = in.readInt();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(RandomEvictionPolicy.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicyMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicyMBean.java
deleted file mode 100644
index 3193483..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicyMBean.java
+++ /dev/null
@@ -1,42 +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.cache.eviction.random;
-
-import org.apache.ignite.mxbean.MXBeanDescription;
-
-/**
- * MBean for {@code random} eviction policy.
- */
-@MXBeanDescription("MBean for random cache eviction policy.")
-public interface RandomEvictionPolicyMBean {
-    /**
-     * Gets maximum allowed cache size.
-     *
-     * @return Maximum allowed cache size.
-     */
-    @MXBeanDescription("Maximum allowed cache size.")
-    public int getMaxSize();
-
-    /**
-     * Sets maximum allowed cache size.
-     *
-     * @param max Maximum allowed cache size.
-     */
-    @MXBeanDescription("Sets maximum allowed cache size.")
-    public void setMaxSize(int max);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
index 6189552..47f8479 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
@@ -29,8 +29,8 @@ import java.util.LinkedHashSet;
 import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.cache.eviction.AbstractEvictionPolicy;
 import org.apache.ignite.cache.eviction.EvictableEntry;
-import org.apache.ignite.cache.eviction.EvictionPolicy;
 import org.apache.ignite.internal.util.GridConcurrentSkipListSet;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.jetbrains.annotations.Nullable;
@@ -59,22 +59,10 @@ import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE
  * <p>
  * User defined comparator should implement {@link Serializable} interface.
  */
-public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedEvictionPolicyMBean, Externalizable {
+public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements SortedEvictionPolicyMBean, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Maximum size. */
-    private volatile int max = DFLT_CACHE_SIZE;
-
-    /** Batch size. */
-    private volatile int batchSize = 1;
-
-    /** Max memory size. */
-    private volatile long maxMemSize;
-
-    /** Memory size. */
-    private final LongAdder8 memSize = new LongAdder8();
-
     /** Comparator. */
     private Comparator<Holder<K, V>> comp;
 
@@ -118,11 +106,8 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
      * @param comp Entries comparator.
      */
     public SortedEvictionPolicy(int max, int batchSize, @Nullable Comparator<EvictableEntry<K, V>> comp) {
-        A.ensure(max >= 0, "max >= 0");
-        A.ensure(batchSize > 0, "batchSize > 0");
-
-        this.max = max;
-        this.batchSize = batchSize;
+        setMaxSize(max);
+        setBatchSize(batchSize);
         this.comp = comp == null ? new DefaultHolderComparator<K, V>() : new HolderComparator<>(comp);
         this.set = new GridConcurrentSkipListSetEx<>(this.comp);
     }
@@ -138,60 +123,6 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     }
 
     /**
-     * Gets maximum allowed size of cache before entry will start getting evicted.
-     *
-     * @return Maximum allowed size of cache before entry will start getting evicted.
-     */
-    @Override public int getMaxSize() {
-        return max;
-    }
-
-    /**
-     * Sets maximum allowed size of cache before entry will start getting evicted.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    @Override public void setMaxSize(int max) {
-        A.ensure(max >= 0, "max >= 0");
-
-        this.max = max;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getBatchSize() {
-        return batchSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setBatchSize(int batchSize) {
-        A.ensure(batchSize > 0, "batchSize > 0");
-
-        this.batchSize = batchSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getCurrentSize() {
-        return set.sizex();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getMaxMemorySize() {
-        return maxMemSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setMaxMemorySize(long maxMemSize) {
-        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
-
-        this.maxMemSize = maxMemSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getCurrentMemorySize() {
-        return memSize.longValue();
-    }
-
-    /**
      * Gets read-only view of backed queue in proper order.
      *
      * @return Read-only view of backed queue.
@@ -205,31 +136,11 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
         return Collections.unmodifiableCollection(cp);
     }
 
-    /** {@inheritDoc} */
-    @Override public void onEntryAccessed(boolean rmv, EvictableEntry<K, V> entry) {
-        if (!rmv) {
-            if (!entry.isCached())
-                return;
-
-            if (touch(entry))
-                shrink();
-        }
-        else {
-            Holder<K, V> holder = entry.removeMeta();
-
-            if (holder != null) {
-                removeHolder(holder);
-
-                memSize.add(-entry.size());
-            }
-        }
-    }
-
     /**
      * @param entry Entry to touch.
      * @return {@code True} if backed queue has been changed by this call.
      */
-    private boolean touch(EvictableEntry<K, V> entry) {
+    @Override protected boolean touch(EvictableEntry<K, V> entry) {
         Holder<K, V> holder = entry.meta();
 
         // Entry has not been added yet to backed queue.
@@ -245,12 +156,12 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
                 if (holder.order > 0) {
                     if (!entry.isCached()) {
                         // Was concurrently evicted, need to remove it from queue.
-                        removeHolder(holder);
+                        removeMeta(holder);
 
                         return false;
                     }
 
-                    memSize.add(entry.size());
+                    addToMemorySize(entry.size());
 
                     return true;
                 }
@@ -264,38 +175,9 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
         return false;
     }
 
-    /**
-     * Shrinks backed queue to maximum allowed size.
-     */
-    private void shrink() {
-        long maxMem = this.maxMemSize;
-
-        if (maxMem > 0) {
-            long startMemSize = memSize.longValue();
-
-            if (startMemSize >= maxMem)
-                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
-                    int size = shrink0();
-
-                    if (size == -1)
-                        break;
-
-                    i += size;
-                }
-        }
-
-        int max = this.max;
-
-        if (max > 0) {
-            int startSize = set.sizex();
-
-            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize)) {
-                for (int i = max; i < startSize && set.sizex() > max; i++) {
-                    if (shrink0() == -1)
-                        break;
-                }
-            }
-        }
+    /** {@inheritDoc} */
+    @Override public int getCurrentSize() {
+        return set.size();
     }
 
     /**
@@ -303,7 +185,7 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
      *
      * @return number of bytes that was free. {@code -1} if queue is empty.
      */
-    private int shrink0() {
+    @Override protected int shrink0() {
         Holder<K, V> h = set.pollFirst();
 
         if (h == null)
@@ -313,10 +195,10 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
 
         EvictableEntry<K, V> entry = h.entry;
 
-        if (h.order > 0 && entry.removeMeta(h)) {
+        if (entry != null && h.order > 0 && entry.removeMeta(h)) {
             size = entry.size();
 
-            memSize.add(-size);
+            addToMemorySize(-size);
 
             if (!entry.evict())
                 touch(entry);
@@ -327,33 +209,31 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeInt(max);
-        out.writeInt(batchSize);
-        out.writeLong(maxMemSize);
+        super.writeExternal(out);
         out.writeObject(comp);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        max = in.readInt();
-        batchSize = in.readInt();
-        maxMemSize = in.readLong();
+        super.readExternal(in);
         comp = (Comparator<Holder<K, V>>)in.readObject();
     }
 
     /**
      * Removes holder from backed queue and marks holder as removed.
      *
-     * @param holder Holder.
+     * @param meta Holder.
      */
-    private void removeHolder(Holder<K, V> holder) {
+    @Override protected boolean removeMeta(Object meta) {
+        Holder<K, V> holder = (Holder<K, V>)meta;
+
         long order0 = holder.order;
 
         if (order0 > 0)
             holder.order = -order0;
 
-        set.remove(holder);
+        return set.remove(holder);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
index 3f5003a..8e4590e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
@@ -53,7 +53,7 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.eviction.EvictionPolicy;
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicyMBean;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicyMBean;
-import org.apache.ignite.cache.eviction.random.RandomEvictionPolicyMBean;
+import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicyMBean;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
@@ -693,12 +693,12 @@ public class VisorTaskUtils {
         if (plc instanceof LruEvictionPolicyMBean)
             return ((LruEvictionPolicyMBean)plc).getMaxSize();
 
-        if (plc instanceof RandomEvictionPolicyMBean)
-            return ((RandomEvictionPolicyMBean)plc).getMaxSize();
-
         if (plc instanceof FifoEvictionPolicyMBean)
             return ((FifoEvictionPolicyMBean)plc).getMaxSize();
 
+        if (plc instanceof SortedEvictionPolicyMBean)
+            return ((SortedEvictionPolicyMBean)plc).getMaxSize();
+
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 8cfed3a..eb8bfad 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -67,7 +67,6 @@ org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy
 org.apache.ignite.cache.eviction.igfs.IgfsEvictionFilter
 org.apache.ignite.cache.eviction.igfs.IgfsPerBlockLruEvictionPolicy
 org.apache.ignite.cache.eviction.lru.LruEvictionPolicy
-org.apache.ignite.cache.eviction.random.RandomEvictionPolicy
 org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy
 org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy$DefaultHolderComparator
 org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy$GridConcurrentSkipListSetEx

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
index a1f917f..7157531 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.eviction.EvictionFilter;
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
-import org.apache.ignite.cache.eviction.random.RandomEvictionPolicy;
+import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DeploymentMode;
@@ -382,7 +382,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             new C1<CacheConfiguration, Void>() {
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
-                    cfg.setEvictionPolicy(new RandomEvictionPolicy());
+                    cfg.setEvictionPolicy(new SortedEvictionPolicy());
                     return null;
                 }
             },
@@ -632,7 +632,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             @Override public Void apply(CacheConfiguration cfg) {
                 NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 
-                nearCfg.setNearEvictionPolicy(new RandomEvictionPolicy());
+                nearCfg.setNearEvictionPolicy(new LruEvictionPolicy());
 
                 cfg.setNearConfiguration(nearCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/19566fbc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
index 4acf1af..cee04ed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
@@ -1030,7 +1030,7 @@ public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
          */
         public long getCurrentMemorySize() {
             try {
-                return (Long)plc.getClass().getDeclaredMethod("getCurrentMemorySize").invoke(plc);
+                return (Long)plc.getClass().getMethod("getCurrentMemorySize").invoke(plc);
             }
             catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
                 throw new RuntimeException(e);
@@ -1056,7 +1056,7 @@ public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
         @Override public void onEntryAccessed(boolean rmv, EvictableEntry entry) {
             try {
                 plc.getClass()
-                    .getDeclaredMethod("onEntryAccessed", boolean.class, EvictableEntry.class)
+                    .getMethod("onEntryAccessed", boolean.class, EvictableEntry.class)
                     .invoke(plc, rmv, entry);
             }
             catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {


[05/14] ignite git commit: Web Console: Cleanup

Posted by vo...@apache.org.
Web Console: Cleanup


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

Branch: refs/heads/ignite-2.0
Commit: 3da7794fb59ebb836523ba717722c6a6df845019
Parents: 0b996e6
Author: Andrey Novikov <an...@gridgain.com>
Authored: Tue Mar 14 15:54:14 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Tue Mar 14 15:54:14 2017 +0700

----------------------------------------------------------------------
 .../ui-grid-settings/ui-grid-settings.scss      | 35 ++++++++++++++++++++
 1 file changed, 35 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3da7794f/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss b/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss
index 4beb2a1..24f4d9b 100644
--- a/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss
+++ b/modules/web-console/frontend/app/components/ui-grid-settings/ui-grid-settings.scss
@@ -70,6 +70,41 @@
         }
     }
 
+    &-number-filter {
+        float: right;
+
+        .ignite-form-field {
+            width: 180px;
+            margin-right: 0;
+
+            &__label {
+            }
+
+            &__control {
+            }
+
+            &:nth-child(1) {
+                float: left;
+
+                .ignite-form-field__label {
+                    margin-right: 0;
+                    width: 70%;
+                    max-width: 100%;
+                }
+
+                .ignite-form-field__control {
+                    width: 30%;
+                }
+            }
+        }
+
+        button {
+            width: auto;
+            display: inline-block;
+            margin-left: 5px;
+        }
+    }
+
     &-dateperiod {
         float: right;
 


[09/14] ignite git commit: IGNITE-4666 Clone to Input service.

Posted by vo...@apache.org.
IGNITE-4666 Clone to Input service.


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

Branch: refs/heads/ignite-2.0
Commit: 83579ce80abf3960cfe4eb07019f8ce6154df5c2
Parents: 84880a8
Author: Andrey Novikov <an...@gridgain.com>
Authored: Fri Mar 17 11:08:39 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri Mar 17 11:08:39 2017 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/app.js         | 10 +--
 .../app/components/input-dialog/index.js        | 24 ++++++
 .../input-dialog/input-dialog.controller.js     | 35 ++++++++
 .../input-dialog/input-dialog.service.js        | 88 ++++++++++++++++++++
 .../input-dialog/input-dialog.tpl.pug           | 39 +++++++++
 .../frontend/app/services/Clone.service.js      | 66 ---------------
 .../frontend/controllers/caches-controller.js   | 10 +--
 .../frontend/controllers/clusters-controller.js |  6 +-
 .../frontend/controllers/domains-controller.js  | 12 ++-
 .../frontend/controllers/igfs-controller.js     | 10 +--
 .../frontend/views/templates/clone.tpl.pug      | 39 ---------
 11 files changed, 207 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/app/app.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js
index 5e3bb07..1e21d24 100644
--- a/modules/web-console/frontend/app/app.js
+++ b/modules/web-console/frontend/app/app.js
@@ -16,9 +16,9 @@
  */
 
 import '../public/stylesheets/style.scss';
-import '../app/components/ui-grid-header/ui-grid-header.scss';
-import '../app/components/ui-grid-settings/ui-grid-settings.scss';
-import '../app/components/form-field-datepicker/form-field-datepicker.scss';
+import './components/ui-grid-header/ui-grid-header.scss';
+import './components/ui-grid-settings/ui-grid-settings.scss';
+import './components/form-field-datepicker/form-field-datepicker.scss';
 
 import './app.config';
 
@@ -80,7 +80,6 @@ import igniteRetainSelection from './directives/retain-selection.directive';
 
 // Services.
 import ChartColors from './services/ChartColors.service';
-import Clone from './services/Clone.service.js';
 import Confirm from './services/Confirm.service.js';
 import ConfirmBatch from './services/ConfirmBatch.service.js';
 import CopyToClipboard from './services/CopyToClipboard.service';
@@ -117,6 +116,7 @@ import resetPassword from './controllers/reset-password.controller';
 // Components
 import igniteListOfRegisteredUsers from './components/list-of-registered-users';
 import IgniteActivitiesUserDialog from './components/activities-user-dialog';
+import './components/input-dialog';
 
 // Inject external modules.
 import 'ignite_modules_temp/index';
@@ -149,6 +149,7 @@ angular
     'ignite-console.core',
     'ignite-console.ace',
     'ignite-console.Form',
+    'ignite-console.input-dialog',
     'ignite-console.user',
     'ignite-console.branding',
     'ignite-console.socket',
@@ -203,7 +204,6 @@ angular
 .service('JavaTypes', JavaTypes)
 .service('SqlTypes', SqlTypes)
 .service(...ChartColors)
-.service(...Clone)
 .service(...Confirm)
 .service(...ConfirmBatch)
 .service(...CopyToClipboard)

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/app/components/input-dialog/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/input-dialog/index.js b/modules/web-console/frontend/app/components/input-dialog/index.js
new file mode 100644
index 0000000..4bb9642
--- /dev/null
+++ b/modules/web-console/frontend/app/components/input-dialog/index.js
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+import angular from 'angular';
+
+import inputDialog from './input-dialog.service';
+
+angular
+    .module('ignite-console.input-dialog', [])
+    .service('IgniteInput', inputDialog);

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/app/components/input-dialog/input-dialog.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/input-dialog/input-dialog.controller.js b/modules/web-console/frontend/app/components/input-dialog/input-dialog.controller.js
new file mode 100644
index 0000000..3f6e97b
--- /dev/null
+++ b/modules/web-console/frontend/app/components/input-dialog/input-dialog.controller.js
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+export default class InputDialogController {
+    static $inject = ['deferred', 'ui'];
+
+    constructor(deferred, {title, label, value, toValidValue}) {
+        this.deferred = deferred;
+        this.title = title;
+        this.label = label;
+        this.value = value;
+        this.toValidValue = toValidValue;
+    }
+
+    confirm() {
+        if (_.isFunction(this.toValidValue))
+            return this.deferred.resolve(this.toValidValue(this.value));
+
+        this.deferred.resolve(this.value);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/app/components/input-dialog/input-dialog.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/input-dialog/input-dialog.service.js b/modules/web-console/frontend/app/components/input-dialog/input-dialog.service.js
new file mode 100644
index 0000000..fc3cb85
--- /dev/null
+++ b/modules/web-console/frontend/app/components/input-dialog/input-dialog.service.js
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+
+import controller from './input-dialog.controller';
+import templateUrl from './input-dialog.tpl.pug';
+
+export default class InputDialog {
+    static $inject = ['$modal', '$q'];
+
+    constructor($modal, $q) {
+        this.$modal = $modal;
+        this.$q = $q;
+    }
+
+    /**
+     * Open input dialog to configure custom value.
+     *
+     * @param {String} title Dialog title.
+     * @param {String} label Input field label.
+     * @param {String} value Default value.
+     * @param {Function} [toValidValue] Validator function.
+     * @returns {Promise.<String>} User input.
+     */
+    input(title, label, value, toValidValue) {
+        const deferred = this.$q.defer();
+
+        const modal = this.$modal({
+            templateUrl,
+            resolve: {
+                deferred: () => deferred,
+                ui: () => ({
+                    title,
+                    label,
+                    value,
+                    toValidValue
+                })
+            },
+            placement: 'center',
+            controller,
+            controllerAs: 'ctrl'
+        });
+
+        const modalHide = modal.hide;
+
+        modal.hide = () => deferred.reject('cancelled');
+
+        return deferred.promise
+            .finally(modalHide);
+    }
+
+    /**
+     * Open input dialog to configure cloned object name.
+     *
+     * @param {String} srcName Name of source object.
+     * @param {Array.<String>} names List of already exist names.
+     * @returns {Promise.<String>} New name
+     */
+    clone(srcName, names) {
+        const uniqueName = (value) => {
+            let num = 1;
+            let tmpName = value;
+
+            while (_.includes(names, tmpName)) {
+                tmpName = `${value}_${num}`;
+
+                num++;
+            }
+
+            return tmpName;
+        };
+
+        return this.input('Clone', 'New name', uniqueName(srcName), uniqueName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/app/components/input-dialog/input-dialog.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/input-dialog/input-dialog.tpl.pug b/modules/web-console/frontend/app/components/input-dialog/input-dialog.tpl.pug
new file mode 100644
index 0000000..95549d7
--- /dev/null
+++ b/modules/web-console/frontend/app/components/input-dialog/input-dialog.tpl.pug
@@ -0,0 +1,39 @@
+//-
+    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.
+
+include /app/helpers/jade/mixins
+
+.modal(tabindex='-1' role='dialog')
+    .modal-dialog
+        .modal-content
+            .modal-header
+                button.close(ng-click='$hide()') &times;
+                h4.modal-title 
+                    i.fa.fa-clone
+                    | {{ ctrl.title }}
+            form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate)
+                div
+                    .col-sm-2
+                        label.required.labelFormField {{ ctrl.label }}:&nbsp;
+                    .col-sm-10
+                        .input-tip
+                            +ignite-form-field-input('"input-field"', 'ctrl.value', false, 'true', 'Enter value')(
+                                data-ignite-form-field-input-autofocus='true'
+                                ignite-on-enter='form.$valid && ctrl.confirm()'
+                            )
+            .modal-footer
+                button.btn.btn-default(id='copy-btn-cancel' ng-click='$hide()') Cancel
+                button.btn.btn-primary(id='copy-btn-confirm' ng-disabled='ui.inputForm.$invalid' ng-click='ctrl.confirm()') Confirm

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/app/services/Clone.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/Clone.service.js b/modules/web-console/frontend/app/services/Clone.service.js
deleted file mode 100644
index d079141..0000000
--- a/modules/web-console/frontend/app/services/Clone.service.js
+++ /dev/null
@@ -1,66 +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.
- */
-
-import templateUrl from 'views/templates/clone.tpl.pug';
-
-// Service for clone objects.
-export default ['IgniteClone', ['$rootScope', '$q', '$modal', ($root, $q, $modal) => {
-    const scope = $root.$new();
-
-    let _names = [];
-    let deferred;
-    let _validator;
-
-    function _nextAvailableName(name) {
-        let num = 1;
-        let tmpName = name;
-
-        while (_.includes(_names, tmpName)) {
-            tmpName = name + '_' + num.toString();
-
-            num++;
-        }
-
-        return tmpName;
-    }
-
-    const cloneModal = $modal({templateUrl, scope, placement: 'center', show: false});
-
-    scope.ok = function(newName) {
-        if (!_validator || _validator(newName)) {
-            deferred.resolve(_nextAvailableName(newName));
-
-            cloneModal.hide();
-        }
-    };
-
-    cloneModal.confirm = function(oldName, names, validator) {
-        _names = names;
-
-        scope.newName = _nextAvailableName(oldName);
-
-        _validator = validator;
-
-        deferred = $q.defer();
-
-        cloneModal.$promise.then(cloneModal.show);
-
-        return deferred.promise;
-    };
-
-    return cloneModal;
-}]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/controllers/caches-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/caches-controller.js b/modules/web-console/frontend/controllers/caches-controller.js
index b50fde3..d4a13e5 100644
--- a/modules/web-console/frontend/controllers/caches-controller.js
+++ b/modules/web-console/frontend/controllers/caches-controller.js
@@ -19,8 +19,8 @@ import infoMessageTemplateUrl from 'views/templates/message.tpl.pug';
 
 // Controller for Caches screen.
 export default ['cachesController', [
-    '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'IgniteLegacyTable',
-    function($scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, Resource, ErrorPopover, FormUtils, LegacyTable) {
+    '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteInput', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'IgniteLegacyTable',
+    function($scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Confirm, Input, Loading, ModelNormalizer, UnsavedChangesGuard, Resource, ErrorPopover, FormUtils, LegacyTable) {
         UnsavedChangesGuard.install($scope);
 
         const emptyCache = {empty: true};
@@ -517,15 +517,13 @@ export default ['cachesController', [
         };
 
         function _cacheNames() {
-            return _.map($scope.caches, function(cache) {
-                return cache.name;
-            });
+            return _.map($scope.caches, (cache) => cache.name);
         }
 
         // Clone cache with new name.
         $scope.cloneItem = function() {
             if (validate($scope.backupItem)) {
-                Clone.confirm($scope.backupItem.name, _cacheNames()).then(function(newName) {
+                Input.clone($scope.backupItem.name, _cacheNames()).then((newName) => {
                     const item = angular.copy($scope.backupItem);
 
                     delete item._id;

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/controllers/clusters-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js
index 7f90b90..c8392cf 100644
--- a/modules/web-console/frontend/controllers/clusters-controller.js
+++ b/modules/web-console/frontend/controllers/clusters-controller.js
@@ -17,8 +17,8 @@
 
 // Controller for Clusters screen.
 export default ['clustersController', [
-    '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils',
-    function($root, $scope, $http, $state, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, igniteEventGroups, DemoInfo, LegacyTable, Resource, ErrorPopover, FormUtils) {
+    '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteInput', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils',
+    function($root, $scope, $http, $state, $timeout, LegacyUtils, Messages, Confirm, Input, Loading, ModelNormalizer, UnsavedChangesGuard, igniteEventGroups, DemoInfo, LegacyTable, Resource, ErrorPopover, FormUtils) {
         UnsavedChangesGuard.install($scope);
 
         const emptyCluster = {empty: true};
@@ -774,7 +774,7 @@ export default ['clustersController', [
         // Clone cluster with new name.
         $scope.cloneItem = function() {
             if (validate($scope.backupItem)) {
-                Clone.confirm($scope.backupItem.name, _clusterNames()).then(function(newName) {
+                Input.clone($scope.backupItem.name, _clusterNames()).then((newName) => {
                     const item = angular.copy($scope.backupItem);
 
                     delete item._id;

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/controllers/domains-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js
index 5c9e511..806dd45 100644
--- a/modules/web-console/frontend/controllers/domains-controller.js
+++ b/modules/web-console/frontend/controllers/domains-controller.js
@@ -19,8 +19,8 @@ import templateUrl from 'views/configuration/domains-import.tpl.pug';
 
 // Controller for Domain model screen.
 export default ['domainsController', [
-    '$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteAgentMonitor', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'JavaTypes', 'SqlTypes', 'IgniteActivitiesData',
-    function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, IgniteAgentMonitor, LegacyTable, Resource, ErrorPopover, FormUtils, JavaTypes, SqlTypes, ActivitiesData) {
+    '$rootScope', '$scope', '$http', '$state', '$filter', '$timeout', '$modal', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteConfirmBatch', 'IgniteInput', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteAgentMonitor', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils', 'JavaTypes', 'SqlTypes', 'IgniteActivitiesData',
+    function($root, $scope, $http, $state, $filter, $timeout, $modal, LegacyUtils, Messages, Focus, Confirm, ConfirmBatch, Input, Loading, ModelNormalizer, UnsavedChangesGuard, IgniteAgentMonitor, LegacyTable, Resource, ErrorPopover, FormUtils, JavaTypes, SqlTypes, ActivitiesData) {
         UnsavedChangesGuard.install($scope);
 
         const emptyDomain = {empty: true};
@@ -1429,7 +1429,7 @@ export default ['domainsController', [
 
                 item.cacheStoreChanges = [];
 
-                _.forEach(item.caches, function(cacheId) {
+                _.forEach(item.caches, (cacheId) => {
                     const cache = _.find($scope.caches, {value: cacheId}).cache;
 
                     const change = LegacyUtils.autoCacheStoreConfiguration(cache, [item]);
@@ -1444,9 +1444,7 @@ export default ['domainsController', [
         };
 
         function _domainNames() {
-            return _.map($scope.domains, function(domain) {
-                return domain.valueType;
-            });
+            return _.map($scope.domains, (domain) => domain.valueType);
         }
 
         function _newNameIsValidJavaClass(newName) {
@@ -1457,7 +1455,7 @@ export default ['domainsController', [
         // Save domain model with new name.
         $scope.cloneItem = function() {
             if ($scope.tableReset(true) && validate($scope.backupItem)) {
-                Clone.confirm($scope.backupItem.valueType, _domainNames(), _newNameIsValidJavaClass).then(function(newName) {
+                Input.clone($scope.backupItem.valueType, _domainNames(), _newNameIsValidJavaClass).then((newName) => {
                     const item = angular.copy($scope.backupItem);
 
                     delete item._id;

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/controllers/igfs-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/igfs-controller.js b/modules/web-console/frontend/controllers/igfs-controller.js
index b3c6043..504e28d 100644
--- a/modules/web-console/frontend/controllers/igfs-controller.js
+++ b/modules/web-console/frontend/controllers/igfs-controller.js
@@ -17,8 +17,8 @@
 
 // Controller for IGFS screen.
 export default ['igfsController', [
-    '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils',
-    function($scope, $http, $state, $filter, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, LegacyTable, Resource, ErrorPopover, FormUtils) {
+    '$scope', '$http', '$state', '$filter', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteInput', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils',
+    function($scope, $http, $state, $filter, $timeout, LegacyUtils, Messages, Confirm, Input, Loading, ModelNormalizer, UnsavedChangesGuard, LegacyTable, Resource, ErrorPopover, FormUtils) {
         UnsavedChangesGuard.install($scope);
 
         const emptyIgfs = {empty: true};
@@ -328,15 +328,13 @@ export default ['igfsController', [
         };
 
         function _igfsNames() {
-            return _.map($scope.igfss, function(igfs) {
-                return igfs.name;
-            });
+            return _.map($scope.igfss, (igfs) => igfs.name);
         }
 
         // Clone IGFS with new name.
         $scope.cloneItem = function() {
             if ($scope.tableReset(true) && validate($scope.backupItem)) {
-                Clone.confirm($scope.backupItem.name, _igfsNames()).then(function(newName) {
+                Input.clone($scope.backupItem.name, _igfsNames()).then((newName) => {
                     const item = angular.copy($scope.backupItem);
 
                     delete item._id;

http://git-wip-us.apache.org/repos/asf/ignite/blob/83579ce8/modules/web-console/frontend/views/templates/clone.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/clone.tpl.pug b/modules/web-console/frontend/views/templates/clone.tpl.pug
deleted file mode 100644
index e62cb18..0000000
--- a/modules/web-console/frontend/views/templates/clone.tpl.pug
+++ /dev/null
@@ -1,39 +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.
-
-include /app/helpers/jade/mixins
-
-.modal(tabindex='-1' role='dialog')
-    .modal-dialog
-        .modal-content
-            .modal-header
-                button.close(ng-click='$hide()') &times;
-                h4.modal-title 
-                    i.fa.fa-clone
-                    | Clone
-            form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate)
-                div
-                    .col-sm-2
-                        label.required.labelFormField New name:&nbsp;
-                    .col-sm-10
-                        .input-tip
-                            +ignite-form-field-input('"copy-new-name"','newName', false, 'true', 'Enter new name')(
-                                data-ignite-form-field-input-autofocus='true'
-                                ignite-on-enter='form.$valid && ok(newName)'
-                            )
-            .modal-footer
-                button.btn.btn-default(id='copy-btn-cancel' ng-click='$hide()') Cancel
-                button.btn.btn-primary(id='copy-btn-confirm' ng-disabled='ui.inputForm.$invalid' ng-click='ok(newName)') Confirm


[07/14] ignite git commit: IGNITE-4821 Implemented enforce join order option on query tab.

Posted by vo...@apache.org.
IGNITE-4821 Implemented enforce join order option on query tab.


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

Branch: refs/heads/ignite-2.0
Commit: 94c1e7cb505e0a40e1d3c28de5a25b5aefbd55c0
Parents: f361da4
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Wed Mar 15 11:43:23 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Wed Mar 15 11:43:23 2017 +0700

----------------------------------------------------------------------
 modules/web-console/backend/app/agent.js        | 14 ++++++-
 modules/web-console/backend/app/browser.js      |  8 ++--
 .../frontend/app/modules/agent/agent.module.js  | 10 +++--
 .../frontend/app/modules/sql/sql.controller.js  | 25 ++++++++++---
 .../frontend/public/stylesheets/style.scss      |  8 +++-
 .../web-console/frontend/views/sql/sql.tpl.pug  | 39 +++++++++++++-------
 6 files changed, 74 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94c1e7cb/modules/web-console/backend/app/agent.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js
index b4394b7..f65eabb 100644
--- a/modules/web-console/backend/app/agent.js
+++ b/modules/web-console/backend/app/agent.js
@@ -227,17 +227,27 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo,
          * @param {String} cacheName Cache name.
          * @param {String} query Query.
          * @param {Boolean} nonCollocatedJoins Flag whether to execute non collocated joins.
+         * @param {Boolean} enforceJoinOrder Flag whether enforce join order is enabled.
          * @param {Boolean} local Flag whether to execute query locally.
          * @param {int} pageSize Page size.
          * @returns {Promise}
          */
-        fieldsQuery(demo, nid, cacheName, query, nonCollocatedJoins, local, pageSize) {
+        fieldsQuery(demo, nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, local, pageSize) {
             const cmd = new Command(demo, 'exe')
                 .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask')
                 .addParam('p1', nid)
                 .addParam('p2', 'org.apache.ignite.internal.visor.query.VisorQueryTask');
 
-            if (nonCollocatedJoins) {
+            if (enforceJoinOrder) {
+                cmd.addParam('p3', 'org.apache.ignite.internal.visor.query.VisorQueryArgV3')
+                    .addParam('p4', cacheName)
+                    .addParam('p5', query)
+                    .addParam('p6', nonCollocatedJoins)
+                    .addParam('p7', enforceJoinOrder)
+                    .addParam('p8', local)
+                    .addParam('p9', pageSize);
+            }
+            else if (nonCollocatedJoins) {
                 cmd.addParam('p3', 'org.apache.ignite.internal.visor.query.VisorQueryArgV2')
                     .addParam('p4', cacheName)
                     .addParam('p5', query)

http://git-wip-us.apache.org/repos/asf/ignite/blob/94c1e7cb/modules/web-console/backend/app/browser.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/browser.js b/modules/web-console/backend/app/browser.js
index 56f2acc..00ae751 100644
--- a/modules/web-console/backend/app/browser.js
+++ b/modules/web-console/backend/app/browser.js
@@ -98,9 +98,9 @@ module.exports.factory = (_, socketio, agentMgr, configure) => {
                 });
 
                 // Execute query on node and return first page to browser.
-                socket.on('node:query', (nid, cacheName, query, distributedJoins, local, pageSize, cb) => {
+                socket.on('node:query', (nid, cacheName, query, distributedJoins, enforceJoinOrder, local, pageSize, cb) => {
                     agentMgr.findAgent(accountId())
-                        .then((agent) => agent.fieldsQuery(demo, nid, cacheName, query, distributedJoins, local, pageSize))
+                        .then((agent) => agent.fieldsQuery(demo, nid, cacheName, query, distributedJoins, enforceJoinOrder, local, pageSize))
                         .then((res) => cb(null, res))
                         .catch((err) => cb(_errorToJson(err)));
                 });
@@ -114,13 +114,13 @@ module.exports.factory = (_, socketio, agentMgr, configure) => {
                 });
 
                 // Execute query on node and return full result to browser.
-                socket.on('node:query:getAll', (nid, cacheName, query, distributedJoins, local, cb) => {
+                socket.on('node:query:getAll', (nid, cacheName, query, distributedJoins, enforceJoinOrder, local, cb) => {
                     // Set page size for query.
                     const pageSize = 1024;
 
                     agentMgr.findAgent(accountId())
                         .then((agent) => {
-                            const firstPage = agent.fieldsQuery(demo, nid, cacheName, query, distributedJoins, local, pageSize)
+                            const firstPage = agent.fieldsQuery(demo, nid, cacheName, query, distributedJoins, enforceJoinOrder, local, pageSize)
                                 .then(({result}) => {
                                     if (result.key)
                                         return Promise.reject(result.key);

http://git-wip-us.apache.org/repos/asf/ignite/blob/94c1e7cb/modules/web-console/frontend/app/modules/agent/agent.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/agent.module.js b/modules/web-console/frontend/app/modules/agent/agent.module.js
index c0e92d5..724fc6c 100644
--- a/modules/web-console/frontend/app/modules/agent/agent.module.js
+++ b/modules/web-console/frontend/app/modules/agent/agent.module.js
@@ -265,12 +265,13 @@ class IgniteAgentMonitor {
      * @param {String} cacheName Cache name.
      * @param {String} [query] Query if null then scan query.
      * @param {Boolean} nonCollocatedJoins Flag whether to execute non collocated joins.
+     * @param {Boolean} enforceJoinOrder Flag whether enforce join order is enabled.
      * @param {Boolean} local Flag whether to execute query locally.
      * @param {int} pageSize
      * @returns {Promise}
      */
-    query(nid, cacheName, query, nonCollocatedJoins, local, pageSize) {
-        return this._rest('node:query', nid, maskNull(cacheName), maskNull(query), nonCollocatedJoins, local, pageSize)
+    query(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, local, pageSize) {
+        return this._rest('node:query', nid, maskNull(cacheName), maskNull(query), nonCollocatedJoins, enforceJoinOrder, local, pageSize)
             .then(({result}) => {
                 if (_.isEmpty(result.key))
                     return result.value;
@@ -284,11 +285,12 @@ class IgniteAgentMonitor {
      * @param {String} cacheName Cache name.
      * @param {String} [query] Query if null then scan query.
      * @param {Boolean} nonCollocatedJoins Flag whether to execute non collocated joins.
+     * @param {Boolean} enforceJoinOrder Flag whether enforce join order is enabled.
      * @param {Boolean} local Flag whether to execute query locally.
      * @returns {Promise}
      */
-    queryGetAll(nid, cacheName, query, nonCollocatedJoins, local) {
-        return this._rest('node:query:getAll', nid, maskNull(cacheName), maskNull(query), nonCollocatedJoins, local);
+    queryGetAll(nid, cacheName, query, nonCollocatedJoins, enforceJoinOrder, local) {
+        return this._rest('node:query:getAll', nid, maskNull(cacheName), maskNull(query), nonCollocatedJoins, enforceJoinOrder, local);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/94c1e7cb/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index c082d4e..7ded2d5 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -34,6 +34,8 @@ const SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE = 'VISOR_SCAN_CACHE_WITH_FILTER_CASE
 
 const NON_COLLOCATED_JOINS_SINCE = '1.7.0';
 
+const ENFORCE_JOIN_VERS = [['1.7.9', '1.8.0'], ['1.8.4', '1.9.0'], ['1.9.1']];
+
 const _fullColName = (col) => {
     const res = [];
 
@@ -1336,7 +1338,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             agentMonitor.awaitAgent()
                 .then(() => _closeOldQuery(paragraph))
                 .then(() => args.localNid || _chooseNode(args.cacheName, false))
-                .then((nid) => agentMonitor.query(nid, args.cacheName, args.query, args.nonCollocatedJoins, !!args.localNid, args.pageSize))
+                .then((nid) => agentMonitor.query(nid, args.cacheName, args.query, args.nonCollocatedJoins,
+                    args.enforceJoinOrder, !!args.localNid, args.pageSize))
                 .then(_processQueryResult.bind(this, paragraph, false))
                 .catch((err) => paragraph.errMsg = err.message);
         };
@@ -1369,8 +1372,18 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             return false;
         };
 
+        $scope.enforceJoinOrderAvailable = (paragraph) => {
+            const cache = _.find($scope.caches, {name: paragraph.cacheName});
+
+            if (cache)
+                return !!_.find(cache.nodes, (node) => Version.includes(node.version, ...ENFORCE_JOIN_VERS));
+
+            return false;
+        };
+
         $scope.execute = (paragraph, local = false) => {
             const nonCollocatedJoins = !!paragraph.nonCollocatedJoins;
+            const enforceJoinOrder = !!paragraph.enforceJoinOrder;
 
             $scope.actionAvailable(paragraph, true) && _chooseNode(paragraph.cacheName, local)
                 .then((nid) => {
@@ -1390,6 +1403,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                                 pageSize: paragraph.pageSize,
                                 maxPages: paragraph.maxPages,
                                 nonCollocatedJoins,
+                                enforceJoinOrder,
                                 localNid: local ? nid : null
                             };
 
@@ -1397,7 +1411,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
                             ActivitiesData.post({ action: '/queries/execute' });
 
-                            return agentMonitor.query(nid, args.cacheName, qry, nonCollocatedJoins, local, args.pageSize);
+                            return agentMonitor.query(nid, args.cacheName, qry, nonCollocatedJoins, enforceJoinOrder, local, args.pageSize);
                         })
                         .then((res) => {
                             _processQueryResult(paragraph, true, res);
@@ -1450,7 +1464,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
                     ActivitiesData.post({ action: '/queries/explain' });
 
-                    return agentMonitor.query(nid, args.cacheName, args.query, false, false, args.pageSize);
+                    return agentMonitor.query(nid, args.cacheName, args.query, false, !!paragraph.enforceJoinOrder, false, args.pageSize);
                 })
                 .then(_processQueryResult.bind(this, paragraph, true))
                 .catch((err) => {
@@ -1488,7 +1502,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
                             ActivitiesData.post({ action: '/queries/scan' });
 
-                            return agentMonitor.query(nid, args.cacheName, query, false, local, args.pageSize);
+                            return agentMonitor.query(nid, args.cacheName, query, false, false, local, args.pageSize);
                         })
                         .then((res) => _processQueryResult(paragraph, true, res))
                         .catch((err) => {
@@ -1609,7 +1623,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             const args = paragraph.queryArgs;
 
             return Promise.resolve(args.localNid || _chooseNode(args.cacheName, false))
-                .then((nid) => agentMonitor.queryGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.localNid))
+                .then((nid) => agentMonitor.queryGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins,
+                    !!args.enforceJoinOrder, !!args.localNid))
                 .then((res) => _export(paragraph.name + '-all.csv', paragraph.gridOptions.columnDefs, res.columns, res.rows))
                 .catch(Messages.showError)
                 .then(() => paragraph.ace && paragraph.ace.focus());

http://git-wip-us.apache.org/repos/asf/ignite/blob/94c1e7cb/modules/web-console/frontend/public/stylesheets/style.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss
index ab7e3dd..a07472e 100644
--- a/modules/web-console/frontend/public/stylesheets/style.scss
+++ b/modules/web-console/frontend/public/stylesheets/style.scss
@@ -809,7 +809,7 @@ button.form-control {
     }
 
     .btn-group {
-        vertical-align:top;
+        vertical-align: top;
         margin-left: 10px;
 
         i { line-height: 18px; }
@@ -1854,6 +1854,12 @@ th[st-sort] {
     display: inline-block;
 }
 
+.panel-top-align {
+    label {
+        vertical-align: top !important;
+    }
+}
+
 button.dropdown-toggle {
     margin-right: 5px;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/94c1e7cb/modules/web-console/frontend/views/sql/sql.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/sql/sql.tpl.pug b/modules/web-console/frontend/views/sql/sql.tpl.pug
index 931d632..bc134a6 100644
--- a/modules/web-console/frontend/views/sql/sql.tpl.pug
+++ b/modules/web-console/frontend/views/sql/sql.tpl.pug
@@ -91,20 +91,31 @@ mixin paragraph-rename
             input.form-control(id='paragraph-name-{{paragraph.id}}' ng-model='paragraph.editName' required ng-click='$event.stopPropagation();' ignite-on-enter='renameParagraph(paragraph, paragraph.editName)' ignite-on-escape='paragraph.edit = false')
 
 mixin query-settings
-    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Configure periodical execution of last successfully executed query') Refresh rate:
-        button.btn.btn-default.fa.fa-clock-o.tipLabel(ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='{{ $ctrl.paragraphRateTemplateUrl }}' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}}
-
-    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Max number of rows to show in query result as one page') Page size:
-        button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes')
-
-    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Limit query max results to specified number of pages') Max pages:
-        button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.maxPages' bs-select bs-options='item.value as item.label for item in maxPages')
-
-    label.tipLabel(ng-if='nonCollocatedJoinsAvailable(paragraph)' bs-tooltip data-placement='bottom' data-title='Non-collocated joins is a special mode that allow to join data across cluster without collocation.<br/>\
-        Nested joins are not supported for now.<br/>\
-        <b>NOTE</b>: In some cases it may consume more heap memory or may take a long time than collocated joins.' data-trigger='hover')
-        input(type='checkbox' ng-model='paragraph.nonCollocatedJoins')
-        span Allow non-collocated joins
+    .panel-top-align
+        label.tipLabel(bs-tooltip data-placement='bottom' data-title='Configure periodical execution of last successfully executed query') Refresh rate:
+            button.btn.btn-default.fa.fa-clock-o.tipLabel(ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='{{ $ctrl.paragraphRateTemplateUrl }}' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}}
+
+        label.tipLabel(bs-tooltip data-placement='bottom' data-title='Max number of rows to show in query result as one page') Page size:
+            button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes')
+
+        label.tipLabel(bs-tooltip data-placement='bottom' data-title='Limit query max results to specified number of pages') Max pages:
+            button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.maxPages' bs-select bs-options='item.value as item.label for item in maxPages')
+
+        .panel-tip-container
+            .row(ng-if='nonCollocatedJoinsAvailable(paragraph)')
+                label.tipLabel(bs-tooltip data-placement='bottom' data-title='Non-collocated joins is a special mode that allow to join data across cluster without collocation.<br/>\
+                    Nested joins are not supported for now.<br/>\
+                    <b>NOTE</b>: In some cases it may consume more heap memory or may take a long time than collocated joins.' data-trigger='hover')
+                    input(type='checkbox' ng-model='paragraph.nonCollocatedJoins')
+                    span Allow non-collocated joins
+            .row(ng-if='enforceJoinOrderAvailable(paragraph)')
+                label.tipLabel(bs-tooltip data-placement='bottom' data-title='Enforce join order of tables in the query.<br/>\
+                    If <b>set</b> then query optimizer will not reorder tables in join.<br/>\
+                    <b>NOTE:</b> It is not recommended to enable this property until you are sure that\
+                    your indexes and the query itself are correct and tuned as much as possible but\
+                    query optimizer still produces wrong join order.' data-trigger='hover')
+                    input(type='checkbox' ng-model='paragraph.enforceJoinOrder')
+                    span Enforce join order
 
 mixin query-actions
     button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute


[06/14] ignite git commit: IGNITE-933: GridFailFastNodeFailureDetectionSelfTest.testFailFast fails periodically

Posted by vo...@apache.org.
IGNITE-933: GridFailFastNodeFailureDetectionSelfTest.testFailFast fails periodically


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

Branch: refs/heads/ignite-2.0
Commit: f361da4305db72cd1e3784721ebbbc8520382353
Parents: 3da7794
Author: Vadim Opolskij <va...@gmail.com>
Authored: Tue Mar 14 19:03:08 2017 -0700
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Mar 14 19:03:08 2017 -0700

----------------------------------------------------------------------
 .../ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f361da43/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
index 49a3f94..a4886cd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
@@ -113,7 +113,7 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
 
         failNode(ignite1);
 
-        assert failLatch.await(1000, MILLISECONDS);
+        assert failLatch.await(1500, MILLISECONDS);
     }
 
     /**


[10/14] ignite git commit: IGNITE-4830 Implemented better SQL errors handling.

Posted by vo...@apache.org.
IGNITE-4830 Implemented better SQL errors handling.


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

Branch: refs/heads/ignite-2.0
Commit: fbb994081f13091b1442b0f882312341471dcc5a
Parents: 83579ce
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Mar 21 10:20:39 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Mar 21 10:20:39 2017 +0700

----------------------------------------------------------------------
 .../frontend/app/modules/sql/sql.controller.js  | 84 +++++++++++++++-----
 .../web-console/frontend/views/sql/sql.tpl.pug  |  9 ++-
 .../frontend/views/templates/message.tpl.pug    |  4 +-
 3 files changed, 71 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fbb99408/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index 7ded2d5..a3a5a5f 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -18,7 +18,7 @@
 import paragraphRateTemplateUrl from 'views/sql/paragraph-rate.tpl.pug';
 import cacheMetadataTemplateUrl from 'views/sql/cache-metadata.tpl.pug';
 import chartSettingsTemplateUrl from 'views/sql/chart-settings.tpl.pug';
-import showQueryTemplateUrl from 'views/templates/message.tpl.pug';
+import messageTemplateUrl from 'views/templates/message.tpl.pug';
 
 // Time line X axis descriptor.
 const TIME_LINE = {value: -1, type: 'java.sql.Date', label: 'TIME_LINE'};
@@ -53,7 +53,7 @@ const _fullColName = (col) => {
 let paragraphId = 0;
 
 class Paragraph {
-    constructor($animate, $timeout, paragraph) {
+    constructor($animate, $timeout, JavaTypes, paragraph) {
         const self = this;
 
         self.id = 'paragraph-' + paragraphId++;
@@ -140,13 +140,36 @@ class Paragraph {
         }});
 
         Object.defineProperty(this, 'chartHistory', {value: []});
+
+        Object.defineProperty(this, 'error', {value: {
+            root: {},
+            message: ''
+        }});
+
+        this.setError = (err) => {
+            this.error.root = err;
+            this.error.message = err.message;
+
+            let cause = err;
+
+            while (_.nonNil(cause)) {
+                if (_.nonEmpty(cause.className) &&
+                    _.includes(['SQLException', 'JdbcSQLException'], JavaTypes.shortClassName(cause.className))) {
+                    this.error.message = cause.message;
+
+                    break;
+                }
+
+                cause = cause.cause;
+            }
+        };
     }
 
     resultType() {
         if (_.isNil(this.queryArgs))
             return null;
 
-        if (!_.isEmpty(this.errMsg))
+        if (_.nonEmpty(this.error.message))
             return 'error';
 
         if (_.isEmpty(this.rows))
@@ -172,7 +195,7 @@ class Paragraph {
     }
 
     queryExecuted() {
-        return !_.isEmpty(this.meta) || !_.isEmpty(this.errMsg);
+        return _.nonEmpty(this.meta) || _.nonEmpty(this.error.message);
     }
 
     scanExplain() {
@@ -184,17 +207,17 @@ class Paragraph {
     }
 
     chartColumnsConfigured() {
-        return !_.isEmpty(this.chartKeyCols) && !_.isEmpty(this.chartValCols);
+        return _.nonEmpty(this.chartKeyCols) && _.nonEmpty(this.chartValCols);
     }
 
     chartTimeLineEnabled() {
-        return !_.isEmpty(this.chartKeyCols) && _.eq(this.chartKeyCols[0], TIME_LINE);
+        return _.nonEmpty(this.chartKeyCols) && _.eq(this.chartKeyCols[0], TIME_LINE);
     }
 }
 
 // Controller for SQL notebook screen.
-export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion', 'IgniteActivitiesData',
-    function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, Nodes, uiGridExporterConstants, Version, ActivitiesData) {
+export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion', 'IgniteActivitiesData', 'JavaTypes',
+    function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, Nodes, uiGridExporterConstants, Version, ActivitiesData, JavaTypes) {
         const $ctrl = this;
 
         // Define template urls.
@@ -909,7 +932,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     $scope.notebook.paragraphs = [];
 
                 $scope.notebook.paragraphs = _.map($scope.notebook.paragraphs,
-                    (paragraph) => new Paragraph($animate, $timeout, paragraph));
+                    (paragraph) => new Paragraph($animate, $timeout, JavaTypes, paragraph));
 
                 if (_.isEmpty($scope.notebook.paragraphs))
                     $scope.addQuery();
@@ -981,7 +1004,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
             ActivitiesData.post({ action: '/queries/add/query' });
 
-            const paragraph = new Paragraph($animate, $timeout, {
+            const paragraph = new Paragraph($animate, $timeout, JavaTypes, {
                 name: 'Query' + (sz === 0 ? '' : sz),
                 query: '',
                 pageSize: $scope.pageSizes[1],
@@ -1009,7 +1032,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
             ActivitiesData.post({ action: '/queries/add/scan' });
 
-            const paragraph = new Paragraph($animate, $timeout, {
+            const paragraph = new Paragraph($animate, $timeout, JavaTypes, {
                 name: 'Scan' + (sz === 0 ? '' : sz),
                 query: '',
                 pageSize: $scope.pageSizes[1],
@@ -1246,7 +1269,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
             paragraph.resNodeId = res.responseNodeId;
 
-            delete paragraph.errMsg;
+            paragraph.setError({message: ''});
 
             // Prepare explain results for display in table.
             if (paragraph.queryArgs.query && paragraph.queryArgs.query.startsWith('EXPLAIN') && res.rows) {
@@ -1341,7 +1364,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 .then((nid) => agentMonitor.query(nid, args.cacheName, args.query, args.nonCollocatedJoins,
                     args.enforceJoinOrder, !!args.localNid, args.pageSize))
                 .then(_processQueryResult.bind(this, paragraph, false))
-                .catch((err) => paragraph.errMsg = err.message);
+                .catch((err) => paragraph.setError(err));
         };
 
         const _tryStartRefresh = function(paragraph) {
@@ -1419,7 +1442,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                             _tryStartRefresh(paragraph);
                         })
                         .catch((err) => {
-                            paragraph.errMsg = err.message;
+                            paragraph.setError(err);
 
                             _showLoading(paragraph, false);
 
@@ -1468,7 +1491,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 })
                 .then(_processQueryResult.bind(this, paragraph, true))
                 .catch((err) => {
-                    paragraph.errMsg = err.message;
+                    paragraph.setError(err);
 
                     _showLoading(paragraph, false);
                 })
@@ -1506,7 +1529,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                         })
                         .then((res) => _processQueryResult(paragraph, true, res))
                         .catch((err) => {
-                            paragraph.errMsg = err.message;
+                            paragraph.setError(err);
 
                             _showLoading(paragraph, false);
                         });
@@ -1560,7 +1583,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                         delete paragraph.queryId;
                 })
                 .catch((err) => {
-                    paragraph.errMsg = err.message;
+                    paragraph.setError(err);
 
                     _showLoading(paragraph, false);
                 })
@@ -1756,15 +1779,36 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 }
                 else if (paragraph.queryArgs.query .startsWith('EXPLAIN ')) {
                     scope.title = 'Explain query';
-                    scope.content = [paragraph.queryArgs.query];
+                    scope.content = paragraph.queryArgs.query.split(/\r?\n/);
                 }
                 else {
                     scope.title = 'SQL query';
-                    scope.content = [paragraph.queryArgs.query];
+                    scope.content = paragraph.queryArgs.query.split(/\r?\n/);
+                }
+
+                // Show a basic modal from a controller
+                $modal({scope, templateUrl: messageTemplateUrl, placement: 'center', show: true});
+            }
+        };
+
+        $scope.showStackTrace = function(paragraph) {
+            if (!_.isNil(paragraph)) {
+                const scope = $scope.$new();
+
+                scope.title = 'Error details';
+                scope.content = [];
+
+                let cause = paragraph.error.root;
+
+                while (_.nonNil(cause)) {
+                    scope.content.push((scope.content.length > 0 ? '&nbsp;&nbsp;&nbsp;&nbsp;' : '') +
+                        '[' + JavaTypes.shortClassName(cause.className) + '] ' + cause.message);
+
+                    cause = cause.cause;
                 }
 
                 // Show a basic modal from a controller
-                $modal({scope, templateUrl: showQueryTemplateUrl, placement: 'center', show: true});
+                $modal({scope, templateUrl: messageTemplateUrl, placement: 'center', show: true});
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fbb99408/modules/web-console/frontend/views/sql/sql.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/sql/sql.tpl.pug b/modules/web-console/frontend/views/sql/sql.tpl.pug
index bc134a6..701ee14 100644
--- a/modules/web-console/frontend/views/sql/sql.tpl.pug
+++ b/modules/web-console/frontend/views/sql/sql.tpl.pug
@@ -199,7 +199,7 @@ mixin paragraph-scan
                 | Scan on selected node
 
         .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
-            .error(ng-switch-when='error') Error: {{paragraph.errMsg}}
+            .error(ng-switch-when='error') Error: {{paragraph.error.message}}
             .empty(ng-switch-when='empty') Result set is empty
             .table(ng-switch-when='table')
                 +table-result-heading-scan
@@ -247,7 +247,7 @@ mixin paragraph-query
             .pull-right
                 +query-settings
         .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
-            .error(ng-switch-when='error') Error: {{paragraph.errMsg}}
+            .error(ng-switch-when='error') Error: {{paragraph.error.message}}
             .empty(ng-switch-when='empty') Result set is empty
             .table(ng-switch-when='table')
                 +table-result-heading-query
@@ -255,9 +255,10 @@ mixin paragraph-query
             .chart(ng-switch-when='chart')
                 +chart-result
             .footer.clearfix
-                a.pull-left(ng-click='showResultQuery(paragraph)') Show query
+                a.pull-left(ng-show='paragraph.resultType() === "error"' ng-click='showStackTrace(paragraph)') Show error details
+                a.pull-left(ng-show='paragraph.resultType() !== "error"' ng-click='showResultQuery(paragraph)') Show query
 
-                -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())'
+                -var nextVisibleCondition = 'paragraph.resultType() !== "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())'
 
                 .pull-right(ng-show=`${nextVisibleCondition}` ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)')
                     i.fa.fa-chevron-circle-right

http://git-wip-us.apache.org/repos/asf/ignite/blob/fbb99408/modules/web-console/frontend/views/templates/message.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/message.tpl.pug b/modules/web-console/frontend/views/templates/message.tpl.pug
index 6eff74b..aa3615f 100644
--- a/modules/web-console/frontend/views/templates/message.tpl.pug
+++ b/modules/web-console/frontend/views/templates/message.tpl.pug
@@ -22,7 +22,7 @@
                 h4.modal-title
                     i.fa.fa-info-circle
                     | {{title}}
-            .modal-body(ng-show='content')
-                p(ng-bind-html='content.join("<br/>")' style='text-align: left;')
+            .modal-body(ng-show='content' style='overflow: auto; max-height: 300px;')
+                p(ng-bind-html='content.join("<br/>")' style='text-align: left; white-space: nowrap;')
             .modal-footer
                 button.btn.btn-primary(id='confirm-btn-confirm' ng-click='$hide()') Ok


[13/14] ignite git commit: Fixes after merge from master.

Posted by vo...@apache.org.
Fixes after merge from master.


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

Branch: refs/heads/ignite-2.0
Commit: 7e6f214e8404e45c63585f96966136fb1112c382
Parents: 0b6b838
Author: devozerov <vo...@gridgain.com>
Authored: Tue Mar 21 11:10:58 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Mar 21 11:10:58 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  1 -
 .../processors/query/GridQueryProcessor.java    |  3 ---
 .../internal/processors/query/QueryUtils.java   | 20 +++++++++++++++++---
 3 files changed, 17 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7e6f214e/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 d383ce8..a7d38a7 100755
--- 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
@@ -102,7 +102,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManag
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
-import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e6f214e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 47341da..a58ca53 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -31,9 +31,6 @@ import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.binary.BinaryField;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.cache.CacheTypeMetadata;
 import org.apache.ignite.cache.QueryEntity;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e6f214e/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index c033801..20570ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -703,12 +703,26 @@ public class QueryUtils {
             res = buildClassProperty(false, valCls, pathStr, resType, aliases, coCtx);
 
         if (res == null)
-            throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' of type '" +
-                resType.getName() + "' for key class '" + keyCls + "' and value class '" + valCls + "'. " +
-                "Make sure that one of these classes contains respective getter method or field.");
+            throw new IgniteCheckedException(propertyInitializationExceptionMessage(keyCls, valCls, pathStr, resType));
 
         return res;
     }
+
+    /**
+     * Exception message to compare in tests.
+     *
+     * @param keyCls key class
+     * @param valCls value class
+     * @param pathStr property name
+     * @param resType property type
+     * @return Exception message.
+     */
+    public static String propertyInitializationExceptionMessage(Class<?> keyCls, Class<?> valCls, String pathStr,
+        Class<?> resType) {
+        return "Failed to initialize property '" + pathStr + "' of type '" +
+            resType.getName() + "' for key class '" + keyCls + "' and value class '" + valCls + "'. " +
+            "Make sure that one of these classes contains respective getter method or field.";
+    }
     
     /**
      * @param key If this is a key property.


[03/14] ignite git commit: IGNITE-4804: Remove duplicated properties in parent-pom

Posted by vo...@apache.org.
IGNITE-4804: Remove duplicated properties in parent-pom


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

Branch: refs/heads/ignite-2.0
Commit: e6703518f4549a183993a5422fbf75b18f1237de
Parents: 748d1fd
Author: Vyacheslav Daradur <da...@gmail.com>
Authored: Fri Mar 10 11:35:26 2017 -0800
Committer: Denis Magda <dm...@gridgain.com>
Committed: Fri Mar 10 11:35:26 2017 -0800

----------------------------------------------------------------------
 parent/pom.xml | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e6703518/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 32da7ec..5361d15 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -71,7 +71,6 @@
         <guava14.version>14.0.1</guava14.version>
         <guava16.version>16.0.1</guava16.version>
         <h2.version>1.4.191</h2.version>
-        <hadoop.version>2.4.1</hadoop.version>
         <httpclient.version>4.5.1</httpclient.version>
         <httpcore.version>4.4.3</httpcore.version>
         <jackson.version>1.9.13</jackson.version>
@@ -99,7 +98,6 @@
         <scala211.library.version>2.11.7</scala211.library.version>
         <slf4j.version>1.7.7</slf4j.version>
         <slf4j16.version>1.6.4</slf4j16.version>
-        <spring.version>4.1.0.RELEASE</spring.version>
         <spring41.osgi.feature.version>4.1.7.RELEASE_1</spring41.osgi.feature.version>
         <tomcat.version>8.0.23</tomcat.version>
         <twitter.hbc.version>2.2.0</twitter.hbc.version>


[14/14] ignite git commit: Compilation fixes after merge.

Posted by vo...@apache.org.
Compilation fixes after merge.


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

Branch: refs/heads/ignite-2.0
Commit: 92beda22f0bed33035584c79f99a90a1f5d97f46
Parents: 7e6f214
Author: devozerov <vo...@gridgain.com>
Authored: Tue Mar 21 11:15:09 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Mar 21 11:15:09 2017 +0300

----------------------------------------------------------------------
 .../processors/cache/IncorrectCacheTypeMetadataTest.java         | 4 ++--
 .../internal/processors/cache/IncorrectQueryEntityTest.java      | 4 ++--
 2 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/92beda22/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java
index 4178cde..9695ffb 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java
@@ -23,7 +23,7 @@ import java.util.Map;
 import org.apache.ignite.cache.CacheTypeMetadata;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -63,7 +63,7 @@ public class IncorrectCacheTypeMetadataTest extends GridCommonAbstractTest {
         }
         catch (Exception exception) {
             if (!exception.getMessage().contains(
-                GridQueryProcessor.propertyInitializationExceptionMessage(
+                QueryUtils.propertyInitializationExceptionMessage(
                     Object.class, Object.class, "exceptionOid", Object.class))) {
                 fail("property initialization exception must be thrown, but got " + exception.getMessage());
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92beda22/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java
index 5d89a58..9bdadc6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java
@@ -23,7 +23,7 @@ import java.util.Set;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -66,7 +66,7 @@ public class IncorrectQueryEntityTest extends GridCommonAbstractTest {
         }
         catch (Exception exception) {
             if (!exception.getMessage().contains(
-                GridQueryProcessor.propertyInitializationExceptionMessage(
+                QueryUtils.propertyInitializationExceptionMessage(
                     Object.class, Object.class, "exceptionOid", Object.class))) {
                 fail("property initialization exception must be thrown, but got " + exception.getMessage());
             }


[04/14] ignite git commit: IGNITE-1178 fix for NPE in GridCacheProcessor.onKernalStop(). Fixes #1517

Posted by vo...@apache.org.
IGNITE-1178 fix for NPE in GridCacheProcessor.onKernalStop(). Fixes #1517


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

Branch: refs/heads/ignite-2.0
Commit: 0b996e62119b316f9758da09623c722ec2fb7921
Parents: e670351
Author: voipp <al...@gmail.com>
Authored: Mon Mar 13 17:23:07 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Mar 13 17:23:07 2017 +0300

----------------------------------------------------------------------
 .../cache/GridCacheAffinityManager.java         | 21 +++---
 .../cache/GridCacheManagerAdapter.java          |  2 +-
 .../processors/cache/GridCacheProcessor.java    |  2 +-
 .../processors/query/GridQueryProcessor.java    | 20 +++++-
 .../cache/IncorrectCacheTypeMetadataTest.java   | 72 +++++++++++++++++++
 .../cache/IncorrectQueryEntityTest.java         | 75 ++++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |  4 ++
 7 files changed, 183 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b996e62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index 17c9319..621634c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -17,6 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.binary.BinaryObject;
@@ -33,12 +38,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteFuture;
 import org.jetbrains.annotations.Nullable;
 
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
 /**
  * Cache affinity manager.
  */
@@ -88,10 +87,15 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      *
      */
     public void cancelFutures() {
+        if (!starting.get())
+            // Ignoring attempt to stop manager that has never been started.
+            return;
+
         IgniteCheckedException err =
             new IgniteCheckedException("Failed to wait for topology update, cache (or node) is stopping.");
 
-        aff.cancelFutures(err);
+        if (aff != null)
+            aff.cancelFutures(err);
     }
 
     /** {@inheritDoc} */
@@ -99,7 +103,8 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
         IgniteCheckedException err = new IgniteClientDisconnectedCheckedException(reconnectFut,
             "Failed to wait for topology update, client disconnected.");
 
-        aff.cancelFutures(err);
+        if (aff != null)
+            aff.cancelFutures(err);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b996e62/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java
index 8ad0ea8..ab965de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java
@@ -34,7 +34,7 @@ public class GridCacheManagerAdapter<K, V> implements GridCacheManager<K, V> {
     protected IgniteLogger log;
 
     /** Starting flag. */
-    private final AtomicBoolean starting = new AtomicBoolean(false);
+    protected final AtomicBoolean starting = new AtomicBoolean(false);
 
     /** {@inheritDoc} */
     @Override public final void start(GridCacheContext<K, V> cctx) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b996e62/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 50e1379..b016883 100755
--- 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
@@ -63,7 +63,6 @@ import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
 import org.apache.ignite.internal.IgniteComponentType;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -102,6 +101,7 @@ import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProces
 import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b996e62/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 17060fb..fddb8df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1653,14 +1653,28 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             res = buildClassProperty(false, valCls, pathStr, resType, aliases, coCtx);
 
         if (res == null)
-            throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' of type '" +
-                resType.getName() + "' for key class '" + keyCls + "' and value class '" + valCls + "'. " +
-                "Make sure that one of these classes contains respective getter method or field.");
+            throw new IgniteCheckedException(propertyInitializationExceptionMessage(keyCls, valCls, pathStr, resType));
 
         return res;
     }
 
     /**
+     * Exception message to compare in tests.
+     *
+     * @param keyCls key class
+     * @param valCls value class
+     * @param pathStr property name
+     * @param resType property type
+     * @return
+     */
+    public static String propertyInitializationExceptionMessage(Class<?> keyCls, Class<?> valCls, String pathStr,
+        Class<?> resType) {
+        return "Failed to initialize property '" + pathStr + "' of type '" +
+            resType.getName() + "' for key class '" + keyCls + "' and value class '" + valCls + "'. " +
+            "Make sure that one of these classes contains respective getter method or field.";
+    }
+
+    /**
      * @param key If this is a key property.
      * @param cls Source type class.
      * @param pathStr String representing path to the property. May contains dots '.' to identify nested fields.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b996e62/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java
new file mode 100644
index 0000000..4178cde
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectCacheTypeMetadataTest.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.cache.CacheTypeMetadata;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * A test for {@link CacheTypeMetadata} initialization with incorrect query field name
+ */
+public class IncorrectCacheTypeMetadataTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration dfltCacheCfg = defaultCacheConfiguration();
+
+        CacheTypeMetadata cacheTypeMetadata = new CacheTypeMetadata();
+
+        Map<String, Class<?>> queryFieldsMap = new HashMap<>();
+
+        queryFieldsMap.put("exceptionOid", Object.class);
+
+        cacheTypeMetadata.setQueryFields(queryFieldsMap);
+        cacheTypeMetadata.setValueType(Object.class);
+
+        dfltCacheCfg.setTypeMetadata(Collections.singleton(cacheTypeMetadata));
+
+        cfg.setCacheConfiguration(dfltCacheCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Grid must be stopped with property initialization exception.
+     *
+     * @throws Exception If failed.
+     */
+    public void testIncorrectQueryField() throws Exception {
+        try {
+            startGrid();
+        }
+        catch (Exception exception) {
+            if (!exception.getMessage().contains(
+                GridQueryProcessor.propertyInitializationExceptionMessage(
+                    Object.class, Object.class, "exceptionOid", Object.class))) {
+                fail("property initialization exception must be thrown, but got " + exception.getMessage());
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b996e62/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java
new file mode 100644
index 0000000..5d89a58
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IncorrectQueryEntityTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Set;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+public class IncorrectQueryEntityTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration dfltCacheCfg = defaultCacheConfiguration();
+
+        QueryEntity queryEntity = new QueryEntity(Object.class.getName(), Object.class.getName());
+
+        LinkedHashMap<String, String> fields = new LinkedHashMap<>();
+
+        fields.put("exceptionOid", Object.class.getName());
+
+        queryEntity.setFields(fields);
+
+        Set<String> keyFields = new HashSet<>();
+
+        keyFields.add("exceptionOid");
+
+        queryEntity.setKeyFields(keyFields);
+
+        dfltCacheCfg.setQueryEntities(F.asList(queryEntity));
+
+        cfg.setCacheConfiguration(dfltCacheCfg);
+
+        return cfg;
+    }
+
+    /**
+     * Grid must be stopped with property initialization exception.
+     *
+     * @throws Exception If failed.
+     */
+    public void testIncorrectQueryField() throws Exception {
+        try {
+            startGrid();
+        }
+        catch (Exception exception) {
+            if (!exception.getMessage().contains(
+                GridQueryProcessor.propertyInitializationExceptionMessage(
+                    Object.class, Object.class, "exceptionOid", Object.class))) {
+                fail("property initialization exception must be thrown, but got " + exception.getMessage());
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b996e62/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 0c74f12..b7be713 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -68,6 +68,8 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheQueryIndexSelfTest
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryLoadSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCrossCachesJoinsQueryTest;
+import org.apache.ignite.internal.processors.cache.IncorrectCacheTypeMetadataTest;
+import org.apache.ignite.internal.processors.cache.IncorrectQueryEntityTest;
 import org.apache.ignite.internal.processors.cache.QueryEntityCaseMismatchTest;
 import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest;
@@ -133,6 +135,8 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
 
         // Config.
         suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class);
+        suite.addTestSuite(IncorrectCacheTypeMetadataTest.class);
+        suite.addTestSuite(IncorrectQueryEntityTest.class);
 
         // Queries tests.
         suite.addTestSuite(IgniteSqlSplitterSelfTest.class);


[08/14] ignite git commit: Merge ignite-1.7.9 into master

Posted by vo...@apache.org.
Merge ignite-1.7.9 into master


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

Branch: refs/heads/ignite-2.0
Commit: 84880a8108a586604ab00d7bf48ba6c9f8f658ee
Parents: 94c1e7c bcb1398
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu Mar 16 16:34:28 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Mar 16 16:34:36 2017 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   4 +
 .../cache/DynamicCacheChangeBatch.java          |  14 ++
 .../service/GridServiceProcessor.java           |  49 ++---
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   5 +
 .../org/apache/ignite/spi/IgniteSpiContext.java |   6 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  32 +++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  19 ++
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   5 +
 .../tcp/internal/TcpDiscoveryNode.java          |   6 +-
 .../GridServiceContinuousQueryRedeploy.java     | 167 +++++++++++++++++
 ...veryNodeAttributesUpdateOnReconnectTest.java | 110 +++++++++++
 .../TcpDiscoverySslSecuredUnsecuredTest.java    | 185 +++++++++++++++++++
 .../tcp/TestReconnectPluginProvider.java        | 111 +++++++++++
 .../discovery/tcp/TestReconnectProcessor.java   |  93 ++++++++++
 .../testframework/GridSpiTestContext.java       |   5 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   5 +
 .../org.apache.ignite.plugin.PluginProvider     |   1 +
 .../processors/query/h2/IgniteH2Indexing.java   |   1 +
 parent/pom.xml                                  |   1 +
 20 files changed, 793 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 99146aa,4eeafed..74cca8e
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@@ -1586,16 -1576,19 +1576,22 @@@ public class GridServiceProcessor exten
                          if (!((CacheAffinityChangeMessage)msg).exchangeNeeded())
                              return;
                      }
+                     else if (msg instanceof DynamicCacheChangeBatch) {
+                         if (!((DynamicCacheChangeBatch)msg).exchangeNeeded())
+                             return;
+                     }
+                     else
+                         return;
                  }
                  else
 -                    topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
 +                    topVer = new AffinityTopologyVersion((evt).topologyVersion(), 0);
  
 -                depExe.submit(new BusyRunnable() {
 +                depExe.execute(new BusyRunnable() {
                      @Override public void run0() {
 -                        ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer);
 +                        // In case the cache instance isn't tracked by DiscoveryManager anymore.
 +                        discoCache.updateAlives(ctx.discovery());
 +
 +                        ClusterNode oldest = discoCache.oldestAliveServerNodeWithCache();
  
                          if (oldest != null && oldest.isLocal()) {
                              final Collection<GridServiceDeployment> retries = new ConcurrentLinkedQueue<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index 41035ec,5977702..df27868
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@@ -141,7 -144,7 +142,8 @@@ public class IgniteKernalSelfTestSuite 
          suite.addTestSuite(GridServiceProxyClientReconnectSelfTest.class);
          suite.addTestSuite(IgniteServiceReassignmentTest.class);
          suite.addTestSuite(IgniteServiceProxyTimeoutInitializedTest.class);
 +        suite.addTestSuite(IgniteServiceDynamicCachesSelfTest.class);
+         suite.addTestSuite(GridServiceContinuousQueryRedeploy.class);
  
          suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class);
          suite.addTestSuite(IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 46fbb9e,62b47b8..7b188bf
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@@ -83,9 -80,7 +83,10 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
  import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
  import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 +import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 +import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
+ import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
  import org.apache.ignite.internal.processors.query.GridQueryCancel;
  import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
  import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;

http://git-wip-us.apache.org/repos/asf/ignite/blob/84880a81/parent/pom.xml
----------------------------------------------------------------------


[12/14] ignite git commit: Merge branch 'master' into ignite-2.0

Posted by vo...@apache.org.
Merge branch 'master' into ignite-2.0

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java


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

Branch: refs/heads/ignite-2.0
Commit: 0b6b83839086171e06a00d518159ede17cdda3d8
Parents: 99bb87a 92bce6e
Author: devozerov <vo...@gridgain.com>
Authored: Tue Mar 21 11:09:29 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Tue Mar 21 11:09:29 2017 +0300

----------------------------------------------------------------------
 .../cache/eviction/AbstractEvictionPolicy.java  | 203 +++++++++++++++++++
 .../ignite/cache/eviction/EvictionPolicy.java   |   2 -
 .../cache/eviction/fifo/FifoEvictionPolicy.java | 158 +--------------
 .../cache/eviction/lru/LruEvictionPolicy.java   | 159 ++-------------
 .../eviction/random/RandomEvictionPolicy.java   | 124 -----------
 .../random/RandomEvictionPolicyMBean.java       |  42 ----
 .../eviction/sorted/SortedEvictionPolicy.java   | 159 ++-------------
 .../cache/GridCacheAffinityManager.java         |  21 +-
 .../cache/GridCacheManagerAdapter.java          |   2 +-
 .../processors/cache/GridCacheProcessor.java    |   1 +
 .../internal/visor/util/VisorTaskUtils.java     |   8 +-
 .../resources/META-INF/classnames.properties    |   1 -
 ...ridFailFastNodeFailureDetectionSelfTest.java |   2 +-
 ...idCacheConfigurationConsistencySelfTest.java |   6 +-
 .../cache/eviction/EvictionAbstractTest.java    |   4 +-
 .../cache/IncorrectCacheTypeMetadataTest.java   |  72 +++++++
 .../cache/IncorrectQueryEntityTest.java         |  75 +++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   4 +
 modules/web-console/backend/app/agent.js        |  14 +-
 modules/web-console/backend/app/browser.js      |   8 +-
 modules/web-console/frontend/app/app.js         |  10 +-
 .../app/components/input-dialog/index.js        |  24 +++
 .../input-dialog/input-dialog.controller.js     |  35 ++++
 .../input-dialog/input-dialog.service.js        |  88 ++++++++
 .../input-dialog/input-dialog.tpl.pug           |  39 ++++
 .../ui-grid-settings/ui-grid-settings.scss      |  35 ++++
 .../frontend/app/modules/agent/agent.module.js  |  10 +-
 .../frontend/app/modules/sql/sql.controller.js  | 109 +++++++---
 .../frontend/app/services/Clone.service.js      |  66 ------
 .../frontend/controllers/caches-controller.js   |  10 +-
 .../frontend/controllers/clusters-controller.js |   6 +-
 .../frontend/controllers/domains-controller.js  |  12 +-
 .../frontend/controllers/igfs-controller.js     |  10 +-
 .../frontend/public/stylesheets/style.scss      |   8 +-
 .../web-console/frontend/views/sql/sql.tpl.pug  |  48 +++--
 .../frontend/views/templates/clone.tpl.pug      |  39 ----
 .../frontend/views/templates/message.tpl.pug    |   4 +-
 parent/pom.xml                                  |   2 -
 38 files changed, 813 insertions(+), 807 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0b6b8383/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index a7d38a7,b016883..d383ce8
--- 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
@@@ -101,7 -99,9 +101,8 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
  import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
  import org.apache.ignite.internal.processors.plugin.CachePluginManager;
 -import org.apache.ignite.internal.processors.query.GridQueryProcessor;
  import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
+ import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
  import org.apache.ignite.internal.util.F0;
  import org.apache.ignite.internal.util.future.GridCompoundFuture;
  import org.apache.ignite.internal.util.future.GridFinishedFuture;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b6b8383/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b6b8383/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b6b8383/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b6b8383/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0b6b8383/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------


[11/14] ignite git commit: Web Console: minor fix.

Posted by vo...@apache.org.
Web Console: minor fix.


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

Branch: refs/heads/ignite-2.0
Commit: 92bce6ec9cdfa3ed7184b055d07be8efc87dbeb6
Parents: fbb9940
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Tue Mar 21 11:23:18 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Mar 21 11:23:18 2017 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/modules/sql/sql.controller.js | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/92bce6ec/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index a3a5a5f..075bd55 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -154,7 +154,7 @@ class Paragraph {
 
             while (_.nonNil(cause)) {
                 if (_.nonEmpty(cause.className) &&
-                    _.includes(['SQLException', 'JdbcSQLException'], JavaTypes.shortClassName(cause.className))) {
+                    _.includes(['SQLException', 'JdbcSQLException', 'QueryCancelledException'], JavaTypes.shortClassName(cause.className))) {
                     this.error.message = cause.message;
 
                     break;


[02/14] ignite git commit: ignite-2552 Code review

Posted by vo...@apache.org.
ignite-2552 Code review


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

Branch: refs/heads/ignite-2.0
Commit: 748d1fd0a5ff6459f789b71ae1d0e36276965b51
Parents: 19566fb
Author: agura <ag...@apache.org>
Authored: Fri Mar 10 17:29:39 2017 +0300
Committer: agura <ag...@apache.org>
Committed: Fri Mar 10 17:29:39 2017 +0300

----------------------------------------------------------------------
 .../cache/eviction/AbstractEvictionPolicy.java  | 45 +++++++++-----------
 .../cache/eviction/fifo/FifoEvictionPolicy.java | 10 ++---
 .../cache/eviction/lru/LruEvictionPolicy.java   | 10 ++---
 .../eviction/sorted/SortedEvictionPolicy.java   | 17 +++++---
 4 files changed, 40 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/748d1fd0/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java
index eca58e6..cf0aa1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicy.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.cache.eviction;
 
+import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
@@ -24,12 +25,9 @@ import org.apache.ignite.internal.util.typedef.internal.A;
 import org.jsr166.LongAdder8;
 
 /**
- * Common functionality implementation for eviction policies
+ * Common functionality implementation for eviction policies with max size/max memory and batch eviction support.
  */
-public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K, V> {
-    /** Memory size occupied by elements in container. */
-    private final LongAdder8 memSize = new LongAdder8();
-
+public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K, V>, Externalizable {
     /** Max memory size occupied by elements in container. */
     private volatile long maxMemSize;
 
@@ -39,6 +37,9 @@ public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K,
     /** Batch size. */
     private volatile int batchSize = 1;
 
+    /** Memory size occupied by elements in container. */
+    protected final LongAdder8 memSize = new LongAdder8();
+
     /**
      * Shrinks backed container to maximum allowed size.
      */
@@ -48,7 +49,7 @@ public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K,
         if (maxMem > 0) {
             long startMemSize = memSize.longValue();
 
-            if (startMemSize >= maxMem)
+            if (startMemSize >= maxMem) {
                 for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem; ) {
                     int size = shrink0();
 
@@ -57,6 +58,7 @@ public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K,
 
                     i += size;
                 }
+            }
         }
 
         int max = this.max;
@@ -64,10 +66,12 @@ public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K,
         if (max > 0) {
             int startSize = getCurrentSize();
 
-            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
-                for (int i = max; i < startSize && getCurrentSize() > max; i++)
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize)) {
+                for (int i = max; i < startSize && getCurrentSize() > max; i++) {
                     if (shrink0() == -1)
                         break;
+                }
+            }
         }
     }
 
@@ -77,6 +81,7 @@ public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K,
             if (!entry.isCached())
                 return;
 
+            // Shrink only if queue was changed.
             if (touch(entry))
                 shrink();
         }
@@ -86,28 +91,19 @@ public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K,
             if (node != null) {
                 removeMeta(node);
 
-                addToMemorySize(-entry.size());
+                memSize.add(-entry.size());
             }
-
         }
     }
 
     /**
-     * @param x Changing memory size by adding the value.
-     */
-    protected void addToMemorySize(int x) {
-        memSize.add(x);
-    }
-
-    /**
-     *
-     * @return Size of the container.
+     * @return Size of the container with trackable entries.
      */
     protected abstract int getCurrentSize();
 
     /**
      *
-     * @return Size of the memory which was shrinked0.
+     * @return Size of the memory which was shrinked.
      */
     protected abstract int shrink0();
 
@@ -191,18 +187,15 @@ public abstract class AbstractEvictionPolicy<K, V> implements EvictionPolicy<K,
         return batchSize;
     }
 
-    public LongAdder8 getMemSize() {
-        return memSize;
-    }
-
     /** {@inheritDoc} */
-    public void writeExternal(ObjectOutput out) throws IOException {
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(max);
         out.writeInt(batchSize);
         out.writeLong(maxMemSize);
     }
+
     /** {@inheritDoc} */
-    public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         max = in.readInt();
         batchSize = in.readInt();
         maxMemSize = in.readLong();

http://git-wip-us.apache.org/repos/asf/ignite/blob/748d1fd0/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
index dd7ba0f..f95a588 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.cache.eviction.fifo;
 
-import java.io.Externalizable;
 import java.util.Collection;
 import java.util.Collections;
 import org.apache.ignite.cache.eviction.AbstractEvictionPolicy;
@@ -44,7 +43,7 @@ import org.jsr166.ConcurrentLinkedDeque8.Node;
  * table-like data structures. The {@code FIFO} ordering information is
  * maintained by attaching ordering metadata to cache entries.
  */
-public class FifoEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements FifoEvictionPolicyMBean, Externalizable {
+public class FifoEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements FifoEvictionPolicyMBean {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -81,7 +80,7 @@ public class FifoEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imple
 
     /** {@inheritDoc} */
     @Override public int getCurrentSize() {
-        return queue.size();
+        return queue.sizex();
     }
 
     /**
@@ -94,6 +93,7 @@ public class FifoEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imple
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override protected boolean removeMeta(Object meta) {
         return queue.unlinkx((Node<EvictableEntry<K, V>>)meta);
     }
@@ -125,7 +125,7 @@ public class FifoEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imple
                         return false;
                     }
 
-                    addToMemorySize(entry.size());
+                    memSize.add(entry.size());
 
                     return true;
                 }
@@ -157,7 +157,7 @@ public class FifoEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imple
         if (meta != null) {
             size = entry.size();
 
-            addToMemorySize(-size);
+            memSize.add(-size);
 
             if (!entry.evict())
                 touch(entry);

http://git-wip-us.apache.org/repos/asf/ignite/blob/748d1fd0/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
index caa4b3f..17835f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.cache.eviction.lru;
 
-import java.io.Externalizable;
 import java.util.Collection;
 import java.util.Collections;
 import org.apache.ignite.cache.eviction.AbstractEvictionPolicy;
@@ -43,7 +42,7 @@ import org.jsr166.ConcurrentLinkedDeque8.Node;
  * This implementation is very efficient since it is lock-free and does not create any additional table-like
  * data structures. The {@code LRU} ordering information is maintained by attaching ordering metadata to cache entries.
  */
-public class LruEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements LruEvictionPolicyMBean, Externalizable {
+public class LruEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements LruEvictionPolicyMBean {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -69,7 +68,7 @@ public class LruEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implem
 
     /** {@inheritDoc} */
     @Override public int getCurrentSize() {
-        return queue.size();
+        return queue.sizex();
     }
 
     /**
@@ -82,6 +81,7 @@ public class LruEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implem
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override protected boolean removeMeta(Object meta) {
         return queue.unlinkx((Node<EvictableEntry<K, V>>)meta);
     }
@@ -113,7 +113,7 @@ public class LruEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implem
                         return false;
                     }
 
-                    addToMemorySize(entry.size());
+                    memSize.add(entry.size());
 
                     return true;
                 }
@@ -153,7 +153,7 @@ public class LruEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implem
         if (meta != null) {
             size = entry.size();
 
-            addToMemorySize(-size);
+            memSize.add(-size);
 
             if (!entry.evict())
                 touch(entry);

http://git-wip-us.apache.org/repos/asf/ignite/blob/748d1fd0/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
index 47f8479..d0cc975 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.cache.eviction.sorted;
 
-import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
@@ -59,7 +58,7 @@ import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE
  * <p>
  * User defined comparator should implement {@link Serializable} interface.
  */
-public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements SortedEvictionPolicyMBean, Externalizable {
+public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> implements SortedEvictionPolicyMBean {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -108,6 +107,7 @@ public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imp
     public SortedEvictionPolicy(int max, int batchSize, @Nullable Comparator<EvictableEntry<K, V>> comp) {
         setMaxSize(max);
         setBatchSize(batchSize);
+
         this.comp = comp == null ? new DefaultHolderComparator<K, V>() : new HolderComparator<>(comp);
         this.set = new GridConcurrentSkipListSetEx<>(this.comp);
     }
@@ -161,7 +161,7 @@ public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imp
                         return false;
                     }
 
-                    addToMemorySize(entry.size());
+                    memSize.add(entry.size());
 
                     return true;
                 }
@@ -177,7 +177,7 @@ public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imp
 
     /** {@inheritDoc} */
     @Override public int getCurrentSize() {
-        return set.size();
+        return set.sizex();
     }
 
     /**
@@ -195,10 +195,12 @@ public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imp
 
         EvictableEntry<K, V> entry = h.entry;
 
-        if (entry != null && h.order > 0 && entry.removeMeta(h)) {
+        assert entry != null;
+
+        if (h.order > 0 && entry.removeMeta(h)) {
             size = entry.size();
 
-            addToMemorySize(-size);
+            memSize.add(-size);
 
             if (!entry.evict())
                 touch(entry);
@@ -210,6 +212,7 @@ public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imp
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         super.writeExternal(out);
+
         out.writeObject(comp);
     }
 
@@ -217,6 +220,7 @@ public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imp
     @SuppressWarnings("unchecked")
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         super.readExternal(in);
+
         comp = (Comparator<Holder<K, V>>)in.readObject();
     }
 
@@ -225,6 +229,7 @@ public class SortedEvictionPolicy<K, V> extends AbstractEvictionPolicy<K, V> imp
      *
      * @param meta Holder.
      */
+    @SuppressWarnings("unchecked")
     @Override protected boolean removeMeta(Object meta) {
         Holder<K, V> holder = (Holder<K, V>)meta;