Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 52B3017A3B for ; Wed, 4 Mar 2015 01:32:02 +0000 (UTC) Received: (qmail 67181 invoked by uid 500); 4 Mar 2015 01:32:02 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 67147 invoked by uid 500); 4 Mar 2015 01:32:02 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 67137 invoked by uid 99); 4 Mar 2015 01:32:02 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 04 Mar 2015 01:32:02 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Wed, 04 Mar 2015 01:31:55 +0000 Received: (qmail 63669 invoked by uid 99); 4 Mar 2015 01:31:31 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 04 Mar 2015 01:31:31 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 53AFCE102B; Wed, 4 Mar 2015 01:31:31 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vkulichenko@apache.org To: commits@ignite.incubator.apache.org Date: Wed, 04 Mar 2015 01:31:47 -0000 Message-Id: <187b57971f7248f1bad6d3192f931ec1@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [17/50] [abbrv] incubator-ignite git commit: IGNITE-261 Scalar: fixed examples and tests. X-Virus-Checked: Checked by ClamAV on apache.org IGNITE-261 Scalar: fixed examples and 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/f0209856 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/f0209856 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/f0209856 Branch: refs/heads/master Commit: f0209856df3b4898fd54b331e05ba0a7956ab250 Parents: 7007771 Author: AKuznetsov Authored: Mon Feb 16 19:46:06 2015 +0700 Committer: AKuznetsov Committed: Mon Feb 16 19:46:06 2015 +0700 ---------------------------------------------------------------------- .../examples/datagrid/CacheQueryExample.java | 25 +- .../scalar/examples/ScalarCacheExample.scala | 4 - .../ScalarCachePopularNumbersExample.scala | 9 +- .../examples/ScalarCacheQueryExample.scala | 32 +- .../examples/ScalarSnowflakeSchemaExample.scala | 26 +- .../ignite/scalar/pimps/ScalarCachePimp.scala | 1340 +----------------- .../scalar/tests/ScalarCacheQueriesSpec.scala | 461 +----- .../ignite/scalar/tests/ScalarCacheSpec.scala | 16 +- 8 files changed, 118 insertions(+), 1795 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0209856/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java ---------------------------------------------------------------------- diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java index e041244..9fe9faf 100644 --- a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java +++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheQueryExample.java @@ -109,10 +109,8 @@ public class CacheQueryExample { /** * Example for SQL queries based on salary ranges. - * - * @throws IgniteCheckedException In case of error. */ - private static void sqlQuery() throws IgniteCheckedException { + private static void sqlQuery() { IgniteCache, Person> cache = Ignition.ignite().jcache(CACHE_NAME); // SQL clause which selects salaries based on range. @@ -131,10 +129,8 @@ public class CacheQueryExample { /** * Example for SQL queries based on all employees working for a specific organization. - * - * @throws IgniteCheckedException In case of error. */ - private static void sqlQueryWithJoin() throws IgniteCheckedException { + private static void sqlQueryWithJoin() { IgniteCache, Person> cache = Ignition.ignite().jcache(CACHE_NAME); // SQL clause query which joins on 2 types to select people for a specific organization. @@ -152,10 +148,8 @@ public class CacheQueryExample { /** * Example for TEXT queries using LUCENE-based indexing of people's resumes. - * - * @throws IgniteCheckedException In case of error. */ - private static void textQuery() throws IgniteCheckedException { + private static void textQuery() { IgniteCache, Person> cache = Ignition.ignite().jcache(CACHE_NAME); // Query for all people with "Master Degree" in their resumes. @@ -173,10 +167,8 @@ public class CacheQueryExample { /** * Example for SQL-based fields queries that return only required * fields instead of whole key-value pairs. - * - * @throws IgniteCheckedException In case of error. */ - private static void sqlFieldsQuery() throws IgniteCheckedException { + private static void sqlFieldsQuery() { IgniteCache cache = Ignition.ignite().jcache(CACHE_NAME); // Create query to get names of all employees. @@ -194,10 +186,8 @@ public class CacheQueryExample { /** * Example for SQL-based fields queries that return only required * fields instead of whole key-value pairs. - * - * @throws IgniteCheckedException In case of error. */ - private static void sqlFieldsQueryWithJoin() throws IgniteCheckedException { + private static void sqlFieldsQueryWithJoin() { IgniteCache cache = Ignition.ignite().jcache(CACHE_NAME); // Execute query to get names of all employees. @@ -214,11 +204,8 @@ public class CacheQueryExample { /** * Populate cache with test data. - * - * @throws IgniteCheckedException In case of error. - * @throws InterruptedException In case of error. */ - private static void initialize() throws IgniteCheckedException, InterruptedException { + private static void initialize() { IgniteCache cache = Ignition.ignite().jcache(CACHE_NAME); // Organizations. http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0209856/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala ---------------------------------------------------------------------- diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala index a263350..e834da3 100644 --- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala +++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheExample.scala @@ -17,8 +17,6 @@ package org.apache.ignite.scalar.examples -import org.apache.ignite.cache.CachePeekMode - import org.apache.ignite.events.Event import org.apache.ignite.events.EventType._ import org.apache.ignite.lang.IgnitePredicate @@ -38,8 +36,6 @@ object ScalarCacheExample extends App { /** Name of cache specified in spring configuration. */ private val NAME = "partitioned" - private val peekModes = Array.empty[CachePeekMode] - scalar("examples/config/example-cache.xml") { // Clean up caches on all nodes before run. cache$(NAME).get.clear() http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0209856/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala ---------------------------------------------------------------------- diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala index 69fe0ed..98d9637 100644 --- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala +++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCachePopularNumbersExample.scala @@ -25,6 +25,7 @@ import org.apache.ignite.scalar.scalar import org.apache.ignite.scalar.scalar._ import scala.util.Random +import collection.JavaConversions._ /** * Real time popular number counter. @@ -106,9 +107,11 @@ object ScalarCachePopularNumbersExample extends App { * @param cnt Number of most popular numbers to return. */ def query(cnt: Int) { - cache$[Int, Long](CACHE_NAME).get. - sqlFields(clause = "select _key, _val from Long order by _val desc limit " + cnt). - sortBy(_(1).asInstanceOf[Long]).reverse.take(cnt).foreach(println) + val results = cache$[Int, Long](CACHE_NAME).get + .sqlFields(clause = "select _key, _val from Long order by _val desc limit " + cnt) + .getAll + + results.foreach(res => println(res.get(0) + "=" + res.get(1))) println("------------------") } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0209856/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheQueryExample.scala ---------------------------------------------------------------------- diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheQueryExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheQueryExample.scala index 44b0deb..6d40544 100644 --- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheQueryExample.scala +++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarCacheQueryExample.scala @@ -25,6 +25,8 @@ import org.apache.ignite.cache.affinity.CacheAffinityKey import org.apache.ignite.scalar.scalar import org.apache.ignite.scalar.scalar._ +import collection.JavaConversions._ + /** * Demonstrates cache ad-hoc queries with Scalar. *

@@ -62,42 +64,18 @@ object ScalarCacheQueryExample { // Using distributed queries for partitioned cache and local queries for replicated cache. // Since in replicated caches data is available on all nodes, including local one, // it is enough to just query the local node. - val prj = if (cache$[Any, Any](CACHE_NAME).get.configuration().getCacheMode == PARTITIONED) + val prj = if (cache$(CACHE_NAME).get.configuration().getCacheMode == PARTITIONED) ignite.cluster().forRemotes() else ignite.cluster().forLocal() // Example for SQL-based querying employees based on salary ranges. // Gets all persons with 'salary > 1000'. - print("People with salary more than 1000: ", cache.sql(prj, "salary > 1000").map(_._2)) + print("People with salary more than 1000: ", cache.sql("salary > 1000").getAll.map(e => e.getValue)) // Example for TEXT-based querying for a given string in people resumes. // Gets all persons with 'Bachelor' degree. - print("People with Bachelor degree: ", cache.text(prj, "Bachelor").map(_._2)) - - // Example for SQL-based querying with custom remote transformer to make sure - // that only required data without any overhead is returned to caller. - // Gets last names of all 'Ignite' employees. - print("Last names of all 'Ignite' employees: ", - cache.sqlTransform( - prj, - "from Person, Organization where Person.orgId = Organization.id " + - "and Organization.name = 'Ignite'", - (p: Person) => p.lastName - ).map(_._2) - ) - - // Example for SQL-based querying with custom remote and local reducers - // to calculate average salary among all employees within a company. - // Gets average salary of persons with 'Master' degree. - print("Average salary of people with Master degree: ", - cache.textReduce( - prj, - "Master", - (e: Iterable[(CacheAffinityKey[UUID], Person)]) => (e.map(_._2.salary).sum, e.size), - (e: Iterable[(Double, Int)]) => e.map(_._1).sum / e.map(_._2).sum - ) - ) + print("People with Bachelor degree: ", cache.text("Bachelor").getAll.map(e => e.getValue)) } /** http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0209856/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarSnowflakeSchemaExample.scala ---------------------------------------------------------------------- diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarSnowflakeSchemaExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarSnowflakeSchemaExample.scala index 723f3be..8e7e434 100644 --- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarSnowflakeSchemaExample.scala +++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarSnowflakeSchemaExample.scala @@ -17,14 +17,16 @@ package org.apache.ignite.scalar.examples -import org.apache.ignite.Ignition import org.apache.ignite.scalar.scalar import org.apache.ignite.scalar.scalar._ import org.jdk8.backport.ThreadLocalRandom8 +import javax.cache.Cache import java.util.ConcurrentModificationException +import collection.JavaConversions._ + /** * Snowflake Schema is a logical * arrangement of data in which data is split into `dimensions` and `facts` @@ -131,7 +133,7 @@ object ScalarSnowflakeSchemaExample { "from \"replicated\".DimStore, \"partitioned\".FactPurchase " + "where DimStore.id=FactPurchase.storeId and DimStore.name=?", "Store1") - printQueryResults("All purchases made at store1:", storePurchases) + printQueryResults("All purchases made at store1:", storePurchases.getAll) } /** @@ -141,7 +143,7 @@ object ScalarSnowflakeSchemaExample { * stored in `partitioned` cache. */ private def queryProductPurchases() { - val factCache = Ignition.ignite.jcache(PART_CACHE_NAME) + val factCache = ignite$.jcache[Int, FactPurchase](PART_CACHE_NAME) // All purchases for certain product made at store2. // ================================================= @@ -151,14 +153,14 @@ object ScalarSnowflakeSchemaExample { println("IDs of products [p1=" + p1.id + ", p2=" + p2.id + ", p3=" + p3.id + ']') -// val prodPurchases = factCache.sql( -// "from \"replicated\".DimStore, \"replicated\".DimProduct, \"partitioned\".FactPurchase " + -// "where DimStore.id=FactPurchase.storeId and " + -// "DimProduct.id=FactPurchase.productId and " + -// "DimStore.name=? and DimProduct.id in(?, ?, ?)", -// "Store2", p1.id, p2.id, p3.id) -// -// printQueryResults("All purchases made at store2 for 3 specific products:", prodPurchases) + val prodPurchases = factCache.sql( + "from \"replicated\".DimStore, \"replicated\".DimProduct, \"partitioned\".FactPurchase " + + "where DimStore.id=FactPurchase.storeId and " + + "DimProduct.id=FactPurchase.productId and " + + "DimStore.name=? and DimProduct.id in(?, ?, ?)", + "Store2", p1.id, p2.id, p3.id) + + printQueryResults("All purchases made at store2 for 3 specific products:", prodPurchases.getAll) } /** @@ -167,7 +169,7 @@ object ScalarSnowflakeSchemaExample { * @param msg Initial message. * @param res Results to print. */ - private def printQueryResults[V](msg: String, res: Iterable[(Int, V)]) { + private def printQueryResults[V](msg: String, res: Iterable[Cache.Entry[Int, V]]) { println(msg) for (e <- res) http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f0209856/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarCachePimp.scala ---------------------------------------------------------------------- diff --git a/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarCachePimp.scala b/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarCachePimp.scala index ec2249b..05a5e2b 100644 --- a/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarCachePimp.scala +++ b/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarCachePimp.scala @@ -17,20 +17,18 @@ package org.apache.ignite.scalar.pimps -import org.apache.ignite.cache.query.{SqlQuery, QueryCursor} +import org.apache.ignite.cache.query._ import org.apache.ignite.configuration.CacheConfiguration import javax.cache.Cache import org.apache.ignite._ -import org.apache.ignite.cluster.ClusterGroup -import org.apache.ignite.internal.util.scala.impl -import org.apache.ignite.lang.{IgniteBiTuple, IgniteClosure, IgnitePredicate, IgniteReducer} +import org.apache.ignite.lang.{IgnitePredicate, IgniteReducer} import org.apache.ignite.scalar.pimps.ScalarCacheConfigurationHelper._ import org.apache.ignite.scalar.scalar._ import org.jetbrains.annotations.Nullable -import java.util.{Set => JavaSet} +import java.util.{List => JavaList, Set => JavaSet} import scala.collection._ import scala.collection.JavaConversions._ @@ -137,15 +135,6 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { } } - private def toRemoteTransformer[K, V, T](trans: V => T): - IgniteClosure[java.util.Map.Entry[K, V], java.util.Map.Entry[K, T]] = { - new IgniteClosure[java.util.Map.Entry[K, V], java.util.Map.Entry[K, T]] { - @impl def apply(e: java.util.Map.Entry[K, V]): java.util.Map.Entry[K, T] = { - new IgniteBiTuple[K, T](e.getKey, trans(e.getValue)) - } - } - } - /** * Retrieves value mapped to the specified key from cache. The return value of `null` * means entry did not pass the provided filter or cache has no mapping for the key. @@ -499,60 +488,7 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { } /** - * Creates and executes ad-hoc `SCAN` query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @return Collection of cache key-value pairs. - */ - def scan(@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], kvp: KvPred): Iterable[(K, V)] = { - assert(cls != null) - assert(kvp != null) - - // val q = value.queries().createScanQuery(kvp) - // - // (if (grid != null) q.projection(grid) else q).execute().get.map(e => (e.getKey, e.getValue)) - - Iterable.empty - } - - /** - * Creates and executes ad-hoc `SCAN` query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @return Collection of cache key-value pairs. - */ - def scan(@Nullable grid: ClusterGroup, kvp: KvPred) - (implicit m: Manifest[V]): Iterable[(K, V)] = { - assert(kvp != null) - - scan(grid, m.erasure.asInstanceOf[Class[V]], kvp) - } - - /** - * Creates and executes ad-hoc `SCAN` query on global projection returning its result. + * Creates and executes ad-hoc `SCAN` query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -566,15 +502,15 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. * @return Collection of cache key-value pairs. */ - def scan(cls: Class[_ <: V], kvp: KvPred): Iterable[(K, V)] = { + def scan(cls: Class[_ <: V], kvp: KvPred): QueryCursor[Cache.Entry[K, V]] = { assert(cls != null) assert(kvp != null) - scan(null, cls, kvp) + value.query(new ScanQuery(kvp)) } /** - * Creates and executes ad-hoc `SCAN` query on global projection returning its result. + * Creates and executes ad-hoc `SCAN` query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -589,14 +525,14 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. * @return Collection of cache key-value pairs. */ - def scan(kvp: KvPred)(implicit m: Manifest[V]): Iterable[(K, V)] = { + def scan(kvp: KvPred)(implicit m: Manifest[V]): QueryCursor[Cache.Entry[K, V]] = { assert(kvp != null) scan(m.erasure.asInstanceOf[Class[V]], kvp) } /** - * Creates and executes ad-hoc `SQL` query on given projection returning its result. + * Creates and executes ad-hoc `SQL` query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -605,77 +541,27 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * all results at once without pagination and therefore memory limits should be * taken into account. * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` * query needs to know the exact type it should operate on. * @param clause Query SQL clause. See `CacheQuery` for more details. * @param args Optional list of query arguments. * @return Collection of cache key-value pairs. */ - def sql(@Nullable grid: ClusterGroup, cls: Class[_ <: V], clause: String, args: Any*): Iterable[(K, V)] = { + def sql(cls: Class[_ <: V], clause: String, args: Any*): QueryCursor[Cache.Entry[K, V]] = { assert(cls != null) assert(clause != null) assert(args != null) - value.query(new SqlQuery(cls, clause).setArgs(args)) - - return Iterable.empty - } - - /** - * Creates and executes ad-hoc `SQL` query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @return Collection of cache key-value pairs. - */ - def sql(@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], clause: String): Iterable[(K, V)] = { - assert(cls != null) - assert(clause != null) - - sql(grid, cls, clause, Nil: _*) - } + val query = new SqlQuery(cls, clause) - /** - * Creates and executes ad-hoc `SQL` query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param args Optional list of query arguments. - * @return Collection of cache key-value pairs. - */ - def sql(@Nullable grid: ClusterGroup, clause: String, args: Any*) - (implicit m: Manifest[V]): Iterable[(K, V)] = { - assert(clause != null) - assert(args != null) + if (args != null && args.size > 0) + query.setArgs(args.map(_.asInstanceOf[AnyRef]) : _*) - sql(grid, m.erasure.asInstanceOf[Class[V]], clause, args: _*) + value.query(query) } /** - * Creates and executes ad-hoc `SQL` query on global projection returning its result. + * Creates and executes ad-hoc `SQL` query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -687,18 +573,17 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` * query needs to know the exact type it should operate on. * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param args Optional list of query arguments. * @return Collection of cache key-value pairs. */ - def sql(cls: Class[_ <: V], clause: String, args: Any*): Iterable[(K, V)] = { + def sql(cls: Class[_ <: V], clause: String): QueryCursor[Cache.Entry[K, V]] = { assert(cls != null) assert(clause != null) - sql(null.asInstanceOf[ClusterGroup], cls, clause, args: _*) + sql(cls, clause, Nil:_*) } /** - * Creates and executes ad-hoc `SQL` query on global projection returning its result. + * Creates and executes ad-hoc `SQL` query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -714,66 +599,16 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * @param args Optional list of query arguments. * @return Collection of cache key-value pairs. */ - def sql(clause: String, args: Any*)(implicit m: Manifest[V]): Iterable[(K, V)] = { - assert(clause != null) - - sql(m.erasure.asInstanceOf[Class[V]], clause, args: _*) - } - - /** - * Creates and executes ad-hoc `TEXT` query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query text clause. See `CacheQuery` for more details. - * @return Collection of cache key-value pairs. - */ - def text(@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], clause: String): Iterable[(K, V)] = { - assert(cls != null) - assert(clause != null) - - // val q = value.cache().queries().createFullTextQuery(cls, clause) - // - // (if (grid != null) q.projection(grid) else q).execute().get.map(e => (e.getKey, e.getValue)) - - Iterable.empty - } - - /** - * Creates and executes ad-hoc `TEXT` query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param clause Query text clause. See `CacheQuery` for more details. - * @return Collection of cache key-value pairs. - */ - def text(@Nullable grid: ClusterGroup, clause: String)(implicit m: Manifest[V]): Iterable[(K, V)] = { + def sql(clause: String, args: Any*) + (implicit m: Manifest[V]): QueryCursor[Cache.Entry[K, V]] = { assert(clause != null) + assert(args != null) - text(grid, m.erasure.asInstanceOf[Class[V]], clause) + sql(m.erasure.asInstanceOf[Class[V]], clause, args:_*) } /** - * Creates and executes ad-hoc `TEXT` query on global projection returning its result. + * Creates and executes ad-hoc `TEXT` query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -787,15 +622,15 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * @param clause Query text clause. See `CacheQuery` for more details. * @return Collection of cache key-value pairs. */ - def text(cls: Class[_ <: V], clause: String): Iterable[(K, V)] = { + def text(cls: Class[_ <: V], clause: String): QueryCursor[Cache.Entry[K, V]] = { assert(cls != null) assert(clause != null) - text(null, cls, clause) + value.query(new TextQuery(cls, clause)) } /** - * Creates and executes ad-hoc `TEXT` query on global projection returning its result. + * Creates and executes ad-hoc `TEXT` query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -810,180 +645,14 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * @param clause Query text clause. See `CacheQuery` for more details. * @return Collection of cache key-value pairs. */ - def text(clause: String)(implicit m: Manifest[V]): Iterable[(K, V)] = { + def text(clause: String)(implicit m: Manifest[V]): QueryCursor[Cache.Entry[K, V]] = { assert(clause != null) text(m.erasure.asInstanceOf[Class[V]], clause) } /** - * Creates and executes ad-hoc `SCAN` transform query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def scanTransform[T](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], kvp: KvPred, trans: V => T): - Iterable[(K, T)] = { - assert(cls != null) - assert(kvp != null) - assert(trans != null) - - // val q = value.cache[K, V]().queries().createScanQuery(kvp) - // - // toScalaItr[K, T]((if (grid != null) q.projection(grid) else q).execute(toRemoteTransformer[K, V, T](trans)).get) - - Iterable.empty - } - - /** - * Creates and executes ad-hoc `SCAN` transform query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the global projection will be used. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def scanTransform[T](@Nullable grid: ClusterGroup, kvp: KvPred, trans: V => T)(implicit m: Manifest[V]): - Iterable[(K, T)] = { - assert(kvp != null) - assert(trans != null) - - scanTransform(grid, m.erasure.asInstanceOf[Class[V]], kvp, trans) - } - - /** - * Creates and executes ad-hoc `SCAN` transform query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def scanTransform[T](cls: Class[_ <: V], kvp: KvPred, trans: V => T): Iterable[(K, T)] = { - assert(cls != null) - assert(kvp != null) - assert(trans != null) - - scanTransform(null, cls, kvp, trans) - } - - /** - * Creates and executes ad-hoc `SCAN` transform query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def scanTransform[T](kvp: KvPred, trans: V => T) - (implicit m: Manifest[V]): Iterable[(K, T)] = { - assert(kvp != null) - assert(trans != null) - - scanTransform(m.erasure.asInstanceOf[Class[V]], kvp, trans) - } - - /** - * Creates and executes ad-hoc `SQL` transform query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @param args Optional list of query arguments. - * @return Collection of cache key-value pairs. - */ - def sqlTransform[T](@Nullable grid: ClusterGroup, cls: Class[_ <: V], clause: String, - trans: V => T, args: Any*): Iterable[(K, T)] = { - assert(cls != null) - assert(clause != null) - assert(trans != null) - assert(args != null) - - // val q = value.cache[K, V]().queries().createSqlQuery(cls, clause) - // - // toScalaItr((if (grid != null) q.projection(grid) else q) - // .execute(toRemoteTransformer[K, V, T](trans), args.asInstanceOf[Seq[Object]]: _*).get) - Iterable.empty - } - - /** - * Creates and executes ad-hoc `SQL` transform query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def sqlTransform[T](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], clause: String, - trans: V => T): Iterable[(K, T)] = { - assert(cls != null) - assert(clause != null) - assert(trans != null) - - sqlTransform(grid, cls, clause, trans, Nil: _*) - } - - /** - * Creates and executes ad-hoc `SQL` transform query on given projection returning its result. + * Creates and executes ad-hoc `SQL` fields query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -992,957 +661,24 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * all results at once without pagination and therefore memory limits should be * taken into account. * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. * @param args Optional list of query arguments. - * @return Collection of cache key-value pairs. + * @return Sequence of sequences of field values. */ - def sqlTransform[T](@Nullable grid: ClusterGroup, clause: String, trans: V => T, args: Any*) - (implicit m: Manifest[V]): Iterable[(K, T)] = { + def sqlFields(clause: String, args: Any*): QueryCursor[JavaList[_]] = { assert(clause != null) - assert(trans != null) assert(args != null) - sqlTransform(grid, m.erasure.asInstanceOf[Class[V]], clause, trans, args: _*) - } - - /** - * Creates and executes ad-hoc `SQL` transform query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @param args Optional list of query arguments. - * @return Collection of cache key-value pairs. - */ - def sqlTransform[T](cls: Class[_ <: V], clause: String, trans: V => T, args: Any*): Iterable[(K, T)] = { - assert(cls != null) - assert(clause != null) - assert(trans != null) - assert(args != null) - - sqlTransform(null, cls, clause, trans, args: _*) - } - - /** - * Creates and executes ad-hoc `SQL` transform query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @param args Optional list of query arguments. - * @return Collection of cache key-value pairs. - */ - def sqlTransform[T](clause: String, trans: V => T, args: Any*) - (implicit m: Manifest[V]): Iterable[(K, T)] = { - assert(clause != null) - assert(trans != null) - assert(args != null) - - sqlTransform(m.erasure.asInstanceOf[Class[V]], clause, trans, args: _*) - } - - /** - * Creates and executes ad-hoc `TEXT` transform query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def textTransform[T](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], clause: String, - trans: V => T): Iterable[(K, T)] = { - assert(cls != null) - assert(clause != null) - assert(trans != null) - - // val q = value.cache[K, V]().queries().createFullTextQuery(cls, clause) - // - // toScalaItr((if (grid != null) q.projection(grid) else q).execute(toRemoteTransformer[K, V, T](trans)).get) - - Iterable.empty - } - - /** - * Creates and executes ad-hoc `TEXT` transform query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def textTransform[T](@Nullable grid: ClusterGroup, clause: String, trans: V => T) - (implicit m: Manifest[V]): Iterable[(K, T)] = { - assert(clause != null) - assert(trans != null) - - textTransform(grid, m.erasure.asInstanceOf[Class[V]], clause, trans) - } - - /** - * Creates and executes ad-hoc `TEXT` transform query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def textTransform[T](cls: Class[_ <: V], clause: String, trans: V => T): Iterable[(K, T)] = { - assert(cls != null) - assert(clause != null) - assert(trans != null) - - textTransform(null, cls, clause, trans) - } - - /** - * Creates and executes ad-hoc `TEXT` transform query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param clause Query text clause. See `CacheQuery` for more details. - * @param trans Transform function that will be applied to each returned value. - * @return Collection of cache key-value pairs. - */ - def textTransform[T](clause: String, trans: V => T) - (implicit m: Manifest[V]): Iterable[(K, T)] = { - assert(clause != null) - assert(trans != null) - - textTransform(m.erasure.asInstanceOf[Class[V]], clause, trans) - } - - /** - * Creates and executes ad-hoc `SCAN` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def scanReduce[R1, R2](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], kvp: KvPred, - rmtRdc: Iterable[(K, V)] => R1, locRdc: Iterable[R1] => R2): R2 = { - assert(cls != null) - assert(kvp != null) - assert(rmtRdc != null) - assert(locRdc != null) - - // val q = value.cache[K, V]().queries().createScanQuery(kvp) - // - // locRdc((if (grid != null) q.projection(grid) else q).execute(toEntryReducer(rmtRdc)).get) - - null.asInstanceOf[R2] - } - - /** - * Creates and executes ad-hoc `SCAN` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def scanReduce[R1, R2](@Nullable grid: ClusterGroup, kvp: KvPred, - rmtRdc: Iterable[(K, V)] => R1, locRdc: Iterable[R1] => R2)(implicit m: Manifest[V]): R2 = { - assert(kvp != null) - assert(rmtRdc != null) - assert(locRdc != null) - - scanReduce(grid, m.erasure.asInstanceOf[Class[V]], kvp, rmtRdc, locRdc) - } - - /** - * Creates and executes ad-hoc `SCAN` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def scanReduce[R1, R2](cls: Class[_ <: V], kvp: KvPred, - rmtRdc: Iterable[(K, V)] => R1, locRdc: Iterable[R1] => R2): R2 = { - assert(cls != null) - assert(kvp != null) - assert(rmtRdc != null) - assert(locRdc != null) - - scanReduce(null, cls, kvp, rmtRdc, locRdc) - } - - /** - * Creates and executes ad-hoc `SCAN` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def scanReduce[R1, R2](kvp: KvPred, rmtRdc: Iterable[(K, V)] => R1, - locRdc: Iterable[R1] => R2)(implicit m: Manifest[V]): R2 = { - assert(kvp != null) - assert(rmtRdc != null) - assert(locRdc != null) - - scanReduce(m.erasure.asInstanceOf[Class[V]], kvp, rmtRdc, locRdc) - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @param args Optional list of query arguments. - * @return Reduced value. - */ - def sqlReduce[R1, R2](@Nullable grid: ClusterGroup, cls: Class[_ <: V], clause: String, - rmtRdc: Iterable[(K, V)] => R1, locRdc: Iterable[R1] => R2, args: Any*): R2 = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - assert(args != null) - - // val q = value.cache[K, V]().queries().createSqlQuery(cls, clause) - // - // locRdc((if (grid != null) q.projection(grid) else q) - // .execute(toEntryReducer(rmtRdc), args.asInstanceOf[Seq[Object]]: _*).get) - - null.asInstanceOf[R2] - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def sqlReduce[R1, R2](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], clause: String, - rmtRdc: Iterable[(K, V)] => R1, locRdc: Iterable[R1] => R2): R2 = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - - sqlReduce(grid, cls, clause, rmtRdc, locRdc, Nil: _*) - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @param args Optional list of query arguments. - * @return Reduced value. - */ - def sqlReduce[R1, R2](@Nullable grid: ClusterGroup, clause: String, rmtRdc: Iterable[(K, V)] => R1, - locRdc: Iterable[R1] => R2, args: Any*)(implicit m: Manifest[V]): R2 = { - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - assert(args != null) - - sqlReduce(grid, m.erasure.asInstanceOf[Class[V]], clause, rmtRdc, locRdc, args: _*) - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @param args Optional list of query arguments. - * @return Reduced value. - */ - def sqlReduce[R1, R2](cls: Class[_ <: V], clause: String, rmtRdc: Iterable[(K, V)] => R1, - locRdc: Iterable[R1] => R2, args: Any*): R2 = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - assert(args != null) - - sqlReduce(null, cls, clause, rmtRdc, locRdc, args: _*) - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @param args Optional list of query arguments. - * @return Reduced value. - */ - def sqlReduce[R1, R2](clause: String, rmtRdc: Iterable[(K, V)] => R1, - locRdc: Iterable[R1] => R2, args: Any*)(implicit m: Manifest[V]): R2 = { - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - assert(args != null) - - sqlReduce(m.erasure.asInstanceOf[Class[V]], clause, rmtRdc, locRdc, args: _*) - } - - /** - * Creates and executes ad-hoc `TEXT` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def textReduce[R1, R2](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], clause: String, - rmtRdc: Iterable[(K, V)] => R1, locRdc: Iterable[R1] => R2): R2 = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - - // val q = value.cache[K, V]().queries().createFullTextQuery(cls, clause) - // - // locRdc((if (grid != null) q.projection(grid) else q).execute(toEntryReducer(rmtRdc)).get) - - null.asInstanceOf[R2] - } - - /** - * Creates and executes ad-hoc `TEXT` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def textReduce[R1, R2](@Nullable grid: ClusterGroup, clause: String, rmtRdc: Iterable[(K, V)] => R1, - locRdc: Iterable[R1] => R2)(implicit m: Manifest[V]): R2 = { - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - - textReduce(grid, m.erasure.asInstanceOf[Class[V]], clause, rmtRdc, locRdc) - } - - /** - * Creates and executes ad-hoc `TEXT` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def textReduce[R1, R2](cls: Class[_ <: V], clause: String, rmtRdc: Iterable[(K, V)] => R1, - locRdc: Iterable[R1] => R2): R2 = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - - textReduce(null, cls, clause, rmtRdc, locRdc) - } - - /** - * Creates and executes ad-hoc `TEXT` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param clause Query text clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param locRdc Reduce function that will be called on local node. - * @return Reduced value. - */ - def textReduce[R1, R2](clause: String, rmtRdc: Iterable[(K, V)] => R1, - locRdc: Iterable[R1] => R2)(implicit m: Manifest[V]): R2 = { - assert(clause != null) - assert(rmtRdc != null) - assert(locRdc != null) - - textReduce(m.erasure.asInstanceOf[Class[V]], clause, rmtRdc, locRdc) - } - - /** - * Creates and executes ad-hoc `SCAN` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def scanReduceRemote[R](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], kvp: KvPred, - rmtRdc: Iterable[(K, V)] => R): Iterable[R] = { - assert(cls != null) - assert(kvp != null) - assert(rmtRdc != null) - - // val q = value.cache[K, V]().queries().createScanQuery(kvp) - // - // (if (grid != null) q.projection(grid) else q).execute(toEntryReducer(rmtRdc)).get - - Iterable.empty - } - - /** - * Creates and executes ad-hoc `SCAN` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the global projection will be used. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def scanReduceRemote[R](@Nullable grid: ClusterGroup, kvp: KvPred, - rmtRdc: Iterable[(K, V)] => R)(implicit m: Manifest[V]): Iterable[R] = { - assert(kvp != null) - assert(rmtRdc != null) - - scanReduceRemote(grid, m.erasure.asInstanceOf[Class[V]], kvp, rmtRdc) - } - - /** - * Creates and executes ad-hoc `SCAN` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def scanReduceRemote[R](cls: Class[_ <: V], kvp: KvPred, rmtRdc: Iterable[(K, V)] => R): Iterable[R] = { - assert(cls != null) - assert(kvp != null) - assert(rmtRdc != null) - - scanReduceRemote(null, cls, kvp, rmtRdc) - } - - /** - * Creates and executes ad-hoc `SCAN` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param kvp Filter to be used prior to returning key-value pairs to user. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def scanReduceRemote[R](kvp: KvPred, rmtRdc: Iterable[(K, V)] => R)(implicit m: Manifest[V]): Iterable[R] = { - assert(kvp != null) - assert(rmtRdc != null) - - scanReduceRemote(m.erasure.asInstanceOf[Class[V]], kvp, rmtRdc) - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param args Optional list of query arguments. - * @return Collection of reduced values. - */ - def sqlReduceRemote[R](@Nullable grid: ClusterGroup, cls: Class[_ <: V], clause: String, - rmtRdc: Iterable[(K, V)] => R, args: Any*): Iterable[R] = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - assert(args != null) - - // val q = value.cache[K, V]().queries().createSqlQuery(cls, clause) - // - // (if (grid != null) q.projection(grid) else q) - // .execute(toEntryReducer(rmtRdc), args.asInstanceOf[Seq[Object]]: _*).get - - Iterable.empty - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def sqlReduceRemote[R](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], clause: String, - rmtRdc: Iterable[(K, V)] => R): Iterable[R] = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - - sqlReduceRemote(grid, cls, clause, rmtRdc, Nil: _*) - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param args Optional list of query arguments. - * @return Collection of reduced values. - */ - def sqlReduceRemote[R](@Nullable grid: ClusterGroup, clause: String, rmtRdc: Iterable[(K, V)] => R, - args: Any*)(implicit m: Manifest[V]): Iterable[R] = { - assert(clause != null) - assert(rmtRdc != null) - assert(args != null) - - sqlReduceRemote(grid, m.erasure.asInstanceOf[Class[V]], clause, rmtRdc, args: _*) - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param args Optional list of query arguments. - * @return Collection of reduced values. - */ - def sqlReduceRemote[R](cls: Class[_ <: V], clause: String, rmtRdc: Iterable[(K, V)] => R, - args: Any*): Iterable[R] = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - assert(args != null) - - sqlReduceRemote(null, cls, clause, rmtRdc, args: _*) - } - - /** - * Creates and executes ad-hoc `SQL` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @param args Optional list of query arguments. - * @return Collection of reduced values. - */ - def sqlReduceRemote[R](clause: String, rmtRdc: Iterable[(K, V)] => R, args: Any*) - (implicit m: Manifest[V]): Iterable[R] = { - assert(clause != null) - assert(rmtRdc != null) - assert(args != null) - - sqlReduceRemote(m.erasure.asInstanceOf[Class[V]], clause, rmtRdc, args: _*) - } - - /** - * Creates and executes ad-hoc `TEXT` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def textReduceRemote[R](@Nullable grid: ClusterGroup = null, cls: Class[_ <: V], clause: String, - rmtRdc: Iterable[(K, V)] => R): Iterable[R] = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - - // val q = value.cache[K, V]().queries().createFullTextQuery(cls, clause) - // - // (if (grid != null) q.projection(grid) else q).execute(toEntryReducer(rmtRdc)).get - Iterable.empty - } - - /** - * Creates and executes ad-hoc `TEXT` reduce query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param grid Grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def textReduceRemote[R](@Nullable grid: ClusterGroup, clause: String, rmtRdc: Iterable[(K, V)] => R) - (implicit m: Manifest[V]): Iterable[R] = { - assert(clause != null) - assert(rmtRdc != null) - - textReduceRemote(grid, m.erasure.asInstanceOf[Class[V]], clause, rmtRdc) - } - - /** - * Creates and executes ad-hoc `TEXT` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param cls Query values class. Since cache can, in general, contain values of any subtype of `V` - * query needs to know the exact type it should operate on. - * @param clause Query text clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def textReduceRemote[R](cls: Class[_ <: V], clause: String, - rmtRdc: Iterable[(K, V)] => R): Iterable[R] = { - assert(cls != null) - assert(clause != null) - assert(rmtRdc != null) - - textReduceRemote(null, cls, clause, rmtRdc) - } - - /** - * Creates and executes ad-hoc `TEXT` reduce query on global projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * Note that query value class will be taken implicitly as exact type `V` of this - * cache projection. - * - * @param clause Query text clause. See `CacheQuery` for more details. - * @param rmtRdc Reduce function that will be called on each remote node. - * @return Collection of reduced values. - */ - def textReduceRemote[R](clause: String, rmtRdc: Iterable[(K, V)] => R) - (implicit m: Manifest[V]): Iterable[R] = { - assert(clause != null) - assert(rmtRdc != null) - - textReduceRemote(m.erasure.asInstanceOf[Class[V]], clause, rmtRdc) - } - - /** - * Creates and executes ad-hoc `SQL` fields query on given projection returning its result. - * - * Note that if query is executed more than once (potentially with different - * arguments) it is more performant to create query via standard mechanism - * and execute it multiple times with different arguments. The analogy is - * similar to JDBC `PreparedStatement`. Note also that this function will return - * all results at once without pagination and therefore memory limits should be - * taken into account. - * - * @param grid Optional grid projection on which this query will be executed. If `null` the - * global projection will be used. - * @param clause Query SQL clause. See `CacheQuery` for more details. - * @param args Optional list of query arguments. - * @return Sequence of sequences of field values. - */ - def sqlFields(@Nullable grid: ClusterGroup, clause: String, args: Any*): IndexedSeq[IndexedSeq[Any]] = { - assert(clause != null) - assert(args != null) + val query = new SqlFieldsQuery(clause) - // val q = value.cache[K, V]().queries().createSqlFieldsQuery(clause) - // - // (if (grid != null) q.projection(grid) else q).execute(args.asInstanceOf[Seq[Object]]: _*) - // .get.toIndexedSeq.map((s: java.util.List[_]) => s.toIndexedSeq) + if (args != null && args.nonEmpty) + query.setArgs(args.map(_.asInstanceOf[AnyRef]) : _*) - IndexedSeq.empty + value.queryFields(query) } /** - * Creates and executes ad-hoc `SQL` no-arg fields query on given projection returning its result. + * Creates and executes ad-hoc `SQL` no-arg fields query returning its result. * * Note that if query is executed more than once (potentially with different * arguments) it is more performant to create query via standard mechanism @@ -1951,14 +687,12 @@ with Iterable[Cache.Entry[K, V]] with Ordered[IgniteCache[K, V]] { * all results at once without pagination and therefore memory limits should be * taken into account. * - * @param grid Optional grid projection on which this query will be executed. If `null` the - * global projection will be used. * @param clause Query SQL clause. See `CacheQuery` for more details. * @return Sequence of sequences of field values. */ - def sqlFields(@Nullable grid: ClusterGroup = null, clause: String): IndexedSeq[IndexedSeq[Any]] = { + def sqlFields(clause: String): QueryCursor[JavaList[_]] = { assert(clause != null) - sqlFields(grid, clause, Nil: _*) + sqlFields(clause, Nil:_*) } }