http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4829aab8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java ---------------------------------------------------------------------- diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java index 3957153,0000000..002a462 mode 100644,000000..100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/IgniteCacheQueryNodeRestartSelfTest.java @@@ -1,223 -1,0 +1,223 @@@ +/* + * 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.distributed.near; + +import org.apache.ignite.*; +import org.apache.ignite.cache.*; +import org.apache.ignite.cache.query.*; +import org.apache.ignite.configuration.*; +import org.apache.ignite.events.*; +import org.apache.ignite.internal.*; +import org.apache.ignite.internal.processors.cache.*; +import org.apache.ignite.internal.util.typedef.*; +import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.lang.*; +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 javax.cache.*; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; + +import static org.apache.ignite.cache.CacheAtomicityMode.*; +import static org.apache.ignite.cache.CacheDistributionMode.*; +import static org.apache.ignite.cache.CacheMode.*; + +/** + * Test for distributed queries with node restarts. + */ +public class IgniteCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTest { + /** */ + private static final int GRID_CNT = 3; + + /** */ + private static final int KEY_CNT = 1000; + + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return GRID_CNT; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 90 * 1000; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration c = super.getConfiguration(gridName); + + TcpDiscoverySpi disco = new TcpDiscoverySpi(); + + disco.setIpFinder(ipFinder); + + c.setDiscoverySpi(disco); + + CacheConfiguration cc = defaultCacheConfiguration(); + + cc.setCacheMode(PARTITIONED); + cc.setBackups(1); + cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + cc.setAtomicityMode(TRANSACTIONAL); + cc.setDistributionMode(NEAR_PARTITIONED); + + CacheQueryConfiguration qcfg = new CacheQueryConfiguration(); + + qcfg.setIndexPrimitiveKey(true); + + cc.setQueryConfiguration(qcfg); + + c.setCacheConfiguration(cc); + + return c; + } + + /** + * JUnit. + * + * @throws Exception If failed. + */ + @SuppressWarnings({"TooBroadScope"}) + public void testRestarts() throws Exception { + int duration = 60 * 1000; + int qryThreadNum = 10; + final long nodeLifeTime = 2 * 1000; + final int logFreq = 20; + + final IgniteCache<Integer, Integer> cache = grid(0).jcache(null); + + assert cache != null; + + for (int i = 0; i < KEY_CNT; i++) + cache.put(i, i); + + assertEquals(KEY_CNT, cache.size()); + + final AtomicInteger qryCnt = new AtomicInteger(); + + final AtomicBoolean done = new AtomicBoolean(); + + IgniteInternalFuture<?> fut1 = multithreadedAsync(new CAX() { + @Override public void applyx() throws IgniteCheckedException { + while (!done.get()) { + Collection<Cache.Entry<Integer, Integer>> res = + cache.query(new QuerySqlPredicate(Integer.class, "_val >= 0")).getAll(); + + assertFalse(res.isEmpty()); + + int c = qryCnt.incrementAndGet(); + + if (c % logFreq == 0) + info("Executed queries: " + c); + } + } + }, qryThreadNum); + + final AtomicInteger restartCnt = new AtomicInteger(); + + CollectingEventListener lsnr = new CollectingEventListener(); + + for (int i = 0; i < GRID_CNT; i++) - grid(i).events().localListen(lsnr, IgniteEventType.EVT_CACHE_PRELOAD_STOPPED); ++ grid(i).events().localListen(lsnr, EventType.EVT_CACHE_PRELOAD_STOPPED); + + IgniteInternalFuture<?> fut2 = multithreadedAsync(new Callable<Object>() { + @SuppressWarnings({"BusyWait"}) + @Override public Object call() throws Exception { + while (!done.get()) { + int idx = GRID_CNT; + + startGrid(idx); + + Thread.sleep(nodeLifeTime); + + stopGrid(idx); + + int c = restartCnt.incrementAndGet(); + + if (c % logFreq == 0) + info("Node restarts: " + c); + } + + return true; + } + }, 1); + + Thread.sleep(duration); + + done.set(true); + + fut1.get(); + fut2.get(); + + info("Awaiting preload events [restartCnt=" + restartCnt.get() + ']'); + + boolean success = lsnr.awaitEvents(GRID_CNT * 2 * restartCnt.get(), 15000); + + for (int i = 0; i < GRID_CNT; i++) - grid(i).events().stopLocalListen(lsnr, IgniteEventType.EVT_CACHE_PRELOAD_STOPPED); ++ grid(i).events().stopLocalListen(lsnr, EventType.EVT_CACHE_PRELOAD_STOPPED); + + assert success; + } + + /** Listener that will wait for specified number of events received. */ - private class CollectingEventListener implements IgnitePredicate<IgniteEvent> { ++ private class CollectingEventListener implements IgnitePredicate<Event> { + /** Registered events count. */ + private int evtCnt; + + /** {@inheritDoc} */ - @Override public synchronized boolean apply(IgniteEvent evt) { ++ @Override public synchronized boolean apply(Event evt) { + evtCnt++; + + info("Processed event [evt=" + evt + ", evtCnt=" + evtCnt + ']'); + + notifyAll(); + + return true; + } + + /** + * Waits until total number of events processed is equal or greater then argument passed. + * + * @param cnt Number of events to wait. + * @param timeout Timeout to wait. + * @return {@code True} if successfully waited, {@code false} if timeout happened. + * @throws InterruptedException If thread is interrupted. + */ + public synchronized boolean awaitEvents(int cnt, long timeout) throws InterruptedException { + long start = U.currentTimeMillis(); + + long now = start; + + while (start + timeout > now) { + if (evtCnt >= cnt) + return true; + + wait(start + timeout - now); + + now = U.currentTimeMillis(); + } + + return false; + } + } +}
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4829aab8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java ---------------------------------------------------------------------- diff --cc modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java index 773d49e,0000000..b80de17 mode 100644,000000..100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheReplicatedQuerySelfTest.java @@@ -1,579 -1,0 +1,579 @@@ +/* + * 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.distributed.replicated; + +import org.apache.ignite.*; +import org.apache.ignite.cache.*; +import org.apache.ignite.cache.query.*; +import org.apache.ignite.cache.query.annotations.*; +import org.apache.ignite.events.*; +import org.apache.ignite.internal.*; +import org.apache.ignite.internal.processors.cache.*; +import org.apache.ignite.internal.processors.cache.query.*; +import org.apache.ignite.internal.util.future.*; +import org.apache.ignite.internal.util.lang.*; +import org.apache.ignite.internal.util.typedef.*; +import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.lang.*; +import org.apache.ignite.testframework.*; +import org.apache.ignite.transactions.*; +import org.springframework.util.*; + +import javax.cache.*; +import java.io.*; +import java.lang.reflect.*; +import java.sql.*; +import java.util.*; +import java.util.concurrent.*; + +import static org.apache.ignite.cache.CacheMode.*; - import static org.apache.ignite.events.IgniteEventType.*; ++import static org.apache.ignite.events.EventType.*; + +/** + * Tests replicated query. + */ +public class IgniteCacheReplicatedQuerySelfTest extends IgniteCacheAbstractQuerySelfTest { + /** */ + private static final boolean TEST_DEBUG = false; + + /** Grid1. */ + private static Ignite ignite1; + + /** Grid2. */ + private static Ignite ignite2; + + /** Grid3. */ + private static Ignite ignite3; + + /** Cache1. */ + private static IgniteCache<CacheKey, CacheValue> cache1; + + /** Cache2. */ + private static IgniteCache<CacheKey, CacheValue> cache2; + + /** Cache3. */ + private static IgniteCache<CacheKey, CacheValue> cache3; + + /** Key serialization cnt. */ + private static volatile int keySerCnt; + + /** Key deserialization count. */ + private static volatile int keyDesCnt; + + /** Value serialization count. */ + private static volatile int valSerCnt; + + /** Value deserialization count. */ + private static volatile int valDesCnt; + + /** {@inheritDoc} */ + @Override protected int gridCount() { + return 3; + } + + /** {@inheritDoc} */ + @Override protected CacheMode cacheMode() { + return REPLICATED; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + ignite1 = grid(0); + ignite2 = grid(1); + ignite3 = grid(2); + + cache1 = ignite1.jcache(null); + cache2 = ignite2.jcache(null); + cache3 = ignite3.jcache(null); + } + + /** + * @throws Exception If failed. + */ + public void testClientOnlyNode() throws Exception { + try { + Ignite g = startGrid("client"); + + IgniteCache<Integer, Integer> c = g.jcache(null); + + for (int i = 0; i < 10; i++) + c.put(i, i); + + // Client cache should be empty. + assertEquals(0, c.size()); + + Collection<Cache.Entry<Integer, Integer>> res = + c.query(new QuerySqlPredicate(Integer.class, "_key >= 5 order by _key")).getAll(); + + assertEquals(5, res.size()); + + int i = 5; + + for (Cache.Entry<Integer, Integer> e : res) { + assertEquals(i, e.getKey().intValue()); + assertEquals(i, e.getValue().intValue()); + + i++; + } + } + finally { + stopGrid("client"); + } + } + + /** + * JUnit. + * + * @throws Exception If failed. + */ + public void testIterator() throws Exception { + int keyCnt = 100; + + for (int i = 0; i < keyCnt; i++) + cache1.put(new CacheKey(i), new CacheValue("val" + i)); + + assertEquals(keyCnt, cache1.size()); + assertEquals(keyCnt, cache2.size()); + assertEquals(keyCnt, cache3.size()); + + QueryCursor<Cache.Entry<CacheKey, CacheValue>> qry = + cache1.query(new QuerySqlPredicate(CacheValue.class, "true")); + + Iterator<Cache.Entry<CacheKey, CacheValue>> iter = qry.iterator(); + + assert iter.hasNext(); + + int cnt = 0; + + while (iter.hasNext()) { + iter.next(); + + cnt++; + } + + // Expect duplicates since we run query on full projection of 3 nodes and dedup flag is false. + assertEquals(keyCnt * 3, cnt); + } + + /** + * @throws Exception If test failed. + */ + public void testLocalQuery() throws Exception { + cache1.clear(); + + IgniteTx tx = ignite1.transactions().txStart(); + + try { + cache1.put(new CacheKey(1), new CacheValue("1")); + cache1.put(new CacheKey(2), new CacheValue("2")); + cache1.put(new CacheKey(3), new CacheValue("3")); + cache1.put(new CacheKey(4), new CacheValue("4")); + + tx.commit(); + + info("Committed transaction: " + tx); + } + catch (IgniteException e) { + tx.rollback(); + + throw e; + } + + checkQueryResults(cache1); + checkQueryResults(cache2); + checkQueryResults(cache3); + } + + /** + * @throws Exception If test failed. + */ + public void testDistributedQuery() throws Exception { + int keyCnt = 4; + + final CountDownLatch latch = new CountDownLatch(keyCnt * 2); + - IgnitePredicate<IgniteEvent> lsnr = new IgnitePredicate<IgniteEvent>() { - @Override public boolean apply(IgniteEvent evt) { ++ IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() { ++ @Override public boolean apply(Event evt) { + latch.countDown(); + + return true; + } + }; + - ignite2.events().localListen(lsnr, IgniteEventType.EVT_CACHE_OBJECT_PUT); - ignite3.events().localListen(lsnr, IgniteEventType.EVT_CACHE_OBJECT_PUT); ++ ignite2.events().localListen(lsnr, EventType.EVT_CACHE_OBJECT_PUT); ++ ignite3.events().localListen(lsnr, EventType.EVT_CACHE_OBJECT_PUT); + + IgniteTx tx = ignite1.transactions().txStart(); + + try { + for (int i = 1; i <= keyCnt; i++) + cache1.put(new CacheKey(i), new CacheValue(String.valueOf(i))); + + tx.commit(); + + info("Committed transaction: " + tx); + } + catch (IgniteException e) { + tx.rollback(); + + throw e; + } + + latch.await(); + + QueryCursor<Cache.Entry<CacheKey, CacheValue>> qry = + cache1.query(new QuerySqlPredicate(CacheValue.class, "val > 1 and val < 4")); + + // Distributed query. + assertEquals(6, qry.getAll().size()); + + // Create new query, old query cannot be modified after it has been executed. + qry = cache3.localQuery(new QuerySqlPredicate(CacheValue.class, "val > 1 and val < 4")); + + // Tests execute on node. + Iterator<Cache.Entry<CacheKey, CacheValue>> iter = qry.iterator(); + + assert iter != null; + assert iter.hasNext(); + + iter.next(); + + assert iter.hasNext(); + + iter.next(); + + assert !iter.hasNext(); + } + + /** + * Returns private field {@code qryIters} of {@link GridCacheQueryManager} for the given grid. + * + * @param g Grid which {@link GridCacheQueryManager} should be observed. + * @return {@code qryIters} of {@link GridCacheQueryManager}. + */ + private ConcurrentMap<UUID, + Map<Long, GridFutureAdapter<GridCloseableIterator<IgniteBiTuple<CacheKey, CacheValue>>>>> + distributedQueryManagerQueryItersMap(Ignite g) { + GridCacheContext ctx = ((IgniteKernal)g).internalCache().context(); + + Field qryItersField = ReflectionUtils.findField(ctx.queries().getClass(), "qryIters"); + + qryItersField.setAccessible(true); + + return (ConcurrentMap<UUID, + Map<Long, GridFutureAdapter<GridCloseableIterator<IgniteBiTuple<CacheKey, CacheValue>>>>>) + ReflectionUtils.getField(qryItersField, ctx.queries()); + } + + /** + * @throws Exception If test failed. + */ + public void testToString() throws Exception { + int keyCnt = 4; + + for (int i = 1; i <= keyCnt; i++) + cache1.put(new CacheKey(i), new CacheValue(String.valueOf(i))); + + // Create query with key filter. + + QueryCursor<Cache.Entry<CacheKey, CacheValue>> qry = + cache1.query(new QuerySqlPredicate(CacheValue.class, "val > 0")); + + assertEquals(keyCnt * 3, qry.getAll().size()); + + info("Query result: " + qry.getAll()); + } + + /** + * TODO + * + * @throws Exception If failed. + */ + public void _testLostIterator() throws Exception { + IgniteCache<Integer, Integer> cache = ignite.jcache(null); + + for (int i = 0; i < 1000; i++) + cache.put(i, i); + + QueryCursor<Cache.Entry<Integer, Integer>> fut = null; + + for (int i = 0; i < cache.getConfiguration(CacheConfiguration.class).getMaximumQueryIteratorCount() + 1; i++) { + QueryCursor<Cache.Entry<Integer, Integer>> q = + cache.query(new QuerySqlPredicate(Integer.class, "_key >= 0 order by _key")); + + assertEquals(0, (int)q.iterator().next().getKey()); + + if (fut == null) + fut = q; + } + + final QueryCursor<Cache.Entry<Integer, Integer>> fut0 = fut; + + GridTestUtils.assertThrows(log, new Callable<Object>() { + @Override public Object call() throws Exception { + int i = 0; + + Cache.Entry<Integer, Integer> e; + + while ((e = fut0.iterator().next()) != null) + assertEquals(++i, (int)e.getKey()); + + return null; + } + }, IgniteException.class, null); + } + + /** + * TODO enable + * + * @throws Exception If failed. + */ + public void _testNodeLeft() throws Exception { + try { + Ignite g = startGrid(); + + IgniteCache<Integer, Integer> cache = g.jcache(null); + + for (int i = 0; i < 1000; i++) + cache.put(i, i); + + QueryCursor<Cache.Entry<Integer, Integer>> q = + cache.query(new QuerySqlPredicate(Integer.class, "_key >= 0 order by _key")); + + assertEquals(0, (int) q.iterator().next().getKey()); + + final ConcurrentMap<UUID, Map<Long, GridFutureAdapter<GridCloseableIterator< + IgniteBiTuple<Integer, Integer>>>>> map = + U.field(((IgniteKernal)grid(0)).internalCache().context().queries(), "qryIters"); + + // fut.nextX() does not guarantee the request has completed on remote node + // (we could receive page from local one), so we need to wait. + assertTrue(GridTestUtils.waitForCondition(new PA() { + @Override public boolean apply() { + return map.size() == 1; + } + }, getTestTimeout())); + + Map<Long, GridFutureAdapter<GridCloseableIterator<IgniteBiTuple<Integer, Integer>>>> futs = + map.get(g.cluster().localNode().id()); + + assertEquals(1, futs.size()); + + GridCloseableIterator<IgniteBiTuple<Integer, Integer>> iter = + (GridCloseableIterator<IgniteBiTuple<Integer, Integer>>)((IgniteInternalFuture)F.first(futs.values()).get()).get(); + + ResultSet rs = U.field(iter, "data"); + + assertFalse(rs.isClosed()); + + final UUID nodeId = g.cluster().localNode().id(); + final CountDownLatch latch = new CountDownLatch(1); + - grid(0).events().localListen(new IgnitePredicate<IgniteEvent>() { - @Override public boolean apply(IgniteEvent evt) { - if (((IgniteDiscoveryEvent)evt).eventNode().id().equals(nodeId)) ++ grid(0).events().localListen(new IgnitePredicate<Event>() { ++ @Override public boolean apply(Event evt) { ++ if (((DiscoveryEvent)evt).eventNode().id().equals(nodeId)) + latch.countDown(); + + return true; + } + }, EVT_NODE_LEFT); + + stopGrid(); + + latch.await(); + + assertEquals(0, map.size()); + assertTrue(rs.isClosed()); + } + finally { + // Ensure that additional node is stopped. + stopGrid(); + } + } + + /** + * @param cache Cache. + * @throws Exception If check failed. + */ + private void checkQueryResults(IgniteCache<CacheKey, CacheValue> cache) throws Exception { + QueryCursor<Cache.Entry<CacheKey, CacheValue>> qry = + cache.localQuery(new QuerySqlPredicate(CacheValue.class, "val > 1 and val < 4")); + + Iterator<Cache.Entry<CacheKey, CacheValue>> iter = qry.iterator(); + + assert iter != null; + assert iter.hasNext(); + + Cache.Entry<CacheKey, CacheValue> entry = iter.next(); + + assert entry.getKey().equals(new CacheKey(2)) || entry.getKey().equals(new CacheKey(3)); + + assert iter.hasNext(); + + entry = iter.next(); + + assert entry.getKey().equals(new CacheKey(2)) || entry.getKey().equals(new CacheKey(3)); + assert !iter.hasNext(); + } + + /** + * Cache key. + */ + private static class CacheKey implements Externalizable { + /** Key. */ + private int key; + + /** + * @param key Key. + */ + CacheKey(int key) { + this.key = key; + } + + /** + * + */ + public CacheKey() { + /* No-op. */ + } + + /** + * @return Key. + */ + public int getKey() { + return key; + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + key = in.readInt(); + + keyDesCnt++; + + if (TEST_DEBUG) + X.println("Deserialized demo key [keyDesCnt=" + keyDesCnt + ", key=" + this + ']'); + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeInt(key); + + keySerCnt++; + + if (TEST_DEBUG) + X.println("Serialized demo key [serCnt=" + keySerCnt + ", key=" + this + ']'); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + CacheKey cacheKey; + + if (o instanceof CacheKey) + cacheKey = (CacheKey)o; + else + return false; + + return key == cacheKey.key; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return key; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheKey.class, this); + } + } + + /** + * Cache value.. + */ + private static class CacheValue implements Externalizable { + /** Value. */ + @QuerySqlField + private String val; + + /** + * @param val Value. + */ + CacheValue(String val) { + this.val = val; + } + + /** + * + */ + public CacheValue() { + /* No-op. */ + } + + /** + * @return Value. + */ + public String getValue() { + return val; + } + + /** {@inheritDoc} */ + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + val = U.readString(in); + + valDesCnt++; + + if (TEST_DEBUG) + X.println("Deserialized demo value [valDesCnt=" + valDesCnt + ", val=" + this + ']'); + } + + /** {@inheritDoc} */ + @Override public void writeExternal(ObjectOutput out) throws IOException { + U.writeString(out, val); + + valSerCnt++; + + if (TEST_DEBUG) + X.println("Serialized demo value [serCnt=" + valSerCnt + ", val=" + this + ']'); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + CacheValue val = (CacheValue)o; + + return !(this.val != null ? !this.val.equals(val.val) : val.val != null); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return val != null ? val.hashCode() : 0; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheValue.class, this); + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4829aab8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4829aab8/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java ---------------------------------------------------------------------- diff --cc modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java index 446395b,a2b7ddf..9658582 --- a/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java @@@ -18,7 -18,7 +18,6 @@@ package org.apache.ignite.spi.communication.tcp; import org.apache.ignite.*; - import org.apache.ignite.cache.*; -import org.apache.ignite.cache.query.*; import org.apache.ignite.compute.*; import org.apache.ignite.configuration.*; import org.apache.ignite.internal.*; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4829aab8/modules/scalar/src/main/scala/org/apache/ignite/scalar/scalar.scala ----------------------------------------------------------------------