Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-41 a24e95cad -> 4ea573688


# ignite-41


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

Branch: refs/heads/ignite-41
Commit: 4ea573688189d2087ded64116e6f3837eec1872e
Parents: a24e95c
Author: sboikov <[email protected]>
Authored: Tue Dec 23 11:14:42 2014 +0300
Committer: sboikov <[email protected]>
Committed: Tue Dec 23 11:14:42 2014 +0300

----------------------------------------------------------------------
 .../grid/cache/GridCacheConfiguration.java      |  31 ----
 .../processors/cache/GridCacheMapEntry.java     | 126 +-------------
 .../distributed/near/GridNearCacheEntry.java    |   8 +-
 .../visor/cache/VisorCacheConfiguration.java    |  18 --
 .../GridCacheRefreshAheadAbstractSelfTest.java  | 163 -------------------
 .../GridCacheColocatedRefreshAheadSelfTest.java |  46 ------
 ...ridCachePartitionedRefreshAheadSelfTest.java |  46 ------
 ...GridCacheReplicatedRefreshAheadSelfTest.java |  43 -----
 .../GridCacheLocalRefreshAheadSelfTest.java     |  43 -----
 .../bamboo/GridDataGridTestSuite.java           |   4 -
 .../commands/cache/VisorCacheCommand.scala      |   1 -
 11 files changed, 5 insertions(+), 524 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
 
