IGNITE-45 - Fixing tests.

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

Branch: refs/heads/ignite-45
Commit: 5b0778f81857cc23479c388586cd42b7d82c7227
Parents: f69fc79
Author: Alexey Goncharuk <[email protected]>
Authored: Mon Mar 16 20:03:11 2015 -0700
Committer: Alexey Goncharuk <[email protected]>
Committed: Mon Mar 16 20:03:11 2015 -0700

----------------------------------------------------------------------
 .../processors/igfs/IgfsDataManager.java        |   6 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |   5 +-
 .../distributed/GridCacheMixedModeSelfTest.java |   6 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |   5 +-
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |  12 +-
 .../GridCacheStoreValueBytesNode.java           | 131 -------
 .../GridCacheStoreValueBytesTest.java           | 343 -------------------
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |  27 +-
 8 files changed, 40 insertions(+), 495 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5b0778f8/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index b7c2d79..d6db20f 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -150,8 +150,6 @@ public class IgfsDataManager extends IgfsManager {
 
         dataCacheStartLatch = new CountDownLatch(1);
 
-        grpBlockSize = igfsCtx.configuration().getBlockSize() * grpSize;
-
         String igfsName = igfsCtx.configuration().getName();
 
         topic = F.isEmpty(igfsName) ? TOPIC_IGFS : TOPIC_IGFS.topic(igfsName);
@@ -218,6 +216,10 @@ public class IgfsDataManager extends IgfsManager {
         grpSize = mapper instanceof IgfsGroupDataBlocksKeyMapper ?
             ((IgfsGroupDataBlocksKeyMapper)mapper).groupSize() : 1;
 
+        grpBlockSize = igfsCtx.configuration().getBlockSize() * grpSize;
+
+        assert grpBlockSize != 0;
+
         
igfsCtx.kernalContext().cache().internalCache(igfsCtx.configuration().getDataCacheName()).preloader()
             .startFuture().listen(new CI1<IgniteInternalFuture<Object>>() {
             @Override public void apply(IgniteInternalFuture<Object> f) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5b0778f8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
index 0240fef..d7a21ea 100644
--- 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
@@ -90,9 +90,8 @@ public abstract class IgniteCacheAbstractStopBusySelfTest 
extends GridCommonAbst
 
         commSpi.setTcpNoDelay(true);
 
-        // TODO IGNITE-45
-//        if (gridName.endsWith(String.valueOf(CLN_GRD)))
-//            cacheCfg.setDistributionMode(CacheDistributionMode.CLIENT_ONLY);
+        if (gridName.endsWith(String.valueOf(CLN_GRD)))
+            cfg.setClientMode(true);
 
         cacheCfg.setRebalanceMode(CacheRebalanceMode.SYNC);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5b0778f8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
index f544183..93cfe0b 100644
--- 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
@@ -33,6 +33,9 @@ public class GridCacheMixedModeSelfTest extends 
GridCommonAbstractTest {
 
         cfg.setCacheConfiguration(cacheConfiguration(gridName));
 
+        if (F.eq(gridName, getTestGridName(0)))
+            cfg.setClientMode(true);
+
         return cfg;
     }
 
@@ -45,9 +48,6 @@ public class GridCacheMixedModeSelfTest extends 
GridCommonAbstractTest {
 
         cfg.setCacheMode(CacheMode.PARTITIONED);
 
-        if (F.eq(gridName, getTestGridName(0)))
-            return null; // TODO IGNITE-45
-
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5b0778f8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
index 8515ba1..74de525 100644
--- 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
@@ -46,9 +46,8 @@ public class GridCacheRendezvousAffinityClientSelfTest 
extends GridCommonAbstrac
         ccfg.setBackups(1);
         ccfg.setAffinity(new CacheRendezvousAffinityFunction());
 
-        // TODO IGNITE-45.
-//        if (client)
-//            ccfg.setDistributionMode(CLIENT_ONLY);
+        if (client)
+            cfg.setClientMode(true);
 
         cfg.setCacheConfiguration(ccfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5b0778f8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
index 6f51c00..3ed07ff 100644
--- 
a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
+++ 
b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
@@ -19,6 +19,7 @@ package 
org.apache.ignite.internal.processors.cache.eviction.lru;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -63,15 +64,15 @@ public class GridCacheNearOnlyLruNearEvictionPolicySelfTest 
extends GridCommonAb
     @Override protected IgniteConfiguration getConfiguration(String gridName) 
throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 
+        if (cnt == 0)
+            c.setClientMode(true);
+
         CacheConfiguration cc = new CacheConfiguration();
 
         cc.setAtomicityMode(atomicityMode);
         cc.setCacheMode(cacheMode);
         cc.setWriteSynchronizationMode(PRIMARY_SYNC);
         cc.setRebalanceMode(SYNC);
-        // TODO IGNITE-45
-//        cc.setDistributionMode(cnt == 0 ? NEAR_ONLY : PARTITIONED_ONLY);
-//        cc.setNearEvictionPolicy(new 
CacheLruEvictionPolicy(EVICTION_MAX_SIZE));
         cc.setStartSize(100);
         cc.setBackups(0);
 
@@ -135,6 +136,11 @@ public class 
GridCacheNearOnlyLruNearEvictionPolicySelfTest extends GridCommonAb
         startGrids(GRID_COUNT);
 
         try {
+            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+            nearCfg.setNearEvictionPolicy(new 
CacheLruEvictionPolicy(EVICTION_MAX_SIZE));
+
+            grid(0).createCache(nearCfg);
+
             int cnt = 1000;
 
             info("Inserting " + cnt + " keys to cache.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5b0778f8/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesNode.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesNode.java
 
b/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesNode.java
deleted file mode 100644
index a2a15a1..0000000
--- 
a/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesNode.java
+++ /dev/null
@@ -1,131 +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.storevalbytes;
-
-import org.apache.ignite.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.spi.communication.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- *
- */
-public class GridCacheStoreValueBytesNode {
-    /**
-     * @return Discovery SPI.
-     * @throws Exception If failed.
-     */
-    static TcpDiscoverySpi discovery() throws Exception {
-        TcpDiscoverySpi disc = new TcpDiscoverySpi();
-
-        disc.setLocalAddress("localhost");
-
-        TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder();
-
-        Collection<String> addrs = new ArrayList<>();
-
-        for (int i = 0; i < 10; i++)
-            addrs.add("localhost:" + (TcpDiscoverySpi.DFLT_PORT + i));
-
-        ipFinder.setAddresses(addrs);
-
-        disc.setIpFinder(ipFinder);
-
-        return disc;
-    }
-
-    /**
-     * @param size Size.
-     * @return Value.
-     */
-    static String createValue(int size) {
-        StringBuilder str = new StringBuilder();
-
-        str.append(new char[size]);
-
-        return str.toString();
-    }
-
-    /**
-     * @param args Arguments.
-     * @param nearOnly Near only flag.
-     * @return Configuration.
-     * @throws Exception If failed.
-     */
-    static IgniteConfiguration parseConfiguration(String[] args, boolean 
nearOnly) throws Exception {
-        boolean p2pEnabled = false;
-
-        boolean storeValBytes = false;
-
-        for (int i = 0; i < args.length; i++) {
-            String arg = args[i];
-
-            switch (arg) {
-                case "-p2p":
-                    p2pEnabled = Boolean.parseBoolean(args[++i]);
-
-                    break;
-
-                case "-storeValBytes":
-                    storeValBytes = Boolean.parseBoolean(args[++i]);
-
-                    break;
-            }
-        }
-
-        X.println("Peer class loading enabled: " + p2pEnabled);
-        X.println("Store value bytes: " + storeValBytes);
-
-        IgniteConfiguration cfg = new IgniteConfiguration();
-
-        cfg.setDiscoverySpi(discovery());
-
-        cfg.setPeerClassLoadingEnabled(p2pEnabled);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(PARTITIONED);
-
-        cacheCfg.setBackups(1);
-
-        // TODO IGNITE-45.
-//        if (nearOnly) {
-//            cacheCfg.setNearEvictionPolicy(new 
GridCacheAlwaysEvictionPolicy());
-//
-//            cacheCfg.setDistributionMode(NEAR_ONLY);
-//        }
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        return cfg;
-    }
-
-    /**
-     * @param args Arguments.
-     * @throws Exception If failed.
-     */
-    public static void main(String[] args) throws Exception {
-        Ignition.start(parseConfiguration(args, false));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5b0778f8/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java
 
b/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java
deleted file mode 100644
index 3cf96d8..0000000
--- 
a/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java
+++ /dev/null
@@ -1,343 +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.storevalbytes;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.loadtests.util.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.storevalbytes.GridCacheStoreValueBytesNode.*;
-
-/**
- *
- */
-public class GridCacheStoreValueBytesTest {
-    /** */
-    static final int KEYS_NUM = 10000;
-
-    /** */
-    static final Integer[] KEYS = new Integer[KEYS_NUM];
-
-    /** */
-    static final int DFL_MIN_VAL_SIZE = 512;
-
-    /** */
-    static final int DFL_MAX_VAL_SIZE = 1024;
-
-    static {
-        for (int i = 0; i < KEYS_NUM; i++)
-            KEYS[i] = i;
-    }
-
-    /** */
-    private static final int DFLT_THREADS_NUM = 2;
-
-    /** */
-    private static final boolean DFLT_RANDOM_GET = false;
-
-    /** */
-    private static final int DFLT_DURATION_MIN = 3;
-
-    /** */
-    private static final int UPDATE_INTERVAL_SEC = 10;
-
-    /** */
-    private static final int DFLT_WARMUP_TIME_SEC = 10;
-
-    /** */
-    private static final int DFLT_CONCURRENT_GET_NUM = 5000;
-
-    /** */
-    private static final int DFLT_GET_KEY_NUM = 20;
-
-    /**
-     * @param args Args.
-     * @throws Exception If failed.
-     */
-    public static void main(String[] args) throws Exception {
-        boolean randomGet = DFLT_RANDOM_GET;
-
-        int duration = DFLT_DURATION_MIN;
-
-        boolean put = false;
-
-        int warmup = DFLT_WARMUP_TIME_SEC;
-
-        int concurrentGetNum = DFLT_CONCURRENT_GET_NUM;
-
-        int threadsNum = DFLT_THREADS_NUM;
-
-        int getKeyNum = DFLT_GET_KEY_NUM;
-
-        int minSize = DFL_MIN_VAL_SIZE;
-
-        int maxSize = DFL_MAX_VAL_SIZE;
-
-        for (int i = 0; i < args.length; i++) {
-            String arg = args[i];
-
-            switch (arg) {
-                case "-c":
-                    concurrentGetNum = Integer.parseInt(args[++i]);
-
-                    break;
-
-                case "-t":
-                    threadsNum = Integer.parseInt(args[++i]);
-
-                    break;
-
-                case "-k":
-                    getKeyNum = Integer.parseInt(args[++i]);
-
-                    break;
-
-                case "-randomGet":
-                    randomGet = Boolean.parseBoolean(args[++i]);
-
-                    break;
-
-                case "-d":
-                    duration = Integer.parseInt(args[++i]);
-
-                    break;
-
-                case "-w":
-                    warmup = Integer.parseInt(args[++i]);
-
-                    break;
-
-                case "-put":
-                    put = Boolean.parseBoolean(args[++i]);
-
-                    break;
-
-                case "-min":
-                    minSize = Integer.parseInt(args[++i]);
-
-                    break;
-
-                case "-max":
-                    maxSize = Integer.parseInt(args[++i]);
-
-                    break;
-            }
-        }
-
-        X.println("Duration: " + duration + " minutes");
-        X.println("Warmup time: " + warmup + " seconds");
-        X.println("Threads number: " + threadsNum);
-        X.println("Concurrent get number: " + concurrentGetNum);
-        X.println("Get keys number: " + getKeyNum);
-        X.println("Random get: " + randomGet);
-
-        Ignite ignite = 
Ignition.start(GridCacheStoreValueBytesNode.parseConfiguration(args, true));
-
-        if (put) {
-            X.println("Putting data in cache...");
-            X.println("Min value size: " + minSize);
-            X.println("Max value size: " + maxSize);
-
-            Random random = new Random(1);
-
-            int sizeRange = maxSize - minSize;
-
-            IgniteCache<Integer, String> cache = ignite.jcache(null);
-
-            if (sizeRange == 0) {
-                for (Integer key : KEYS)
-                    cache.put(key, createValue(minSize));
-            }
-            else {
-                for (Integer key : KEYS)
-                    cache.put(key, createValue(minSize + 
random.nextInt(sizeRange)));
-            }
-        }
-
-        try {
-            runTest(ignite, concurrentGetNum, threadsNum, getKeyNum, duration 
* 60000, warmup * 1000, randomGet);
-        }
-        finally {
-            G.stopAll(true);
-        }
-    }
-
-    /**
-     * @param exec Pool.
-     * @param ignite Grid.
-     * @param concurrentGetNum Concurrent GET operations.
-     * @param threadsNum Thread count.
-     * @param getKeyNum Keys count.
-     * @param finish Finish flag.
-     * @param cntr Counter.
-     * @param randomGet {@code True} to get random keys.
-     * @return Futures.
-     */
-    static Collection<Future<?>> startThreads(ExecutorService exec, final 
Ignite ignite, int concurrentGetNum,
-        int threadsNum, final int getKeyNum, final AtomicBoolean finish, final 
AtomicLong cntr,
-        final boolean randomGet) {
-
-        final Semaphore sem = new Semaphore(concurrentGetNum);
-
-        final IgniteInClosure<Object> lsnr = new CI1<Object>() {
-            @Override public void apply(Object t) {
-                sem.release();
-            }
-        };
-
-        finish.set(false);
-
-        cntr.set(0);
-
-        Collection<Future<?>> futs = new ArrayList<>(threadsNum);
-
-        for (int i = 0; i < threadsNum; i++) {
-            futs.add(exec.submit(new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    IgniteCache<Integer, String> cache = ignite.jcache(null);
-
-                    Random random = new Random();
-
-                    while (!finish.get()) {
-                        Set<Integer> keys = new TreeSet<>();
-
-                        for (int i = 0; i < KEYS_NUM; i++) {
-                            Integer key = KEYS[randomGet ? 
random.nextInt(KEYS_NUM) : i];
-
-                            keys.add(key);
-
-                            if (keys.size() == getKeyNum) {
-                                sem.acquire();
-
-                                IgniteCache<Integer, String> asyncCache = 
cache.withAsync();
-
-                                asyncCache.getAll(keys);
-
-                                IgniteFuture<Object> f = asyncCache.future();
-
-                                f.listen(lsnr);
-
-                                cntr.incrementAndGet();
-
-                                keys.clear();
-                            }
-                        }
-                    }
-
-                    return null;
-                }
-            }));
-        }
-
-        return futs;
-    }
-
-    /**
-     * @param ignite Grid.
-     * @param concurrentGetNum Number of concurrent getAllAsync operations.
-     * @param threadsNum Thread count.
-     * @param getKeyNum Keys count.
-     * @param duration Test duration.
-     * @param warmup Warmup duration.
-     * @param randomGet If {@code true} then selects keys randomly, otherwise 
selects keys sequentially.
-     * @throws Exception If failed.
-     */
-    static void runTest(final Ignite ignite, int concurrentGetNum, int 
threadsNum, int getKeyNum, final long duration,
-        long warmup, final boolean randomGet) throws Exception {
-        ExecutorService exec = Executors.newFixedThreadPool(threadsNum);
-
-        try {
-            final AtomicBoolean finish = new AtomicBoolean();
-
-            final AtomicLong cntr = new AtomicLong();
-
-            X.println("Warming up...");
-
-            Collection<Future<?>> futs = startThreads(exec, ignite, 
concurrentGetNum, threadsNum, getKeyNum, finish,
-                cntr, randomGet);
-
-            U.sleep(warmup);
-
-            finish.set(true);
-
-            boolean failed = false;
-
-            for (Future<?> fut : futs) {
-                try {
-                    fut.get();
-                }
-                catch (ExecutionException e) {
-                    X.error("Error during warmup: " + e);
-
-                    e.getCause().printStackTrace();
-
-                    failed = true;
-                }
-            }
-
-            if (failed)
-                return;
-
-            X.println("Running test...");
-
-            futs = startThreads(exec, ignite, concurrentGetNum, threadsNum, 
getKeyNum, finish, cntr, randomGet);
-
-            long end = System.currentTimeMillis() + duration;
-
-            GridCumulativeAverage avgGetPerSec = new GridCumulativeAverage();
-
-            while (System.currentTimeMillis() < end) {
-                long c1 = cntr.get();
-
-                U.sleep(UPDATE_INTERVAL_SEC * 1000);
-
-                long c2 = cntr.get();
-
-                long getPerSec = (c2 - c1) / UPDATE_INTERVAL_SEC;
-
-                X.println(">>> Gets/s: " + getPerSec);
-
-                avgGetPerSec.update(getPerSec);
-            }
-
-            finish.set(true);
-
-            for (Future<?> fut : futs) {
-                try {
-                    fut.get();
-                }
-                catch (ExecutionException e) {
-                    X.error("Error during execution: " + e);
-
-                    e.getCause().printStackTrace();
-                }
-            }
-
-            X.println(">>> Average gets/s: " + avgGetPerSec);
-        }
-        finally {
-            exec.shutdown();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5b0778f8/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
 
b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
index ec8b0c9..84f70be 100644
--- 
a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
+++ 
b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
@@ -58,6 +58,18 @@ public class IgfsNearOnlyMultiNodeSelfTest extends 
GridCommonAbstractTest {
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         startGrids(nodeCount());
+
+        NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+        nearCfg.setName("data");
+
+        grid(0).createCache(nearCfg);
+
+        nearCfg = new NearCacheConfiguration();
+
+        nearCfg.setName("meta");
+
+        grid(0).createCache(nearCfg);
     }
 
     /** {@inheritDoc} */
@@ -76,8 +88,8 @@ public class IgfsNearOnlyMultiNodeSelfTest extends 
GridCommonAbstractTest {
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
-        igfsCfg.setDataCacheName("partitioned");
-        igfsCfg.setMetaCacheName("partitioned");
+        igfsCfg.setDataCacheName("data");
+        igfsCfg.setMetaCacheName("meta");
         igfsCfg.setName("igfs");
 
         IgfsIpcEndpointConfiguration endpointCfg = new 
IgfsIpcEndpointConfiguration();
@@ -91,10 +103,13 @@ public class IgfsNearOnlyMultiNodeSelfTest extends 
GridCommonAbstractTest {
 
         cfg.setFileSystemConfiguration(igfsCfg);
 
-        cfg.setCacheConfiguration(cacheConfiguration(gridName));
+        cfg.setCacheConfiguration(cacheConfiguration(gridName, "data"), 
cacheConfiguration(gridName, "meta"));
 
         cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, 
EVT_JOB_MAPPED);
 
+        if (cnt == 0)
+            cfg.setClientMode(true);
+
         cnt++;
 
         return cfg;
@@ -111,13 +126,11 @@ public class IgfsNearOnlyMultiNodeSelfTest extends 
GridCommonAbstractTest {
      * @param gridName Grid name.
      * @return Cache configuration.
      */
-    protected CacheConfiguration cacheConfiguration(String gridName) {
+    protected CacheConfiguration cacheConfiguration(String gridName, String 
cacheName) {
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
 
-        cacheCfg.setName("partitioned");
+        cacheCfg.setName(cacheName);
         cacheCfg.setCacheMode(PARTITIONED);
-        // TODO IGNITE-45.
-//        cacheCfg.setDistributionMode(cnt == 0 ? NEAR_ONLY : 
PARTITIONED_ONLY);
         
cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(GRP_SIZE));
         cacheCfg.setBackups(0);

Reply via email to