Repository: ignite
Updated Branches:
  refs/heads/ignite-2.7 42ab0ebe3 -> 0ccde7c42


http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
index 140eb6e..d5cc0eb 100644
--- 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
@@ -17,21 +17,13 @@
 
 package org.apache.ignite.internal.processors.query;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import 
org.apache.ignite.internal.processors.query.h2.twostep.MapQueryLazyWorker;
@@ -39,11 +31,16 @@ import 
org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
+
 /**
  * Tests for lazy query execution.
  */
 public class LazyQuerySelfTest extends GridCommonAbstractTest {
-    /** Keys count. */
+    /** Keys ocunt. */
     private static final int KEY_CNT = 200;
 
     /** Base query argument. */
@@ -97,91 +94,6 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
     }
 
     /**
-     * Test DDL operation on table with high load queries.
-     *
-     * @throws Exception If failed.
-     */
-    public void testTableWriteLockStarvation() throws Exception {
-        final Ignite srv = startGrid(1);
-
-        srv.createCache(cacheConfiguration(4));
-
-        populateBaseQueryData(srv);
-
-        final AtomicBoolean end = new AtomicBoolean(false);
-
-        final int qryThreads = 10;
-
-        final CountDownLatch latch = new CountDownLatch(qryThreads);
-
-        // Do many concurrent queries.
-        IgniteInternalFuture<Long> fut = 
GridTestUtils.runMultiThreadedAsync(new Runnable() {
-            @Override public void run() {
-                latch.countDown();
-
-                while(!end.get()) {
-                    FieldsQueryCursor<List<?>> cursor = execute(srv, query(0)
-                        .setPageSize(PAGE_SIZE_SMALL));
-
-                    cursor.getAll();
-                }
-            }
-        }, qryThreads, "usr-qry");
-
-        latch.await();
-
-        Thread.sleep(500);
-
-        execute(srv, new SqlFieldsQuery("CREATE INDEX PERSON_NAME ON Person 
(name asc)")).getAll();
-        execute(srv, new SqlFieldsQuery("DROP INDEX PERSON_NAME")).getAll();
-
-        // Test is OK in case DDL operations is passed on hi load queries 
pressure.
-        end.set(true);
-        fut.get();
-    }
-
-    /**
-     * Test release reserved partition after query complete (results is bigger 
than one page).
-     *
-     * @throws Exception If failed.
-     */
-    public void testReleasePartitionReservationSeveralPagesResults() throws 
Exception {
-        checkReleasePartitionReservation(PAGE_SIZE_SMALL);
-    }
-
-    /**
-     * Test release reserved partition after query complete (results is placed 
on one page).
-     *
-     * @throws Exception If failed.
-     */
-    public void testReleasePartitionReservationOnePageResults() throws 
Exception {
-        checkReleasePartitionReservation(KEY_CNT);
-    }
-
-    /**
-     * Test release reserved partition after query complete.
-     *
-     * @param pageSize Results page size.
-     * @throws Exception If failed.
-     */
-    public void checkReleasePartitionReservation(int pageSize) throws 
Exception {
-        Ignite srv1 = startGrid(1);
-        startGrid(2);
-
-        srv1.createCache(cacheConfiguration(1));
-
-        populateBaseQueryData(srv1);
-
-        FieldsQueryCursor<List<?>> cursor = execute(srv1, 
query(0).setPageSize(pageSize));
-
-        cursor.getAll();
-
-        startGrid(3);
-
-        awaitPartitionMapExchange();
-    }
-
-    /**
      * Check local query execution.
      *
      * @param parallelism Query parallelism.
@@ -239,18 +151,18 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
         assertNoWorkers();
 
         // Test server node leave with active worker.
-        FieldsQueryCursor<List<?>> cursor2 = execute(srv1, 
baseQuery().setPageSize(PAGE_SIZE_SMALL));
+        cursor = execute(srv1, baseQuery().setPageSize(PAGE_SIZE_SMALL));
 
         try {
-            Iterator<List<?>> iter2 = cursor2.iterator();
+            iter = cursor.iterator();
 
             for (int i = 0; i < 30; i++)
-                iter2.next();
+                iter.next();
 
             stopGrid(2);
         }
         finally {
-            cursor2.close();
+            cursor.close();
         }
 
         assertNoWorkers();
@@ -321,55 +233,7 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
             }
         }
 
-        checkHoldLazyQuery(node);
-
-        checkShortLazyQuery(node);
-    }
-
-    /**
-     * @param node Ignite node.
-     * @throws Exception If failed.
-     */
-    public void checkHoldLazyQuery(Ignite node) throws Exception {
-        ArrayList rows = new ArrayList<>();
-
-        FieldsQueryCursor<List<?>> cursor0 = execute(node, 
query(BASE_QRY_ARG).setPageSize(PAGE_SIZE_SMALL));
-
-        // Do many concurrent queries to Test full iteration.
-        GridTestUtils.runMultiThreaded(new Runnable() {
-            @Override public void run() {
-                for (int i = 0; i < 5; ++i) {
-                    FieldsQueryCursor<List<?>> cursor = execute(node, 
query(KEY_CNT - PAGE_SIZE_SMALL + 1)
-                        .setPageSize(PAGE_SIZE_SMALL));
-
-                    cursor.getAll();
-                }
-            }
-        }, 5, "usr-qry");
-
-        for (List<?> row : cursor0)
-            rows.add(row);
-
-        assertBaseQueryResults(rows);
-    }
-
-    /**
-     * @param node Ignite node.
-     * @throws Exception If failed.
-     */
-    public void checkShortLazyQuery(Ignite node) throws Exception {
-        ArrayList rows = new ArrayList<>();
-
-        FieldsQueryCursor<List<?>> cursor0 = execute(node, query(KEY_CNT - 
PAGE_SIZE_SMALL + 1).setPageSize(PAGE_SIZE_SMALL));
-
-        Iterator<List<?>> it = cursor0.iterator();
-
         assertNoWorkers();
-
-        while (it.hasNext())
-            rows.add(it.next());
-
-        assertQueryResults(rows, KEY_CNT - PAGE_SIZE_SMALL + 1);
     }
 
     /**
@@ -403,11 +267,8 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
      * @return Default cache configuration.
      */
     private static CacheConfiguration<Long, Person> cacheConfiguration(int 
parallelism) {
-        return new CacheConfiguration<Long, Person>()
-            .setName(CACHE_NAME)
-            .setIndexedTypes(Long.class, Person.class)
-            .setQueryParallelism(parallelism)
-            .setAffinity(new RendezvousAffinityFunction(false, 10));
+        return new CacheConfiguration<Long, 
Person>().setName(CACHE_NAME).setIndexedTypes(Long.class, Person.class)
+            .setQueryParallelism(parallelism);
     }
 
     /**
@@ -417,7 +278,7 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
      * @return Query.
      */
     private static SqlFieldsQuery query(long arg) {
-        return new SqlFieldsQuery("SELECT id, name FROM Person WHERE id >= " + 
arg);
+        return new SqlFieldsQuery("SELECT id, name FROM Person WHERE id >= 
?").setArgs(arg);
     }
 
     /**
@@ -426,23 +287,13 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
      * @param rows Result rows.
      */
     private static void assertBaseQueryResults(List<List<?>> rows) {
-        assertQueryResults(rows, BASE_QRY_ARG);
-    }
-
-    /**
-     * Assert base query results.
-     *
-     * @param rows Result rows.
-     * @param resSize Result size.
-     */
-    private static void assertQueryResults(List<List<?>> rows, int resSize) {
-        assertEquals(KEY_CNT - resSize, rows.size());
+        assertEquals(KEY_CNT - BASE_QRY_ARG, rows.size());
 
         for (List<?> row : rows) {
             Long id = (Long)row.get(0);
             String name = (String)row.get(1);
 
-            assertTrue(id >= resSize);
+            assertTrue(id >= BASE_QRY_ARG);
             assertEquals(nameForId(id), name);
         }
     }
@@ -466,7 +317,7 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
      */
     @SuppressWarnings("unchecked")
     private static FieldsQueryCursor<List<?>> execute(Ignite node, 
SqlFieldsQuery qry) {
-        return cache(node).query(qry);
+        return cache(node).query(qry.setLazy(true));
     }
 
     /**
@@ -474,8 +325,8 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
      *
      * @throws Exception If failed.
      */
-    private void assertNoWorkers() throws Exception {
-        if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
+    private static void assertNoWorkers() throws Exception {
+        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
                 for (Ignite node : Ignition.allGrids()) {
                     IgniteH2Indexing idx = (IgniteH2Indexing) 
((IgniteKernal)node).context().query().getIndexing();
@@ -486,22 +337,7 @@ public class LazyQuerySelfTest extends 
GridCommonAbstractTest {
 
                 return MapQueryLazyWorker.activeCount() == 0;
             }
-        }, 1000L)) {
-            log.error("Lazy workers on nodes:");
-
-            for (Ignite node : Ignition.allGrids()) {
-                IgniteH2Indexing idx = (IgniteH2Indexing) 
((IgniteKernal)node).context().query().getIndexing();
-
-                if (idx.mapQueryExecutor().registeredLazyWorkers() != 0) {
-                    log.error("[node=" + node + ", " + "registeredLazyWorkers="
-                        + idx.mapQueryExecutor().registeredLazyWorkers() + 
']');
-                }
-
-                log.error("Active lazy workers: " + 
MapQueryLazyWorker.activeCount());
-
-                fail("There are not stopped lazy workers. See error message 
above.");
-            }
-        }
+        }, 1000L);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolSelfTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolSelfTest.java
deleted file mode 100644
index 9be0870..0000000
--- 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ObjectPoolSelfTest.java
+++ /dev/null
@@ -1,125 +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.internal.processors.query.h2;
-
-import java.util.concurrent.CompletableFuture;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- *
- */
-public class ObjectPoolSelfTest extends GridCommonAbstractTest {
-    /** */
-    private ObjectPool<Obj> pool = new ObjectPool<>(Obj::new, 1, null, null);
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectIsReusedAfterRecycling() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-
-        Obj o1 = r1.object();
-
-        r1.recycle();
-
-        ObjectPoolReusable<Obj> r2 = pool.borrow();
-
-        Obj o2 = r2.object();
-
-        assertSame(o1, o2);
-
-        assertFalse(o1.isClosed());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBorrowedObjectIsNotReturnedTwice() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-        ObjectPoolReusable<Obj> r2 = pool.borrow();
-
-        assertNotSame(r1.object(), r2.object());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectShouldBeClosedOnRecycleIfPoolIsFull() throws 
Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-        ObjectPoolReusable<Obj> r2 = pool.borrow();
-
-        Obj o2 = r2.object();
-
-        r1.recycle();
-        r2.recycle();
-
-        assertNull(r1.object());
-        assertNull(r2.object());
-
-        assertTrue(o2.isClosed());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectShouldNotBeReturnedIfPoolIsFull() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-        ObjectPoolReusable<Obj> r2 = pool.borrow();
-
-        r1.recycle();
-
-        assertEquals(1, pool.bagSize());
-
-        r2.recycle();
-
-        assertEquals(1, pool.bagSize());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectShouldReturnedToBag() throws Exception {
-        ObjectPoolReusable<Obj> r1 = pool.borrow();
-
-        CompletableFuture.runAsync(() -> {
-            r1.recycle();
-
-            assertEquals(1, pool.bagSize());
-        }).join();
-
-        assertEquals(1, pool.bagSize());
-    }
-
-    /** */
-    private static class Obj implements AutoCloseable {
-        /** */
-        private boolean closed = false;
-
-        /** {@inheritDoc} */
-        @Override public void close() {
-            closed = true;
-        }
-
-        /**
-         * @return {@code True} if closed.
-         */
-        public boolean isClosed() {
-            return closed;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java
new file mode 100644
index 0000000..b7b7a37
--- /dev/null
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/ThreadLocalObjectPoolSelfTest.java
@@ -0,0 +1,113 @@
+/*
+ * 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.query.h2;
+
+import java.util.concurrent.CompletableFuture;
+import 
org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPool.Reusable;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class ThreadLocalObjectPoolSelfTest extends GridCommonAbstractTest {
+    /** */
+    private ThreadLocalObjectPool<Obj> pool = new 
ThreadLocalObjectPool<>(Obj::new, 1);
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectIsReusedAfterRecycling() throws Exception {
+        Reusable<Obj> o1 = pool.borrow();
+        o1.recycle();
+        Reusable<Obj> o2 = pool.borrow();
+
+        assertSame(o1.object(), o2.object());
+        assertFalse(o1.object().isClosed());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBorrowedObjectIsNotReturnedTwice() throws Exception {
+        Reusable<Obj> o1 = pool.borrow();
+        Reusable<Obj> o2 = pool.borrow();
+
+        assertNotSame(o1.object(), o2.object());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectShouldBeClosedOnRecycleIfPoolIsFull() throws 
Exception {
+        Reusable<Obj> o1 = pool.borrow();
+        Reusable<Obj> o2 = pool.borrow();
+        o1.recycle();
+        o2.recycle();
+
+        assertTrue(o2.object().isClosed());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectShouldNotBeReturnedIfPoolIsFull() throws Exception {
+        Reusable<Obj> o1 = pool.borrow();
+        Reusable<Obj> o2 = pool.borrow();
+
+        o1.recycle();
+
+        assertEquals(1, pool.bagSize());
+
+        o2.recycle();
+
+        assertEquals(1, pool.bagSize());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectShouldReturnedToRecyclingThreadBag() throws 
Exception {
+        Reusable<Obj> o1 = pool.borrow();
+
+        CompletableFuture.runAsync(() -> {
+            o1.recycle();
+
+            assertEquals(1, pool.bagSize());
+        }).join();
+
+        assertEquals(0, pool.bagSize());
+    }
+
+    /** */
+    private static class Obj implements AutoCloseable {
+        /** */
+        private boolean closed = false;
+
+        /** {@inheritDoc} */
+        @Override public void close() {
+            closed = true;
+        }
+
+        /**
+         * @return {@code True} if closed.
+         */
+        public boolean isClosed() {
+            return closed;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
----------------------------------------------------------------------
diff --git 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
index ac467d5..dbb2c59 100644
--- 
a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
+++ 
b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/twostep/RetryCauseMessageSelfTest.java
@@ -384,6 +384,11 @@ public class RetryCauseMessageSelfTest extends 
GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public void cancelLazyWorkers() {
+            startedExecutor.cancelLazyWorkers();
+        }
+
+        /** {@inheritDoc} */
         @Override GridSpinBusyLock busyLock() {
             return startedExecutor.busyLock();
         }
@@ -394,8 +399,19 @@ public class RetryCauseMessageSelfTest extends 
GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public void stopAndUnregisterCurrentLazyWorker() {
+            startedExecutor.stopAndUnregisterCurrentLazyWorker();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void unregisterLazyWorker(MapQueryLazyWorker worker) {
+            startedExecutor.unregisterLazyWorker(worker);
+        }
+
+        /** {@inheritDoc} */
         @Override public int registeredLazyWorkers() {
             return startedExecutor.registeredLazyWorkers();
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/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 24ff297..7633d2a 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
@@ -180,8 +180,8 @@ import 
org.apache.ignite.internal.processors.query.h2.H2ResultSetIteratorNullify
 import org.apache.ignite.internal.processors.query.h2.H2StatementCacheSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest;
 import org.apache.ignite.internal.processors.query.h2.IgniteSqlQueryMinMaxTest;
-import org.apache.ignite.internal.processors.query.h2.ObjectPoolSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.PreparedStatementExSelfTest;
+import 
org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPoolSelfTest;
 import 
org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
 import 
org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest;
@@ -475,7 +475,7 @@ public class IgniteCacheQuerySelfTestSuite extends 
TestSuite {
         suite.addTestSuite(SqlUserCommandSelfTest.class);
         suite.addTestSuite(EncryptedSqlTableTest.class);
 
-        suite.addTestSuite(ObjectPoolSelfTest.class);
+        suite.addTestSuite(ThreadLocalObjectPoolSelfTest.class);
         suite.addTestSuite(H2StatementCacheSelfTest.class);
         suite.addTestSuite(PreparedStatementExSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
----------------------------------------------------------------------
diff --git 
a/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h 
b/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
index 4e4614c..519b2ed 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/query_sql_fields.h
@@ -54,7 +54,7 @@ namespace ignite
                     loc(false),
                     distributedJoins(false),
                     enforceJoinOrder(false),
-                    lazy(true),
+                    lazy(false),
                     args()
                 {
                     // No-op.
@@ -73,7 +73,7 @@ namespace ignite
                     loc(false),
                     distributedJoins(false),
                     enforceJoinOrder(false),
-                    lazy(true),
+                    lazy(false),
                     args()
                 {
                     // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp 
b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
index c7014f9..4cfc940 100644
--- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
@@ -46,7 +46,7 @@ namespace
     const bool testEnforceJoinOrder = true;
     const bool testReplicatedOnly = true;
     const bool testCollocated = true;
-    const bool testLazy = false;
+    const bool testLazy = true;
     const bool testSkipReducerOnUpdate = true;
 
     const std::string testAddressStr = testServerHost + ':' + 
ignite::common::LexicalCast<std::string>(testServerPort);
@@ -234,7 +234,7 @@ void CheckDsnConfig(const Configuration& cfg)
     BOOST_CHECK_EQUAL(cfg.IsEnforceJoinOrder(), false);
     BOOST_CHECK_EQUAL(cfg.IsReplicatedOnly(), false);
     BOOST_CHECK_EQUAL(cfg.IsCollocated(), false);
-    BOOST_CHECK_EQUAL(cfg.IsLazy(), true);
+    BOOST_CHECK_EQUAL(cfg.IsLazy(), false);
     BOOST_CHECK_EQUAL(cfg.IsSkipReducerOnUpdate(), false);
     BOOST_CHECK(cfg.GetAddresses().empty());
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/platforms/cpp/odbc/src/config/configuration.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp 
b/modules/platforms/cpp/odbc/src/config/configuration.cpp
index 6caf86e..a99894d 100644
--- a/modules/platforms/cpp/odbc/src/config/configuration.cpp
+++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp
@@ -43,7 +43,7 @@ namespace ignite
             const bool Configuration::DefaultValue::enforceJoinOrder = false;
             const bool Configuration::DefaultValue::replicatedOnly = false;
             const bool Configuration::DefaultValue::collocated = false;
-            const bool Configuration::DefaultValue::lazy = true;
+            const bool Configuration::DefaultValue::lazy = false;
             const bool Configuration::DefaultValue::skipReducerOnUpdate = 
false;
 
             const ProtocolVersion& 
Configuration::DefaultValue::protocolVersion = ProtocolVersion::GetCurrent();

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
index f9c318f..ceeeb37 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesTest.cs
@@ -761,11 +761,11 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         public void TestSqlQueryTimeout()
         {
             var cache = Cache();
-            PopulateCache(cache, false, 30000, x => true);
+            PopulateCache(cache, false, 20000, x => true);
 
-            var sqlQry = new SqlQuery(typeof(QueryPerson), "WHERE age < 2000")
+            var sqlQry = new SqlQuery(typeof(QueryPerson), "WHERE age < 500 
AND name like '%1%'")
             {
-                Timeout = TimeSpan.FromMilliseconds(1)
+                Timeout = TimeSpan.FromMilliseconds(2)
             };
 
             // ReSharper disable once ReturnValueOfPureMethodIsNotUsed

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
index 67bde2e..f5b5baa 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/Linq/CacheLinqTest.Introspection.cs
@@ -103,8 +103,6 @@ namespace Apache.Ignite.Core.Tests.Cache.Query.Linq
 #pragma warning restore 618 // Type or member is obsolete
 
             fq = fieldsQuery.GetFieldsQuery();
-               fq.Lazy = false;
-
             Assert.AreEqual(GetSqlEscapeAll()
                     ? "select _T0.\"Name\" from PERSON_ORG_SCHEMA.\"Person\" 
as _T0"
                     : "select _T0.NAME from PERSON_ORG_SCHEMA.Person as _T0",

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
 
b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
index 760a48d..02d13f6 100644
--- 
a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
+++ 
b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/SqlQueryTest.cs
@@ -150,9 +150,9 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
         {
             var cache = GetClientCache<Person>();
 
-            cache.PutAll(Enumerable.Range(1, 1000).ToDictionary(x => x, x => 
new Person(x)));
+            cache.PutAll(Enumerable.Range(1, 30000).ToDictionary(x => x, x => 
new Person(x)));
 
-            var qry = new SqlFieldsQuery("select * from Person p0, Person p1, 
Person p2")
+            var qry = new SqlFieldsQuery("select * from Person where Name like 
'%ers%'")
             {
                 Timeout = TimeSpan.FromMilliseconds(1)
             };

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
----------------------------------------------------------------------
diff --git 
a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs 
b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
index f81e4ac..a93e00d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Query/SqlFieldsQuery.cs
@@ -36,23 +36,22 @@ namespace Apache.Ignite.Core.Cache.Query
         /// <param name="args">Arguments.</param>
         public SqlFieldsQuery(string sql, params object[] args) : this(sql, 
false, args)
         {
-               Lazy = true;
+            // No-op.
         }
 
-               /// <summary>
-               /// Constructor,
-               /// </summary>
-               /// <param name="sql">SQL.</param>
-               /// <param name="loc">Whether query should be executed 
locally.</param>
-               /// <param name="args">Arguments.</param>
-               public SqlFieldsQuery(string sql, bool loc, params object[] 
args)
+        /// <summary>
+        /// Constructor,
+        /// </summary>
+        /// <param name="sql">SQL.</param>
+        /// <param name="loc">Whether query should be executed locally.</param>
+        /// <param name="args">Arguments.</param>
+        public SqlFieldsQuery(string sql, bool loc, params object[] args)
         {
             Sql = sql;
             Local = loc;
             Arguments = args;
 
             PageSize = DefaultPageSize;
-               Lazy = true;
         }
 
         /// <summary>
@@ -136,20 +135,18 @@ namespace Apache.Ignite.Core.Cache.Query
         /// </summary>
         public string Schema { get; set; }
 
-               /// <summary>
-               /// Gets or sets a value indicating whether this <see 
cref="SqlFieldsQuery"/> is lazy.
-               /// <para />
-               /// When lazy mode is turned off Ignite attempts to fetch the 
whole query result set to memory and send it to the client.
-               /// For small and medium result sets this provides optimal 
performance and minimize duration of internal
-               /// database locks, thus increasing concurrency.
-               /// <para />
-               /// If result set is too big to fit in available memory this 
could lead to excessive GC pauses and even
-               /// OutOfMemoryError. Use this flag as a hint for Ignite to 
fetch result set lazily, thus minimizing memory
-               /// consumption at the cost of moderate performance hit.
-               /// <para />
-               /// Default mode is lazy since version 2.7. The default value 
of the flag is changed to 'true'.
-               /// </summary>
-               public bool Lazy { get; set; }
+        /// <summary>
+        /// Gets or sets a value indicating whether this <see 
cref="SqlFieldsQuery"/> is lazy.
+        /// <para />
+        /// By default Ignite attempts to fetch the whole query result set to 
memory and send it to the client.
+        /// For small and medium result sets this provides optimal performance 
and minimize duration of internal
+        /// database locks, thus increasing concurrency.
+        /// <para />
+        /// If result set is too big to fit in available memory this could 
lead to excessive GC pauses and even
+        /// OutOfMemoryError. Use this flag as a hint for Ignite to fetch 
result set lazily, thus minimizing memory
+        /// consumption at the cost of moderate performance hit.
+        /// </summary>
+        public bool Lazy { get; set; }
 
         /// <summary>
         /// Returns a <see cref="string" /> that represents this instance.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/config/benchmark-native-sql-cache-select.properties
----------------------------------------------------------------------
diff --git 
a/modules/yardstick/config/benchmark-native-sql-cache-select.properties 
b/modules/yardstick/config/benchmark-native-sql-cache-select.properties
deleted file mode 100644
index cb397db..0000000
--- a/modules/yardstick/config/benchmark-native-sql-cache-select.properties
+++ /dev/null
@@ -1,96 +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.
-
-#
-# Contains benchmarks for select queries
-#
-
-now0=`date +'%H%M%S'`
-
-# JVM options.
-JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
-
-# Uncomment to enable concurrent garbage collection (GC) if you encounter long 
GC pauses.
-JVM_OPTS=${JVM_OPTS}" \
--Xms8g \
--Xmx8g \
--Xloggc:./gc${now0}.log \
--XX:+PrintGCDetails \
--verbose:gc \
--XX:+UseParNewGC \
--XX:+UseConcMarkSweepGC \
--XX:+PrintGCDateStamps \
-"
-
-# Add to JVM_OPTS to generate JFR profile.
-#-XX:+UnlockCommercialFeatures \
-#-XX:+FlightRecorder 
-XX:StartFlightRecording=delay=300s,duration=120s,filename=#filename#.jfr \
-
-#Ignite version
-ver="RELEASE-"
-
-# List of default probes.
-# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on 
Linux).
-BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe,DStatProbe
-
-# Packages where the specified benchmark is searched by reflection mechanism.
-BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
-
-# Flag which indicates to restart the servers before every benchmark execution.
-RESTART_SERVERS=true
-
-# Probe point writer class name.
-# BENCHMARK_WRITER=
-
-# The benchmark is applicable only for 1 server and 1 driver
-SERVER_HOSTS=127.0.0.1,127.0.0.1
-DRIVER_HOSTS=127.0.0.1
-
-# Remote username.
-# REMOTE_USER=
-
-# Number of nodes, used to wait for the specified number of nodes to start.
-nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo 
${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
-
-# Backups count.
-b=1
-
-# Warmup.
-w=60
-
-# Duration.
-d=300
-
-# Threads count.
-t=4
-
-# Sync mode.
-sm=PRIMARY_SYNC
-
-# Jobs.
-j=10
-
-# Run configuration which contains all benchmarks.
-# Note that each benchmark is set to run for 300 seconds (5 min) with warm-up 
set to 60 seconds (1 minute).
-CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn 
${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn 
NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy false -ds 
${ver}sql-select-native-r1-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn 
${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn 
NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy false -ds 
${ver}sql-select-native-r1K-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn 
${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 2000 -dn 
NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy false -ds 
${ver}sql-select-native-r2K-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn 
${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 0 -dn 
NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy false -ds 
${ver}sql-select-native-r1M-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn 
${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn 
NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy true -ds 
${ver}sql-select-native-r1-lazy-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn 
${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn 
NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy true -ds 
${ver}sql-select-native-r1K-lazy-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn 
${nodesNum} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 2000 -dn 
NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy true -ds 
${ver}sql-select-native-r2K-lazy-${b}-backup,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-sql-query-config.xml -nn 
${nodesNum}  -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 0 -dn 
NativeSqlCacheQueryRangeBenchmark -sn IgniteNode --lazy true -ds 
${ver}sql-select-native-r1M-lazy-${b}-backup,\
-"

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/config/benchmark-native-sql-select.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/benchmark-native-sql-select.properties 
b/modules/yardstick/config/benchmark-native-sql-select.properties
index 23b78be..0f0b606 100644
--- a/modules/yardstick/config/benchmark-native-sql-select.properties
+++ b/modules/yardstick/config/benchmark-native-sql-select.properties
@@ -51,8 +51,8 @@ RESTART_SERVERS=true
 # BENCHMARK_WRITER=
 
 # The benchmark is applicable only for 1 server and 1 driver
-SERVER_HOSTS=127.0.0.1,127.0.0.1
-DRIVER_HOSTS=127.0.0.1,127.0.0.1,127.0.0.1
+SERVER_HOSTS=127.0.0.1
+DRIVER_HOSTS=127.0.0.1
 
 # Remote username.
 # REMOTE_USER=
@@ -64,7 +64,7 @@ nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + 
`echo ${DRIVER_HOSTS}
 b=1
 
 # Warmup.
-w=60
+w=30
 
 # Duration.
 d=300
@@ -81,13 +81,6 @@ j=10
 # Run configuration which contains all benchmarks.
 # Note that each benchmark is set to run for 300 seconds (5 min) with warm-up 
set to 60 seconds (1 minute).
 CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-backups-${b}-r1 -cl --lazy false,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-backups-${b}-r1K -cl --lazy false,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 0 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-backups-${b}-r1M -cl --lazy false,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-backups-${b}-r1-lazy -cl --lazy true,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-backups-${b}-r1K-lazy -cl --lazy true,\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 0 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-backups-${b}-r1M-lazy -cl --lazy true,\
-"
-CONFIGS="\
--cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-backups-${b}-r1 -cl --lazy false -r 10000,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-r1-${b}-backup -cl,\
+-cfg ${SCRIPT_DIR}/../config/ignite-localhost-config.xml -nn ${nodesNum} -b 
${b} -w ${w} -d ${d} -t ${t} -sm ${sm} --sqlRange 1000 -dn 
NativeSqlQueryRangeBenchmark -sn IgniteNode -ds 
${ver}sql-select-native-r1000-${b}-backup -cl,\
 "

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/config/ignite-localhost-sql-query-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-localhost-sql-query-config.xml 
b/modules/yardstick/config/ignite-localhost-sql-query-config.xml
deleted file mode 100644
index 79a4de6..0000000
--- a/modules/yardstick/config/ignite-localhost-sql-query-config.xml
+++ /dev/null
@@ -1,91 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-
-<!--
-  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.
--->
-
-<!--
-    Ignite Spring configuration file to startup grid.
--->
-<beans xmlns="http://www.springframework.org/schema/beans";
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
-       xsi:schemaLocation="
-        http://www.springframework.org/schema/beans 
http://www.springframework.org/schema/beans/spring-beans-2.5.xsd";>
-    <import resource="ignite-base-config.xml"/>
-
-    <bean id="grid.cfg" 
class="org.apache.ignite.configuration.IgniteConfiguration" 
parent="base-ignite.cfg">
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <bean 
class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <value>127.0.0.1:47500</value>
-                                <value>127.0.0.1:47501</value>
-                                <value>127.0.0.1:47502</value>
-                                <value>127.0.0.1:47503</value>
-                                <value>127.0.0.1:47504</value>
-                                <value>127.0.0.1:47505</value>
-                                <value>127.0.0.1:47506</value>
-                                <value>127.0.0.1:47507</value>
-                                <value>127.0.0.1:47508</value>
-                                <value>127.0.0.1:47509</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-            </bean>
-        </property>
-
-        <property name="communicationSpi">
-            <bean 
class="org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi">
-                <property name="sharedMemoryPort" value="-1"/>
-            </bean>
-        </property>
-
-        <property name="cacheConfiguration">
-            <list>
-                <bean 
class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="test"/>
-
-                    <property name="backups" value="1"/>
-
-                    <property name="cacheMode" value="PARTITIONED"/>
-
-                    <property name="atomicityMode" value="ATOMIC"/>
-
-                    <property name="queryEntities">
-                        <list>
-                            <bean class="org.apache.ignite.cache.QueryEntity">
-                                <property name="keyType" 
value="java.lang.Long"/>
-                                <property name="valueType" value="TEST_LONG"/>
-                                <property name="tableName" value="TEST_LONG"/>
-                                <property name="keyFieldName" value="ID"/>
-
-                                <property name="fields">
-                                    <map>
-                                        <entry key="ID" 
value="java.lang.Long"/>
-                                        <entry key="VAL" 
value="java.lang.Long"/>
-                                    </map>
-                                </property>
-                            </bean>
-                        </list>
-                    </property>
-                </bean>
-            </list>
-        </property>
-    </bean>
-</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
----------------------------------------------------------------------
diff --git 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
index 21442dc..5aec308 100644
--- 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
+++ 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteAbstractBenchmark.java
@@ -17,10 +17,9 @@
 
 package org.apache.ignite.yardstick;
 
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCountDownLatch;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteState;
 import org.apache.ignite.Ignition;
@@ -30,7 +29,6 @@ import org.yardstickframework.BenchmarkConfiguration;
 import org.yardstickframework.BenchmarkDriverAdapter;
 import org.yardstickframework.BenchmarkUtils;
 
-import static org.apache.ignite.events.EventType.EVTS_DISCOVERY;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.yardstickframework.BenchmarkUtils.jcommander;
 import static org.yardstickframework.BenchmarkUtils.println;
@@ -39,8 +37,6 @@ import static org.yardstickframework.BenchmarkUtils.println;
  * Abstract class for Ignite benchmarks.
  */
 public abstract class IgniteAbstractBenchmark extends BenchmarkDriverAdapter {
-    private static final long WAIT_NODES_TIMEOUT = 
TimeUnit.SECONDS.toMillis(30);
-
     /** Arguments. */
     protected final IgniteBenchmarkArguments args = new 
IgniteBenchmarkArguments();
 
@@ -130,30 +126,22 @@ public abstract class IgniteAbstractBenchmark extends 
BenchmarkDriverAdapter {
      * @throws Exception If failed.
      */
     private void waitForNodes() throws Exception {
-        IgniteCountDownLatch allNodesReady = 
ignite().countDownLatch("allNodesReady", 1, false, true);
+        final CountDownLatch nodesStartedLatch = new CountDownLatch(1);
 
-        // wait for condition when all nodes are ready and release distributed 
barrier.
         ignite().events().localListen(new IgnitePredicate<Event>() {
             @Override public boolean apply(Event gridEvt) {
-                if (nodesStarted()) {
-                    allNodesReady.countDown();
-                    // todo: return false so unregister?
-                }
+                if (nodesStarted())
+                    nodesStartedLatch.countDown();
 
                 return true;
             }
-        }, EVTS_DISCOVERY);
-
-        if (nodesStarted())
-            allNodesReady.countDown();
+        }, EVT_NODE_JOINED);
 
-        // block on distributed barrier till member 0 release it.
-        println(cfg, "Start waiting for cluster to contain " + args.nodes() + 
".");
+        if (!nodesStarted()) {
+            println(cfg, "Waiting for " + (args.nodes() - 1) + " nodes to 
start...");
 
-        //todo: timeouts?
-        allNodesReady.await();
-
-        println(cfg, "Cluster is ready.");
+            nodesStartedLatch.await();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index 3bd5e87..3f4fddc 100644
--- 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -289,12 +289,6 @@ public class IgniteBenchmarkArguments {
     @GridToStringInclude
     public long mvccContentionRange = 10_000;
 
-    /** */
-    @Parameter(names = {"--lazy"},
-        arity = 1,
-        description = "Lazy mode for SQL query execution (default true).")
-    private boolean lazy = true;
-
     /**
      * @return {@code True} if need set {@link DataStorageConfiguration}.
      */
@@ -718,13 +712,6 @@ public class IgniteBenchmarkArguments {
         return mvccContentionRange;
     }
 
-    /**
-     * @return Lazy query execution mode.
-     */
-    public boolean isLazy() {
-        return lazy;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return GridToStringBuilder.toString(IgniteBenchmarkArguments.class, 
this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
----------------------------------------------------------------------
diff --git 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
index 0f99a9d..129e6a4 100644
--- 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
+++ 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/AbstractNativeBenchmark.java
@@ -21,7 +21,6 @@ import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
 import org.yardstickframework.BenchmarkConfiguration;
-import org.yardstickframework.BenchmarkUtils;
 
 import static org.apache.ignite.yardstick.jdbc.JdbcUtils.fillData;
 
@@ -34,7 +33,5 @@ public abstract class AbstractNativeBenchmark extends 
IgniteAbstractBenchmark {
         super.setUp(cfg);
 
         fillData(cfg, (IgniteEx)ignite(), args.range(), args.atomicMode());
-
-        BenchmarkUtils.println("Lazy mode: " + args.isLazy());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
----------------------------------------------------------------------
diff --git 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
index 4466f89..81d6c17 100644
--- 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
+++ 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/JdbcUtils.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.yardstick.jdbc;
 
-import org.apache.ignite.IgniteSemaphore;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.IgniteEx;
@@ -34,46 +33,30 @@ public class JdbcUtils {
      * @param cfg Benchmark configuration.
      * @param ignite Ignite node.
      * @param range Data key range.
-     * @param atomicMode Cache atomic mode.
      */
     public static void fillData(BenchmarkConfiguration cfg,  IgniteEx ignite, 
long range, CacheAtomicityMode atomicMode) {
-        IgniteSemaphore sem = ignite.semaphore("jdbc-setup", 1, true, true);
+        println(cfg, "Create table...");
 
-        try {
-            if (sem.tryAcquire()) {
-                println(cfg, "Create table...");
+        String withExpr = atomicMode != null ? " WITH \"atomicity=" + 
atomicMode.name() + "\";" : ";";
 
-                String withExpr = atomicMode != null ? " WITH \"atomicity=" + 
atomicMode.name() + "\";" : ";";
+        String qry = "CREATE TABLE test_long (id long primary key, val long)" 
+ withExpr;
 
-                String qry = "CREATE TABLE test_long (id long primary key, val 
long)" + withExpr;
+        println(cfg, "Creating table with schema: " + qry);
 
-                println(cfg, "Creating table with schema: " + qry);
+        ignite.context().query().querySqlFields(
+            new SqlFieldsQuery(qry), true);
 
-                ignite.context().query().querySqlFields(
-                    new SqlFieldsQuery(qry), true);
+        println(cfg, "Populate data...");
 
-                println(cfg, "Populate data...");
+        for (long l = 1; l <= range; ++l) {
+            ignite.context().query().querySqlFields(
+                new SqlFieldsQuery("insert into test_long (id, val) values (?, 
?)")
+                    .setArgs(l, l + 1), true);
 
-                for (long l = 1; l <= range; ++l) {
-                    ignite.context().query().querySqlFields(
-                        new SqlFieldsQuery("insert into test_long (id, val) 
values (?, ?)")
-                            .setArgs(l, l + 1), true);
-
-                    if (l % 10000 == 0)
-                        println(cfg, "Populate " + l);
-                }
-
-                println(cfg, "Finished populating data");
-            }
-            else {
-                // Acquire (wait setup by other client) and immediately 
release/
-                println(cfg, "Waits for setup...");
-
-                sem.acquire();
-            }
-        }
-        finally {
-            sem.release();
+            if (l % 10000 == 0)
+                println(cfg, "Populate " + l);
         }
+
+        println(cfg, "Finished populating data");
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlCacheQueryRangeBenchmark.java
----------------------------------------------------------------------
diff --git 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlCacheQueryRangeBenchmark.java
 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlCacheQueryRangeBenchmark.java
deleted file mode 100644
index e361c9a..0000000
--- 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlCacheQueryRangeBenchmark.java
+++ /dev/null
@@ -1,145 +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.yardstick.jdbc;
-
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteSemaphore;
-import org.apache.ignite.cache.query.FieldsQueryCursor;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
-import org.yardstickframework.BenchmarkConfiguration;
-import org.yardstickframework.BenchmarkUtils;
-
-import static org.apache.ignite.yardstick.jdbc.JdbcUtils.fillData;
-import static org.yardstickframework.BenchmarkUtils.println;
-
-/**
- * Native sql benchmark that performs select operations.
- */
-public class NativeSqlCacheQueryRangeBenchmark extends IgniteAbstractBenchmark 
{
-    private IgniteCache cache;
-
-    /** {@inheritDoc} */
-    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
-        super.setUp(cfg);
-
-        fillData();
-
-        BenchmarkUtils.println("Lazy mode: " + args.isLazy());
-    }
-
-    /**
-     *
-     */
-    private void fillData() {
-        IgniteEx ignite = (IgniteEx)ignite();
-
-        IgniteSemaphore sem = ignite.semaphore("data-setup", 1, true, true);
-
-        cache = ignite.getOrCreateCache("test").withKeepBinary();
-
-        try {
-            if (sem.tryAcquire()) {
-
-                println(cfg, "Populate data...");
-
-                for (long l = 1; l <= args.range(); ++l) {
-                    cache.query(
-                        new SqlFieldsQuery("insert into test_long(id, val) 
values (?, ?)")
-                            .setArgs(l, l + 1));
-
-                    if (l % 10000 == 0)
-                        println(cfg, "Populate " + l);
-                }
-
-                println(cfg, "Finished populating data");
-            }
-            else {
-                // Acquire (wait setup by other client) and immediately 
release/
-                println(cfg, "Waits for setup...");
-
-                sem.acquire();
-            }
-        }
-        finally {
-            sem.release();
-        }
-    }
-
-    /**
-     * Benchmarked action that performs selects and validates results.
-     *
-     * {@inheritDoc}
-     */
-    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        long expRsSize;
-
-        SqlFieldsQuery qry;
-
-        if (args.sqlRange() == 1) {
-            qry = new SqlFieldsQuery("SELECT id, val FROM test_long WHERE id = 
?");
-
-            qry.setArgs(ThreadLocalRandom.current().nextLong(args.range()) + 
1);
-
-            expRsSize = 1;
-        }
-        else if (args.sqlRange() <= 0) {
-            qry = new SqlFieldsQuery("SELECT id, val FROM test_long");
-
-            expRsSize = args.range();
-        }
-        else {
-            qry = new SqlFieldsQuery("SELECT id, val FROM test_long WHERE id 
BETWEEN ? AND ?");
-
-            long id = ThreadLocalRandom.current().nextLong(args.range() - 
args.sqlRange()) + 1;
-            long maxId = id + args.sqlRange() - 1;
-
-            qry.setArgs(id, maxId);
-
-            expRsSize = args.sqlRange();
-        }
-
-        qry.setLazy(args.isLazy());
-
-        long rsSize = 0;
-
-        try (FieldsQueryCursor<List<?>> cursor = cache.query(qry)) {
-            Iterator<List<?>> it = cursor.iterator();
-
-            while (it.hasNext()) {
-                List<?> row = it.next();
-
-                if ((Long)row.get(0) + 1 != (Long)row.get(1))
-                    throw new Exception("Invalid result retrieved");
-
-                rsSize++;
-            }
-        }
-
-        if (rsSize != expRsSize)
-            throw new Exception("Invalid result set size [actual=" + rsSize + 
", expected=" + expRsSize + ']');
-
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ccde7c4/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
----------------------------------------------------------------------
diff --git 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
index 33630fd..8dcdda7 100644
--- 
a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
+++ 
b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/jdbc/NativeSqlQueryRangeBenchmark.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.yardstick.jdbc;
 
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
@@ -46,11 +45,6 @@ public class NativeSqlQueryRangeBenchmark extends 
AbstractNativeBenchmark {
 
             expRsSize = 1;
         }
-        else if (args.sqlRange() <= 0) {
-            qry = new SqlFieldsQuery("SELECT id, val FROM test_long");
-
-            expRsSize = args.range();
-        }
         else {
             qry = new SqlFieldsQuery("SELECT id, val FROM test_long WHERE id 
BETWEEN ? AND ?");
 
@@ -62,17 +56,12 @@ public class NativeSqlQueryRangeBenchmark extends 
AbstractNativeBenchmark {
             expRsSize = args.sqlRange();
         }
 
-        qry.setLazy(args.isLazy());
-
         long rsSize = 0;
 
         try (FieldsQueryCursor<List<?>> cursor = 
((IgniteEx)ignite()).context().query()
                 .querySqlFields(qry, false)) {
-            Iterator<List<?>> it = cursor.iterator();
-
-            while (it.hasNext()) {
-                List<?> row = it.next();
 
+            for (List<?> row : cursor) {
                 if ((Long)row.get(0) + 1 != (Long)row.get(1))
                     throw new Exception("Invalid result retrieved");
 

Reply via email to