b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
index 568ecb5..5a9a675 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheConfiguration.java
@@ -251,9 +251,6 @@ public class GridCacheConfiguration extends 
MutableConfiguration {
     /** Flag indicating if cached values should be additionally stored in 
serialized form. */
     private boolean storeValBytes = DFLT_STORE_VALUE_BYTES;
 
-    /** Refresh-ahead ratio. */
-    private double refreshAheadRatio;
-
     /** Name of class implementing GridCacheTmLookup. */
     private String tmLookupClsName;
 
@@ -381,7 +378,6 @@ public class GridCacheConfiguration extends 
MutableConfiguration {
         preloadTimeout = cc.getPreloadTimeout();
         preloadThrottle = cc.getPreloadThrottle();
         qryCfg = cc.getQueryConfiguration();
-        refreshAheadRatio = cc.getRefreshAheadRatio();
         qryIdxEnabled = cc.isQueryIndexEnabled();
         seqReserveSize = cc.getAtomicSequenceReserveSize();
         startSize = cc.getStartSize();
@@ -970,33 +966,6 @@ public class GridCacheConfiguration extends 
MutableConfiguration {
     }
 
     /**
-     * Gets refresh-ahead ratio. If non-zero, then entry will be preloaded in 
the background
-     * whenever it's accessed and the refresh ratio of it's total time-to-live 
has passed.
-     * This feature ensures that entries are always automatically re-cached 
whenever they are
-     * nearing expiration.
-     * <p>
-     * For example, if refresh ratio is set to {@code 0.75} and entry's 
time-to-live is
-     * {@code 1} minute, then if this entry is accessed any time after {@code 
45} seconds
-     * (which is 0.75 of a minute), the cached value will be immediately 
returned, but
-     * entry will be automatically reloaded from persistent store in the 
background.
-     *
-     * @return Refresh-ahead ratio.
-     */
-    public double getRefreshAheadRatio() {
-        return refreshAheadRatio;
-    }
-
-    /**
-     * Sets refresh-ahead ratio for cache entries. Values other than zero 
specify how soon entries will be auto-reloaded
-     * from persistent store prior to expiration.
-     *
-     * @param refreshAheadRatio Refresh-ahead ratio.
-     */
-    public void setRefreshAheadRatio(double refreshAheadRatio) {
-        this.refreshAheadRatio = refreshAheadRatio;
-    }
-
-    /**
      * Gets class name of transaction manager finder for integration for JEE 
app servers.
      *
      * @return Transaction manager finder.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
index 4bfe91e..0fd64de 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMapEntry.java
@@ -54,13 +54,10 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
 
     /** */
-    private static final byte IS_REFRESHING_MASK = 0x01;
+    private static final byte IS_DELETED_MASK = 0x01;
 
     /** */
-    private static final byte IS_DELETED_MASK = 0x02;
-
-    /** */
-    private static final byte IS_UNSWAPPED_MASK = 0x04;
+    private static final byte IS_UNSWAPPED_MASK = 0x02;
 
     /** */
     private static final Comparator<GridCacheVersion> ATOMIC_VER_COMPARATOR = 
new GridCacheAtomicVersionComparator();
@@ -134,8 +131,8 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
     /**
      * Flags:
      * <ul>
-     *     <li>Refreshing flag - mask {@link #IS_REFRESHING_MASK}</li>
      *     <li>Deleted flag - mask {@link #IS_DELETED_MASK}</li>
+     *     <li>Unswapped flag - mask {@link #IS_UNSWAPPED_MASK}</li>
      * </ul>
      */
     @GridToStringInclude
@@ -603,76 +600,6 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
     }
 
     /**
-     * @param key Key.
-     * @param matchVer Version to match.
-     */
-    protected void refreshAhead(final K key, final GridCacheVersion matchVer) {
-        if (log.isDebugEnabled())
-            log.debug("Scheduling asynchronous refresh for entry: " + this);
-
-        // Asynchronous execution (we don't check filter here).
-        cctx.closures().runLocalSafe(new GPR() {
-            @SuppressWarnings({"unchecked"})
-            @Override public void run() {
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing-ahead entry: " + 
GridCacheMapEntry.this);
-
-                synchronized (GridCacheMapEntry.this){
-                    // If there is a point to refresh.
-                    if (!matchVer.equals(ver)) {
-                        refreshingLocked(false);
-
-                        if (log.isDebugEnabled())
-                            log.debug("Will not refresh value as entry has 
been recently updated: " +
-                                GridCacheMapEntry.this);
-
-                        return;
-                    }
-                }
-
-                V val = null;
-
-                try {
-                    val = cctx.store().loadFromStore(null, key);
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to refresh-ahead entry: " + 
GridCacheMapEntry.this, e);
-                }
-                finally {
-                    synchronized (GridCacheMapEntry.this) {
-                        refreshingLocked(false);
-
-                        // If version matched, set value. Note that we don't 
update
-                        // swap here, as asynchronous refresh happens only if
-                        // value is already in memory.
-                        if (val != null && matchVer.equals(ver)) {
-                            try {
-                                V prev = rawGetOrUnmarshalUnlocked(false);
-
-                                long ttl = ttlExtras();
-
-                                long expTime = toExpireTime(ttl);
-
-                                // Detach value before index update.
-                                if (cctx.portableEnabled())
-                                    val = 
(V)cctx.kernalContext().portable().detachPortable(val);
-
-                                updateIndex(val, null, expTime, ver, prev);
-
-                                // Don't change version for read-through.
-                                update(val, null, expTime, ttl, ver);
-                            }
-                            catch (IgniteCheckedException e) {
-                                U.error(log, "Failed to update cache index: " 
+ GridCacheMapEntry.this, e);
-                            }
-                        }
-                    }
-                }
-            }
-        }, true);
-    }
-
-    /**
      * @param tx Transaction.
      * @param key Key.
      * @param reload flag.
@@ -749,8 +676,6 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
             (new GridCacheFilterEvaluationEntry<>(key, 
rawGetOrUnmarshal(true), this, true)), filter))
             return CU.<V>failed(failFast);
 
-        boolean asyncRefresh = false;
-
         GridCacheVersion startVer;
 
         boolean expired = false;
@@ -769,7 +694,7 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
 
             owner = mvcc == null ? null : mvcc.anyOwner();
 
-            double delta = Double.MAX_VALUE;
+            double delta;
 
             long expireTime = expireTimeExtras();
 
@@ -829,15 +754,6 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
                 }
             }
 
-            // Only calculate asynchronous refresh-ahead, if there is no other
-            // one in progress and if not expired.
-            if (delta > 0 && expireTime > 0 && !refreshingUnlocked()) {
-                double refreshRatio = cctx.config().getRefreshAheadRatio();
-
-                if (1 - delta / ttlExtras() >= refreshRatio)
-                    asyncRefresh = true;
-            }
-
             old = expired || !valid ? null : val;
 
             if (expired) {
@@ -847,8 +763,6 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
             }
 
             if (old == null && !hasOldBytes) {
-                asyncRefresh = false;
-
                 if (updateMetrics)
                     cctx.cache().metrics0().onRead(false);
             }
@@ -858,11 +772,6 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
 
                 // Set retVal here for event notification.
                 ret = old;
-
-                // Mark this entry as refreshing, so other threads won't 
schedule
-                // asynchronous refresh while this one is in progress.
-                if (asyncRefresh || readThrough)
-                    refreshingLocked(true);
             }
 
             if (evt && expired && 
cctx.events().isRecordable(EVT_CACHE_OBJECT_EXPIRED)) {
@@ -894,12 +803,6 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
             }
         }
 
-        if (asyncRefresh && !readThrough && cctx.isStoreEnabled()) {
-            assert ret != null;
-
-            refreshAhead(key, startVer);
-        }
-
         // Check before load.
         if (!cctx.isAll(this, filter))
             return CU.<V>failed(failFast, ret);
@@ -4097,27 +4000,6 @@ public abstract class GridCacheMapEntry<K, V> implements 
GridCacheEntryEx<K, V>
     }
 
     /**
-     * @return {@code True} if refreshing.
-     */
-    protected boolean refreshingUnlocked() {
-        assert Thread.holdsLock(this);
-
-        return (flags & IS_REFRESHING_MASK) != 0;
-    }
-
-    /**
-     * @param refreshing {@code True} if refreshing.
-     */
-    protected void refreshingLocked(boolean refreshing) {
-        assert Thread.holdsLock(this);
-
-        if (refreshing)
-            flags |= IS_REFRESHING_MASK;
-        else
-            flags &= ~IS_REFRESHING_MASK;
-    }
-
-    /**
      * @return Attribute data.
      */
     @Nullable private GridLeanMap<String, Object> attributeDataExtras() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheEntry.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheEntry.java
index b5cc463..aea1214 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -306,11 +306,6 @@ public class GridNearCacheEntry<K, V> extends 
GridDistributedCacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected void refreshAhead(K key, GridCacheVersion matchVer) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override protected V readThrough(IgniteTxEx<K, V> tx, K key, boolean 
reload,
         IgnitePredicate<GridCacheEntry<K, V>>[] filter, UUID subjId, String 
taskName) throws IgniteCheckedException {
         return cctx.near().loadAsync(tx,
@@ -336,6 +331,7 @@ public class GridNearCacheEntry<K, V> extends 
GridDistributedCacheEntry<K, V> {
      * @param expireTime Expiration time.
      * @param evt Event flag.
      * @param topVer Topology version.
+     * @param subjId Subject ID.
      * @return {@code True} if initial value was set.
      * @throws IgniteCheckedException In case of error.
      * @throws GridCacheEntryRemovedException If entry was removed.
@@ -366,8 +362,6 @@ public class GridNearCacheEntry<K, V> extends 
GridDistributedCacheEntry<K, V> {
                 if (isNew() || !valid || expVer == null || 
expVer.equals(this.dhtVer)) {
                     this.primaryNodeId = primaryNodeId;
 
-                    refreshingLocked(false);
-
                     // Change entry only if dht version has changed.
                     if (!dhtVer.equals(dhtVersion())) {
                         update(val, valBytes, expireTime, ttl, ver);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheConfiguration.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheConfiguration.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheConfiguration.java
index cd0c495..806db42 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheConfiguration.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/visor/cache/VisorCacheConfiguration.java
@@ -46,9 +46,6 @@ public class VisorCacheConfiguration implements Serializable {
     /** Eager ttl flag */
     private boolean eagerTtl;
 
-    /** Refresh ahead ratio. */
-    private double refreshAheadRatio;
-
     /** Write synchronization mode. */
     private GridCacheWriteSynchronizationMode writeSynchronizationMode;
 
@@ -144,7 +141,6 @@ public class VisorCacheConfiguration implements 
Serializable {
         cfg.atomicWriteOrderMode(ccfg.getAtomicWriteOrderMode());
         cfg.atomicSequenceReserveSize(ccfg.getAtomicSequenceReserveSize());
         cfg.eagerTtl(ccfg.isEagerTtl());
-        cfg.refreshAheadRatio(ccfg.getRefreshAheadRatio());
         cfg.writeSynchronizationMode(ccfg.getWriteSynchronizationMode());
         cfg.swapEnabled(ccfg.isSwapEnabled());
         cfg.queryIndexEnabled(ccfg.isQueryIndexEnabled());
@@ -289,20 +285,6 @@ public class VisorCacheConfiguration implements 
Serializable {
     }
 
     /**
-     * @return Refresh ahead ratio.
-     */
-    public double refreshAheadRatio() {
-        return refreshAheadRatio;
-    }
-
-    /**
-     * @param refreshAheadRatio New refresh ahead ratio.
-     */
-    public void refreshAheadRatio(double refreshAheadRatio) {
-        this.refreshAheadRatio = refreshAheadRatio;
-    }
-
-    /**
      * @return Write synchronization mode.
      */
     public GridCacheWriteSynchronizationMode writeSynchronizationMode() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheRefreshAheadAbstractSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheRefreshAheadAbstractSelfTest.java
 
b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheRefreshAheadAbstractSelfTest.java
deleted file mode 100644
index 69bb566..0000000
--- 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheRefreshAheadAbstractSelfTest.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.cache;
-
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.transactions.*;
-import org.gridgain.grid.cache.store.*;
-import org.gridgain.grid.util.tostring.*;
-import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.testframework.junits.common.*;
-import org.jetbrains.annotations.*;
-
-import javax.cache.expiry.*;
-import java.util.concurrent.*;
-
-import static java.util.concurrent.TimeUnit.*;
-
-/**
- *
- */
-public abstract class GridCacheRefreshAheadAbstractSelfTest extends 
GridCommonAbstractTest {
-    /** */
-    private static TcpDiscoveryIpFinder ipFinder = new 
TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private final TestStore store = new TestStore();
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) 
throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        return c;
-    }
-
-    /**
-     * @throws IgniteCheckedException If test failed.
-     */
-    public void testReadAhead() throws Exception {
-        store.testThread(Thread.currentThread());
-
-        final ExpiryPolicy expiry = new TouchedExpiryPolicy(new 
Duration(MILLISECONDS, 1000L));
-
-        grid(0).jcache(null).withExpiryPolicy(expiry).put(1, "1");
-
-        Thread.sleep(600);
-
-        store.startAsyncLoadTracking();
-
-        grid(0).jcache(null).get(1);
-
-        assert store.wasAsynchronousLoad() : "No async loads were performed on 
the store: " + store;
-    }
-
-    /**
-     * @return Test store.
-     */
-    protected TestStore testStore() {
-        return store;
-    }
-
-    /**
-     * @return Grid count for test.
-     */
-    protected abstract int gridCount();
-
-    /**
-     * Test cache store.
-     */
-    private class TestStore extends GridCacheStoreAdapter<Object, Object> {
-        /** */
-        private volatile Thread testThread;
-
-        /** */
-        private volatile boolean wasAsyncLoad;
-
-        /** */
-        @GridToStringExclude
-        private final CountDownLatch latch = new CountDownLatch(1);
-
-        /** */
-        private volatile boolean trackLoads;
-
-        /**
-         * @return true if was asynchronous load.
-         * @throws Exception If an error occurs.
-         */
-        public boolean wasAsynchronousLoad() throws Exception {
-            return latch.await(3, SECONDS) && wasAsyncLoad;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object load(IgniteTx tx, Object key) throws 
IgniteCheckedException {
-            if (trackLoads) {
-                wasAsyncLoad = wasAsyncLoad || 
!testThread.equals(Thread.currentThread());
-
-                if (wasAsyncLoad)
-                    latch.countDown();
-
-                info("Load call was tracked on store: " + this);
-            }
-            else
-                info("Load call was not tracked on store: " + this);
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void put(IgniteTx tx, Object key, Object val) throws 
IgniteCheckedException {
-            /* No-op. */
-        }
-
-        /** {@inheritDoc} */
-        @Override public void remove(IgniteTx tx, Object key) throws 
IgniteCheckedException {
-            /* No-op. */
-        }
-
-        /**
-         * @param testThread Thread that runs test.
-         */
-        public void testThread(Thread testThread) {
-            this.testThread = testThread;
-        }
-
-        /**
-         *
-         */
-        public void startAsyncLoadTracking() {
-            trackLoads = true;
-        }
-
-        /** {@inheritDoc} */
-        public String toString() {
-            return S.toString(TestStore.class, this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedRefreshAheadSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedRefreshAheadSelfTest.java
 
b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedRefreshAheadSelfTest.java
deleted file mode 100644
index ef390f4..0000000
--- 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheColocatedRefreshAheadSelfTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.cache.distributed.dht;
-
-import org.apache.ignite.configuration.*;
-import org.gridgain.grid.cache.*;
-import org.gridgain.grid.kernal.processors.cache.*;
-
-import static org.gridgain.grid.cache.GridCacheMode.*;
-import static org.gridgain.grid.cache.GridCacheDistributionMode.*;
-import static org.gridgain.grid.cache.GridCacheWriteSynchronizationMode.*;
-
-/**
- * Test for refresh ahead with near cache disabled.
- */
-public class GridCacheColocatedRefreshAheadSelfTest extends 
GridCacheRefreshAheadAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) 
throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        GridCacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(PARTITIONED);
-        cc.setRefreshAheadRatio(0.5);
-        cc.setDistributionMode(PARTITIONED_ONLY);
-        cc.setStore(testStore());
-        cc.setWriteSynchronizationMode(FULL_ASYNC);
-        cc.setBackups(1);
-
-        c.setCacheConfiguration(cc);
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedRefreshAheadSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedRefreshAheadSelfTest.java
 
b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedRefreshAheadSelfTest.java
deleted file mode 100644
index 72399b8..0000000
--- 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCachePartitionedRefreshAheadSelfTest.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.cache.distributed.dht;
-
-import org.apache.ignite.configuration.*;
-import org.gridgain.grid.cache.*;
-import org.gridgain.grid.kernal.processors.cache.*;
-
-import static org.gridgain.grid.cache.GridCacheMode.*;
-import static org.gridgain.grid.cache.GridCacheWriteSynchronizationMode.*;
-
-/**
- * Refresh ahead test for PARTITIONED cache.
- */
-public class GridCachePartitionedRefreshAheadSelfTest extends 
GridCacheRefreshAheadAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) 
throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        GridCacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(PARTITIONED);
-        cc.setWriteSynchronizationMode(FULL_ASYNC);
-        cc.setBackups(1);
-
-        cc.setRefreshAheadRatio(0.5);
-
-        cc.setStore(testStore());
-
-        c.setCacheConfiguration(cc);
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 2;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedRefreshAheadSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedRefreshAheadSelfTest.java
 
b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedRefreshAheadSelfTest.java
deleted file mode 100644
index 96506dc..0000000
--- 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/replicated/GridCacheReplicatedRefreshAheadSelfTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.cache.distributed.replicated;
-
-import org.apache.ignite.configuration.*;
-import org.gridgain.grid.cache.*;
-import org.gridgain.grid.kernal.processors.cache.*;
-
-import static org.gridgain.grid.cache.GridCacheMode.*;
-
-/**
- * Simple cache test.
- */
-public class GridCacheReplicatedRefreshAheadSelfTest extends 
GridCacheRefreshAheadAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) 
throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        GridCacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(REPLICATED);
-
-        cc.setRefreshAheadRatio(0.5);
-
-        cc.setStore(testStore());
-
-        c.setCacheConfiguration(cc);
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 2;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalRefreshAheadSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalRefreshAheadSelfTest.java
 
b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalRefreshAheadSelfTest.java
deleted file mode 100644
index abf5301..0000000
--- 
a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/local/GridCacheLocalRefreshAheadSelfTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/* @java.file.header */
-
-/*  _________        _____ __________________        _____
- *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
- *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
- *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
- *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
- */
-
-package org.gridgain.grid.kernal.processors.cache.local;
-
-import org.apache.ignite.configuration.*;
-import org.gridgain.grid.cache.*;
-import org.gridgain.grid.cache.eviction.lru.*;
-import org.gridgain.grid.kernal.processors.cache.*;
-
-import static org.gridgain.grid.cache.GridCacheMode.*;
-
-/**
- *
- */
-public class GridCacheLocalRefreshAheadSelfTest extends 
GridCacheRefreshAheadAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) 
throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        GridCacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(LOCAL);
-        cc.setRefreshAheadRatio(0.5);
-        cc.setStore(testStore());
-        cc.setEvictionPolicy(new GridCacheLruEvictionPolicy(1000));
-
-        c.setCacheConfiguration(cc);
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
 
b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
index cb2f5ea..e642753 100644
--- 
a/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
+++ 
b/modules/core/src/test/java/org/gridgain/testsuites/bamboo/GridDataGridTestSuite.java
@@ -105,7 +105,6 @@ public class GridDataGridTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheLocalLoadAllSelfTest.class);
         suite.addTestSuite(GridCacheLocalLockSelfTest.class);
         suite.addTestSuite(GridCacheLocalMultithreadedSelfTest.class);
-        suite.addTestSuite(GridCacheLocalRefreshAheadSelfTest.class);
         suite.addTestSuite(GridCacheLocalTxSingleThreadedSelfTest.class);
         suite.addTestSuite(GridCacheLocalTxTimeoutSelfTest.class);
         suite.addTestSuite(GridCacheLocalEventSelfTest.class);
@@ -144,7 +143,6 @@ public class GridDataGridTestSuite extends TestSuite {
         suite.addTest(new 
TestSuite(GridCachePartitionedMultiThreadedPutGetSelfTest.class));
         suite.addTest(new 
TestSuite(GridCachePartitionedNodeFailureSelfTest.class));
         suite.addTest(new 
TestSuite(GridCachePartitionedExplicitLockNodeFailureSelfTest.class));
-        suite.addTest(new 
TestSuite(GridCachePartitionedRefreshAheadSelfTest.class));
         suite.addTest(new 
TestSuite(GridCachePartitionedTxSingleThreadedSelfTest.class));
         suite.addTest(new 
TestSuite(GridCacheColocatedTxSingleThreadedSelfTest.class));
         suite.addTest(new 
TestSuite(GridCachePartitionedTxTimeoutSelfTest.class));
@@ -175,7 +173,6 @@ public class GridDataGridTestSuite extends TestSuite {
         suite.addTest(new 
TestSuite(GridCacheReplicatedEvictionSelfTest.class));
         suite.addTest(new 
TestSuite(GridCacheDhtEvictionNearReadersSelfTest.class));
         suite.addTest(new 
TestSuite(GridCacheDhtAtomicEvictionNearReadersSelfTest.class));
-        suite.addTest(new 
TestSuite(GridCacheColocatedRefreshAheadSelfTest.class));
 //        suite.addTest(new 
TestSuite(GridCachePartitionedTopologyChangeSelfTest.class)); TODO-gg-5489
         suite.addTest(new 
TestSuite(GridCachePartitionedPreloadEventsSelfTest.class));
         suite.addTest(new 
TestSuite(GridCachePartitionedUnloadEventsSelfTest.class));
@@ -225,7 +222,6 @@ public class GridDataGridTestSuite extends TestSuite {
         //suite.addTestSuite(GridCacheReplicatedMultiNodeSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedNodeFailureSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedProjectionSelfTest.class);
-        suite.addTestSuite(GridCacheReplicatedRefreshAheadSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedTxSingleThreadedSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedTxTimeoutSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedPreloadSelfTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4ea57368/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git 
a/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
 
b/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
index cddc075..28e7e33 100644
--- 
a/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
+++ 
b/modules/visor-console/src/main/scala/org/gridgain/visor/commands/cache/VisorCacheCommand.scala
@@ -807,7 +807,6 @@ object VisorCacheCommand {
         cacheT += ("Time To Live", defaultCfg.timeToLive())
         cacheT += ("Time To Live Eager Flag", cfg.eagerTtl)
 
-        cacheT += ("Refresh Ahead Ratio", cfg.refreshAheadRatio)
         cacheT += ("Write Synchronization Mode", cfg.writeSynchronizationMode)
         cacheT += ("Swap Enabled", cfg.swapEnabled())
         cacheT += ("Invalidate", cfg.invalidate())

Reply via email to