You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/08/17 16:51:48 UTC

[01/19] ignite git commit: IGNITE-5586 Toggle cluster active state.

Repository: ignite
Updated Branches:
  refs/heads/ignite-5901 88096ed1e -> ed1da133d


IGNITE-5586 Toggle cluster active state.


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

Branch: refs/heads/ignite-5901
Commit: 5283e19fe21c5b898965ade70cc86e9c64254388
Parents: c5adf46
Author: vsisko <vs...@gridgain.com>
Authored: Thu Aug 17 14:55:08 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Aug 17 14:55:08 2017 +0700

----------------------------------------------------------------------
 .../commands/top/VisorTopologyCommand.scala     | 111 ++++++++++++-------
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |   9 +-
 .../top/VisorActivationCommandSpec.scala        |  67 +++++++++++
 .../testsuites/VisorConsoleSelfTestSuite.scala  |   3 +-
 4 files changed, 146 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5283e19f/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
index 63b6d2b..b75afc6 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/top/VisorTopologyCommand.scala
@@ -48,6 +48,8 @@ import scala.util.control.Breaks._
  * ====Specification====
  * {{{
  *     top "{-c1=e1<num> -c2=e2<num> ... -ck=ek<num>} {-h=<host1> ... -h=<hostk>} {-a}"
+ *     top -activate
+ *     top -deactivate
  * }}}
  *
  * ====Arguments====
@@ -87,6 +89,10 @@ import scala.util.control.Breaks._
  *     -a
  *         This defines whether to show a separate table of nodes
  *         with detail per-node information.
+ *     -activate
+ *         Activate cluster.
+ *     -deactivate
+ *         Deactivate cluster.
  * }}}
  *
  * ====Examples====
@@ -99,6 +105,10 @@ import scala.util.control.Breaks._
  *         Prints topology for provided hosts.
  *     top
  *         Prints full topology.
+ *     top -activate
+ *         Activate cluster.
+ *     top -deactivate
+ *         Deactivate cluster.
  * }}}
  */
 class VisorTopologyCommand extends VisorConsoleCommand {
@@ -140,47 +150,53 @@ class VisorTopologyCommand extends VisorConsoleCommand {
         else {
             val argLst = parseArgs(args)
 
-            val hosts = argLst.filter(_._1 == "h").map((a: Arg) =>
-                try
-                    InetAddress.getByName(a._2).getHostAddress
+            if (hasArgFlag("activate", argLst))
+                ignite.active(true)
+            else if (hasArgFlag("deactivate", argLst))
+                ignite.active(false)
+            else {
+                val hosts = argLst.filter(_._1 == "h").map((a: Arg) =>
+                    try
+                        InetAddress.getByName(a._2).getHostAddress
+                    catch {
+                        case e: UnknownHostException => scold("Unknown host: " + a._2).^^
+
+                            "" // Never happens.
+                    }
+                ).filter(!_.isEmpty).toSet
+
+                val all = hasArgFlag("a", argLst)
+
+                var f: NodeFilter = (ClusterNode) => true
+
+                try {
+                    argLst foreach (arg => {
+                        val (n, v) = arg
+
+                        n match {
+                            case "cc" if v != null => f = make(v, f, _.metrics.getTotalCpus)
+                            case "cl" if v != null => f = make(v, f, (n: ClusterNode) =>
+                                (n.metrics.getCurrentCpuLoad * 100).toLong)
+                            case "aj" if v != null => f = make(v, f, _.metrics.getCurrentActiveJobs)
+                            case "cj" if v != null => f = make(v, f, _.metrics.getCurrentCancelledJobs)
+                            case "tc" if v != null => f = make(v, f, _.metrics.getCurrentThreadCount)
+                            case "ut" if v != null => f = make(v, f, _.metrics.getUpTime)
+                            case "je" if v != null => f = make(v, f, _.metrics.getCurrentJobExecuteTime)
+                            case "jw" if v != null => f = make(v, f, _.metrics.getCurrentJobWaitTime)
+                            case "wj" if v != null => f = make(v, f, _.metrics.getCurrentWaitingJobs)
+                            case "rj" if v != null => f = make(v, f, _.metrics.getCurrentRejectedJobs)
+                            case "hu" if v != null => f = make(v, f, _.metrics.getHeapMemoryUsed)
+                            case "hm" if v != null => f = make(v, f, _.metrics.getHeapMemoryMaximum)
+                            case _ => ()
+                        }
+                    })
+
+                    show(n => f(n), hosts, all)
+                }
                 catch {
-                    case e: UnknownHostException => scold("Unknown host: " + a._2).^^
-
-                    "" // Never happens.
+                    case e: NumberFormatException => scold(e)
+                    case e: IgniteException => scold(e)
                 }
-            ).filter(!_.isEmpty).toSet
-
-            val all = hasArgFlag("a", argLst)
-
-            var f: NodeFilter = (ClusterNode) => true
-
-            try {
-                argLst foreach (arg => {
-                    val (n, v) = arg
-
-                    n match {
-                        case "cc" if v != null => f = make(v, f, _.metrics.getTotalCpus)
-                        case "cl" if v != null => f = make(v, f, (n: ClusterNode) =>
-                            (n.metrics.getCurrentCpuLoad * 100).toLong)
-                        case "aj" if v != null => f = make(v, f, _.metrics.getCurrentActiveJobs)
-                        case "cj" if v != null => f = make(v, f, _.metrics.getCurrentCancelledJobs)
-                        case "tc" if v != null => f = make(v, f, _.metrics.getCurrentThreadCount)
-                        case "ut" if v != null => f = make(v, f, _.metrics.getUpTime)
-                        case "je" if v != null => f = make(v, f, _.metrics.getCurrentJobExecuteTime)
-                        case "jw" if v != null => f = make(v, f, _.metrics.getCurrentJobWaitTime)
-                        case "wj" if v != null => f = make(v, f, _.metrics.getCurrentWaitingJobs)
-                        case "rj" if v != null => f = make(v, f, _.metrics.getCurrentRejectedJobs)
-                        case "hu" if v != null => f = make(v, f, _.metrics.getHeapMemoryUsed)
-                        case "hm" if v != null => f = make(v, f, _.metrics.getHeapMemoryMaximum)
-                        case _ => ()
-                    }
-                })
-
-                show(n => f(n), hosts, all)
-            }
-            catch {
-                case e: NumberFormatException => scold(e)
-                case e: IgniteException => scold(e)
             }
         }
     }
@@ -322,6 +338,7 @@ class VisorTopologyCommand extends VisorConsoleCommand {
 
         val sumT = VisorTextTable()
 
+        sumT += ("Active", ignite.active)
         sumT += ("Total hosts", U.neighborhood(nodes).size)
         sumT += ("Total nodes", m.getTotalNodes)
         sumT += ("Total CPUs", m.getTotalCpus)
@@ -348,7 +365,9 @@ object VisorTopologyCommand {
         name = "top",
         shortInfo = "Prints current topology.",
         spec = List(
-            "top {-c1=e1<num> -c2=e2<num> ... -ck=ek<num>} {-h=<host1> ... -h=<hostk>} {-a}"
+            "top {-c1=e1<num> -c2=e2<num> ... -ck=ek<num>} {-h=<host1> ... -h=<hostk>} {-a}",
+            "top -activate",
+            "top -deactivate"
         ),
         args = List(
             "-ck=ek<num>" -> List(
@@ -388,6 +407,12 @@ object VisorTopologyCommand {
             "-a" -> List(
                 "This defines whether to show a separate table of nodes",
                 "with detail per-node information."
+            ),
+            "-activate" -> List(
+                "Activate cluster."
+            ),
+            "-deactivate" -> List(
+                "Deactivate cluster."
             )
         ),
         examples = List(
@@ -398,7 +423,11 @@ object VisorTopologyCommand {
             "top -h=10.34.2.122 -h=10.65.3.11" ->
                 "Prints topology for provided hosts.",
             "top" ->
-                "Prints full topology."
+                "Prints full topology.",
+            "top -activate" ->
+                "Activate cluster.",
+            "top -deactivate" ->
+                "Deactivate cluster."
         ),
         emptyArgs = cmd.top,
         withArgs = cmd.top

http://git-wip-us.apache.org/repos/asf/ignite/blob/5283e19f/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala
index 38db760..f5d6e56 100644
--- a/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala
+++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/VisorRuntimeBaseSpec.scala
@@ -20,8 +20,8 @@ package org.apache.ignite.visor
 import org.apache.ignite.Ignition
 import org.apache.ignite.configuration.IgniteConfiguration
 import org.apache.ignite.visor.commands.open.VisorOpenCommand._
-
 import org.scalatest._
+import VisorRuntimeBaseSpec._
 
 /**
  * Base abstract class for unit tests requiring Visor runtime.
@@ -45,7 +45,7 @@ abstract class VisorRuntimeBaseSpec(private[this] val num: Int) extends FunSpec
     }
 
     protected def openVisor() {
-        visor.open(config("visor-demo-node"), "n/a")
+        visor.open(config(VISOR_INSTANCE_NAME), "n/a")
     }
 
     protected def closeVisorQuiet() {
@@ -75,3 +75,8 @@ abstract class VisorRuntimeBaseSpec(private[this] val num: Int) extends FunSpec
         closeVisorQuiet()
     }
 }
+
+/** Singleton companion object. */
+object VisorRuntimeBaseSpec {
+    val VISOR_INSTANCE_NAME = "visor-demo-node"
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5283e19f/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/top/VisorActivationCommandSpec.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/top/VisorActivationCommandSpec.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/top/VisorActivationCommandSpec.scala
new file mode 100644
index 0000000..fb2a7f4
--- /dev/null
+++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/commands/top/VisorActivationCommandSpec.scala
@@ -0,0 +1,67 @@
+/*
+ * 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.visor.commands.top
+
+import org.apache.ignite.Ignition
+import org.apache.ignite.configuration.{IgniteConfiguration, MemoryConfiguration, PersistentStoreConfiguration}
+import org.apache.ignite.visor.commands.top.VisorTopologyCommand._
+import org.apache.ignite.visor.{VisorRuntimeBaseSpec, visor}
+import VisorRuntimeBaseSpec._
+
+/**
+ * Unit test for cluster activation commands.
+ */
+class VisorActivationCommandSpec extends VisorRuntimeBaseSpec(2) {
+    override protected def config(name: String): IgniteConfiguration = {
+        val cfg = super.config(name)
+
+        val memCfg = new MemoryConfiguration
+        memCfg.setDefaultMemoryPolicySize(10 * 1024 * 1024)
+
+        cfg.setMemoryConfiguration(memCfg)
+        cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration)
+
+        cfg
+    }
+
+    describe("A 'top' visor command for cluster activation") {
+        it("should activate cluster") {
+            assert(!Ignition.ignite(VISOR_INSTANCE_NAME).active())
+
+            visor.top()
+
+            visor.top("-activate")
+
+            visor.top()
+
+            assert(Ignition.ignite(VISOR_INSTANCE_NAME).active())
+        }
+
+        it("should deactivate cluster") {
+            assert(Ignition.ignite(VISOR_INSTANCE_NAME).active())
+
+            visor.top()
+
+            visor.top("-deactivate")
+
+            visor.top()
+
+            assert(!Ignition.ignite(VISOR_INSTANCE_NAME).active())
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5283e19f/modules/visor-console/src/test/scala/org/apache/ignite/visor/testsuites/VisorConsoleSelfTestSuite.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/test/scala/org/apache/ignite/visor/testsuites/VisorConsoleSelfTestSuite.scala b/modules/visor-console/src/test/scala/org/apache/ignite/visor/testsuites/VisorConsoleSelfTestSuite.scala
index f23a7c9..5fbc8a0 100644
--- a/modules/visor-console/src/test/scala/org/apache/ignite/visor/testsuites/VisorConsoleSelfTestSuite.scala
+++ b/modules/visor-console/src/test/scala/org/apache/ignite/visor/testsuites/VisorConsoleSelfTestSuite.scala
@@ -39,7 +39,7 @@ import org.apache.ignite.visor.commands.open.VisorOpenCommandSpec
 import org.apache.ignite.visor.commands.ping.VisorPingCommandSpec
 import org.apache.ignite.visor.commands.start.VisorStartCommandSpec
 import org.apache.ignite.visor.commands.tasks.VisorTasksCommandSpec
-import org.apache.ignite.visor.commands.top.VisorTopologyCommandSpec
+import org.apache.ignite.visor.commands.top.{VisorActivationCommandSpec, VisorTopologyCommandSpec}
 import org.junit.runner.RunWith
 import org.scalatest.Suites
 import org.scalatest.junit.JUnitRunner
@@ -69,6 +69,7 @@ class VisorConsoleSelfTestSuite extends Suites (
     new VisorStartCommandSpec,
     new VisorTasksCommandSpec,
     new VisorTopologyCommandSpec,
+    new VisorActivationCommandSpec,
     new VisorArgListSpec
 ) {
     // Mimic GridTestUtils.getNextMulticastGroup behavior because it can't be imported here


[18/19] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-5901

Posted by ag...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-5901


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

Branch: refs/heads/ignite-5901
Commit: f7aa561fd55c0ece8b7372a2a91c1831629dfd24
Parents: 88096ed 5f19c1b
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Aug 17 19:46:23 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 19:46:23 2017 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcDeleteStatementSelfTest.java      |   22 +
 .../jdbc2/JdbcInsertStatementSelfTest.java      |  159 ++
 .../jdbc2/JdbcMergeStatementSelfTest.java       |   41 +
 .../jdbc2/JdbcStatementBatchingSelfTest.java    |  133 ++
 .../jdbc2/JdbcUpdateStatementSelfTest.java      |   24 +
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    3 +
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  337 +++-
 .../ignite/cache/query/SqlFieldsQuery.java      |   38 +-
 .../configuration/CheckpointWriteOrder.java     |   33 +
 .../PersistentStoreConfiguration.java           |   26 +
 .../apache/ignite/internal/IgniteKernal.java    |    8 +
 .../ignite/internal/IgniteNodeAttributes.java   |    3 +
 .../internal/jdbc/thin/JdbcThinConnection.java  |   20 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     | 1587 ++++++++++++++++++
 .../jdbc/thin/JdbcThinParameterMetadata.java    |  115 ++
 .../jdbc/thin/JdbcThinPreparedStatement.java    |   27 +-
 .../internal/jdbc/thin/JdbcThinResultSet.java   |   47 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   |    4 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |  117 +-
 .../internal/jdbc2/JdbcBatchUpdateTask.java     |  215 +++
 .../internal/jdbc2/JdbcDatabaseMetadata.java    |    2 +-
 .../internal/jdbc2/JdbcPreparedStatement.java   |   25 +-
 .../ignite/internal/jdbc2/JdbcStatement.java    |   59 +-
 .../processors/cache/GridCacheProcessor.java    |   21 +
 .../dht/GridDhtPartitionTopologyImpl.java       |   16 +
 .../GridCacheDatabaseSharedManager.java         |  106 +-
 .../persistence/pagemem/PageMemoryImpl.java     |   80 +-
 .../IgniteTxImplicitSingleStateImpl.java        |    5 +-
 .../processors/odbc/SqlListenerNioListener.java |    9 +-
 .../odbc/SqlListenerRequestHandler.java         |    9 +
 .../odbc/jdbc/JdbcBatchExecuteRequest.java      |   20 +-
 .../odbc/jdbc/JdbcBatchExecuteResult.java       |    6 +
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |   75 +-
 .../processors/odbc/jdbc/JdbcIndexMeta.java     |  192 +++
 .../odbc/jdbc/JdbcMetaColumnsRequest.java       |  102 ++
 .../odbc/jdbc/JdbcMetaColumnsResult.java        |   99 ++
 .../odbc/jdbc/JdbcMetaIndexesRequest.java       |   88 +
 .../odbc/jdbc/JdbcMetaIndexesResult.java        |   98 ++
 .../odbc/jdbc/JdbcMetaParamsRequest.java        |   87 +
 .../odbc/jdbc/JdbcMetaParamsResult.java         |   97 ++
 .../odbc/jdbc/JdbcMetaPrimaryKeysRequest.java   |   88 +
 .../odbc/jdbc/JdbcMetaPrimaryKeysResult.java    |   99 ++
 .../odbc/jdbc/JdbcMetaSchemasRequest.java       |   73 +
 .../odbc/jdbc/JdbcMetaSchemasResult.java        |   73 +
 .../odbc/jdbc/JdbcMetaTablesRequest.java        |   87 +
 .../odbc/jdbc/JdbcMetaTablesResult.java         |   97 ++
 .../processors/odbc/jdbc/JdbcParameterMeta.java |  163 ++
 .../odbc/jdbc/JdbcPrimaryKeyMeta.java           |  131 ++
 .../odbc/jdbc/JdbcQueryCloseRequest.java        |    4 +-
 .../odbc/jdbc/JdbcQueryExecuteRequest.java      |    8 +-
 .../odbc/jdbc/JdbcQueryExecuteResult.java       |   12 +-
 .../odbc/jdbc/JdbcQueryFetchRequest.java        |    4 +-
 .../odbc/jdbc/JdbcQueryFetchResult.java         |   12 +-
 .../odbc/jdbc/JdbcQueryMetadataRequest.java     |   18 +-
 .../odbc/jdbc/JdbcQueryMetadataResult.java      |   14 +-
 .../processors/odbc/jdbc/JdbcRequest.java       |   67 +-
 .../odbc/jdbc/JdbcRequestHandler.java           |  273 ++-
 .../processors/odbc/jdbc/JdbcResult.java        |   58 +-
 .../processors/odbc/jdbc/JdbcTableMeta.java     |   82 +
 .../processors/odbc/jdbc/JdbcUtils.java         |   37 +-
 .../odbc/odbc/OdbcRequestHandler.java           |    6 +
 .../processors/query/GridQueryProcessor.java    |    2 +-
 .../query/GridQueryTypeDescriptor.java          |    7 +
 .../query/QueryTypeDescriptorImpl.java          |   15 +
 .../internal/processors/query/QueryUtils.java   |    7 +-
 ...dMemoryConfigurationConsistencySelfTest.java |   79 +
 ...nitePersistenceSequentialCheckpointTest.java |   44 +
 .../IgnitePersistentStoreCacheGroupsTest.java   |   31 +-
 .../db/wal/crc/IgniteDataIntegrityTests.java    |   24 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |    2 +
 .../query/h2/DmlStatementsProcessor.java        |  274 +--
 .../processors/query/h2/IgniteH2Indexing.java   |   57 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  237 ++-
 .../h2/twostep/GridReduceQueryExecutor.java     |    7 +-
 .../query/h2/twostep/MapNodeResults.java        |   19 +-
 .../query/h2/twostep/MapQueryLazyWorker.java    |  176 ++
 .../query/h2/twostep/MapQueryLazyWorkerKey.java |   97 ++
 .../query/h2/twostep/MapQueryResult.java        |   46 +-
 .../query/h2/twostep/MapQueryResults.java       |   26 +-
 .../query/h2/twostep/MapRequestKey.java         |   23 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |    9 +
 .../DynamicIndexAbstractConcurrentSelfTest.java |  115 +-
 .../processors/query/LazyQuerySelfTest.java     |  389 +++++
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   18 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    2 +
 .../commands/top/VisorTopologyCommand.scala     |  111 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |    9 +-
 .../top/VisorActivationCommandSpec.scala        |   67 +
 .../testsuites/VisorConsoleSelfTestSuite.scala  |    3 +-
 89 files changed, 6953 insertions(+), 504 deletions(-)
----------------------------------------------------------------------



[16/19] ignite git commit: IGNITE-6096 Fixed races on partition evict - Fixes #2463.

Posted by ag...@apache.org.
IGNITE-6096 Fixed races on partition evict - Fixes #2463.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-5901
Commit: b3ae58eccb35369041342e4a5d9bb5f661417d41
Parents: 136075a
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Thu Aug 17 18:18:05 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 18:29:04 2017 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java           | 16 ++++++++++++++++
 .../persistence/GridCacheDatabaseSharedManager.java | 11 +++++++----
 .../cache/persistence/pagemem/PageMemoryImpl.java   |  9 ++++++++-
 3 files changed, 31 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b3ae58ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index d7a224c..0eff7fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -660,6 +660,15 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         GridDhtLocalPartition loc = locParts.get(p);
 
         if (loc == null || loc.state() == EVICTED) {
+            if (loc != null) {
+                try {
+                    loc.rent(false).get();
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+
             locParts.set(p, loc = new GridDhtLocalPartition(ctx, grp, p));
 
             T2<Long, Long> cntr = cntrMap.get(p);
@@ -741,6 +750,13 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             boolean belongs = partitionLocalNode(p, topVer);
 
             if (loc != null && state == EVICTED) {
+                try {
+                    loc.rent(false).get();
+                }
+                catch (IgniteCheckedException ex) {
+                    throw new IgniteException(ex);
+                }
+
                 locParts.set(p, loc = null);
 
                 if (!belongs)

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3ae58ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 19af87b..3ef9e61 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -1549,6 +1549,13 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     if (storeMgr.pages(grpId, i) <= 1)
                         continue;
 
+                    GridDhtLocalPartition part = grp.topology().forceCreatePartition(i);
+
+                    assert part != null;
+
+                    // TODO: https://issues.apache.org/jira/browse/IGNITE-6097
+                    grp.offheap().onPartitionInitialCounterUpdated(i, 0);
+
                     long partMetaId = pageMem.partitionMetaPageId(grpId, i);
                     long partMetaPage = pageMem.acquirePage(grpId, partMetaId);
                     try {
@@ -1561,10 +1568,6 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                             T2<Integer, Long> fromWal = partStates.get(new T2<>(grpId, i));
 
-                            GridDhtLocalPartition part = grp.topology().forceCreatePartition(i);
-
-                            assert part != null;
-
                             if (fromWal != null) {
                                 int stateId = fromWal.get1();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b3ae58ec/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index b6e5f46..dbb64f8 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -640,13 +640,14 @@ public class PageMemoryImpl implements PageMemoryEx {
 
         GridUnsafe.setMemory(absPtr + PAGE_OVERHEAD, pageSize(), (byte)0);
 
+        PageHeader.dirty(absPtr, false);
+
         long tmpBufPtr = PageHeader.tempBufferPointer(absPtr);
 
         if (tmpBufPtr != INVALID_REL_PTR) {
             GridUnsafe.setMemory(checkpointPool.absolute(tmpBufPtr) + PAGE_OVERHEAD, pageSize(), (byte)0);
 
             PageHeader.tempBufferPointer(absPtr, INVALID_REL_PTR);
-            PageHeader.dirty(absPtr, false);
 
             // We pinned the page when allocated the temp buffer, release it now.
             PageHeader.releasePage(absPtr);
@@ -657,6 +658,12 @@ public class PageMemoryImpl implements PageMemoryEx {
         if (rmv)
             seg.loadedPages.remove(cacheId, PageIdUtils.effectivePageId(pageId), tag);
 
+        if (seg.segCheckpointPages != null)
+            seg.segCheckpointPages.remove(new FullPageId(pageId, cacheId));
+
+        if (seg.dirtyPages != null)
+            seg.dirtyPages.remove(new FullPageId(pageId, cacheId));
+
         return relPtr;
     }
 


[15/19] ignite git commit: IGNITE-5991: SQL: Lazy query execution. This closes #2437.

Posted by ag...@apache.org.
IGNITE-5991: SQL: Lazy query execution. This closes #2437.


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

Branch: refs/heads/ignite-5901
Commit: 136075ae0f7070999dec6913afc8cef1a26eb307
Parents: 15710a8
Author: devozerov <vo...@gridgain.com>
Authored: Thu Aug 17 18:24:34 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 17 18:24:34 2017 +0300

----------------------------------------------------------------------
 .../ignite/cache/query/SqlFieldsQuery.java      |  38 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  57 ++-
 .../query/h2/twostep/GridMapQueryExecutor.java  | 237 ++++++++---
 .../h2/twostep/GridReduceQueryExecutor.java     |   7 +-
 .../query/h2/twostep/MapNodeResults.java        |  19 +-
 .../query/h2/twostep/MapQueryLazyWorker.java    | 176 +++++++++
 .../query/h2/twostep/MapQueryLazyWorkerKey.java |  97 +++++
 .../query/h2/twostep/MapQueryResult.java        |  46 ++-
 .../query/h2/twostep/MapQueryResults.java       |  26 +-
 .../query/h2/twostep/MapRequestKey.java         |  23 +-
 .../h2/twostep/msg/GridH2QueryRequest.java      |   9 +
 .../processors/query/LazyQuerySelfTest.java     | 389 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 13 files changed, 1041 insertions(+), 85 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
index 2838fe3..54f8396 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
@@ -71,6 +71,9 @@ public class SqlFieldsQuery extends Query<List<?>> {
     /** */
     private boolean replicatedOnly;
 
+    /** */
+    private boolean lazy;
+
     /** Partitions for query */
     private int[] parts;
 
@@ -230,7 +233,7 @@ public class SqlFieldsQuery extends Query<List<?>> {
     /**
      * Check if distributed joins are enabled for this query.
      *
-     * @return {@code true} If distributed joind enabled.
+     * @return {@code true} If distributed joins enabled.
      */
     public boolean isDistributedJoins() {
         return distributedJoins;
@@ -269,6 +272,39 @@ public class SqlFieldsQuery extends Query<List<?>> {
     }
 
     /**
+     * Sets lazy query execution flag.
+     * <p>
+     * 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.
+     * <p>
+     * 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.
+     * <p>
+     * Defaults to {@code false}, meaning that the whole result set is fetched to memory eagerly.
+     *
+     * @param lazy Lazy query execution flag.
+     * @return {@code this} For chaining.
+     */
+    public SqlFieldsQuery setLazy(boolean lazy) {
+        this.lazy = lazy;
+
+        return this;
+    }
+
+    /**
+     * Gets lazy query execution flag.
+     * <p>
+     * See {@link #setLazy(boolean)} for more information.
+     *
+     * @return Lazy flag.
+     */
+    public boolean isLazy() {
+        return lazy;
+    }
+
+    /**
      * Gets partitions for query, in ascending order.
      */
     @Nullable public int[] getPartitions() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 007eeb1..6896f18 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -111,6 +111,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridReduceQueryExecutor;
+import org.apache.ignite.internal.processors.query.h2.twostep.MapQueryLazyWorker;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
@@ -139,6 +140,7 @@ import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.Prepared;
 import org.h2.command.dml.Insert;
+import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
 import org.h2.jdbc.JdbcPreparedStatement;
@@ -905,24 +907,32 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @throws IgniteCheckedException If failed.
      */
     private ResultSet executeSqlQuery(final Connection conn, final PreparedStatement stmt,
-        int timeoutMillis, @Nullable GridQueryCancel cancel)
-        throws IgniteCheckedException {
+        int timeoutMillis, @Nullable GridQueryCancel cancel) throws IgniteCheckedException {
+        final MapQueryLazyWorker lazyWorker = MapQueryLazyWorker.currentWorker();
 
         if (cancel != null) {
             cancel.set(new Runnable() {
                 @Override public void run() {
-                    try {
-                        stmt.cancel();
-                    }
-                    catch (SQLException ignored) {
-                        // No-op.
+                    if (lazyWorker != null) {
+                        lazyWorker.submit(new Runnable() {
+                            @Override public void run() {
+                                cancelStatement(stmt);
+                            }
+                        });
                     }
+                    else
+                        cancelStatement(stmt);
                 }
             });
         }
 
+        Session ses = H2Utils.session(conn);
+
         if (timeoutMillis > 0)
-            H2Utils.session(conn).setQueryTimeout(timeoutMillis);
+            ses.setQueryTimeout(timeoutMillis);
+
+        if (lazyWorker != null)
+            ses.setLazyQueryExecution(true);
 
         try {
             return stmt.executeQuery();
@@ -936,7 +946,24 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
         finally {
             if (timeoutMillis > 0)
-                H2Utils.session(conn).setQueryTimeout(0);
+                ses.setQueryTimeout(0);
+
+            if (lazyWorker != null)
+                ses.setLazyQueryExecution(false);
+        }
+    }
+
+    /**
+     * Cancel prepared statement.
+     *
+     * @param stmt Statement.
+     */
+    private static void cancelStatement(PreparedStatement stmt) {
+        try {
+            stmt.cancel();
+        }
+        catch (SQLException ignored) {
+            // No-op.
         }
     }
 
@@ -1143,6 +1170,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param keepCacheObj Flag to keep cache object.
      * @param enforceJoinOrder Enforce join order of tables.
      * @param parts Partitions.
+     * @param lazy Lazy query execution flag.
      * @return Iterable result.
      */
     private Iterable<List<?>> runQueryTwoStep(
@@ -1153,12 +1181,13 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         final int timeoutMillis,
         final GridQueryCancel cancel,
         final Object[] params,
-        final int[] parts
+        final int[] parts,
+        final boolean lazy
     ) {
         return new Iterable<List<?>>() {
             @Override public Iterator<List<?>> iterator() {
                 return rdcQryExec.query(schemaName, qry, keepCacheObj, enforceJoinOrder, timeoutMillis, cancel, params,
-                    parts);
+                    parts, lazy);
             }
         };
     }
@@ -1402,7 +1431,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
             runQueryTwoStep(schemaName, twoStepQry, keepBinary, enforceJoinOrder, qry.getTimeout(), cancel,
-                qry.getArgs(), partitions), cancel);
+                qry.getArgs(), partitions, qry.isLazy()), cancel);
 
         cursor.fieldsMeta(meta);
 
@@ -2070,6 +2099,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         if (log.isDebugEnabled())
             log.debug("Stopping cache query index...");
 
+        mapQryExec.cancelLazyWorkers();
+
 //        unregisterMBean(); TODO https://issues.apache.org/jira/browse/IGNITE-2139
         if (ctx != null && !ctx.cache().context().database().persistenceEnabled()) {
             for (H2Schema schema : schemas.values())
@@ -2355,6 +2386,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
     /** {@inheritDoc} */
     @Override public void cancelAllQueries() {
+        mapQryExec.cancelLazyWorkers();
+
         for (Connection conn : conns)
             U.close(conn, log);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index ca978e2..0cc4172 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -28,7 +28,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicBoolean;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -52,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridReservabl
 import org.apache.ignite.internal.processors.cache.query.CacheQueryType;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
-import org.apache.ignite.internal.processors.cache.query.QueryTable;
 import org.apache.ignite.internal.processors.query.h2.H2Utils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.DistributedJoinMode;
@@ -69,6 +70,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.thread.IgniteThread;
 import org.h2.jdbc.JdbcResultSet;
 import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
@@ -107,6 +109,15 @@ public class GridMapQueryExecutor {
     /** */
     private final ConcurrentMap<MapReservationKey, GridReservable> reservations = new ConcurrentHashMap8<>();
 
+    /** Lazy workers. */
+    private final ConcurrentHashMap<MapQueryLazyWorkerKey, MapQueryLazyWorker> lazyWorkers = new ConcurrentHashMap<>();
+
+    /** Busy lock for lazy workers. */
+    private final GridSpinBusyLock lazyWorkerBusyLock = new GridSpinBusyLock();
+
+    /** Lazy worker stop guard. */
+    private final AtomicBoolean lazyWorkerStopGuard = new AtomicBoolean();
+
     /**
      * @param busyLock Busy lock.
      */
@@ -162,6 +173,21 @@ public class GridMapQueryExecutor {
     }
 
     /**
+     * Cancel active lazy queries and prevent submit of new queries.
+     */
+    public void cancelLazyWorkers() {
+        if (!lazyWorkerStopGuard.compareAndSet(false, true))
+            return;
+
+        lazyWorkerBusyLock.block();
+
+        for (MapQueryLazyWorker worker : lazyWorkers.values())
+            worker.stop();
+
+        lazyWorkers.clear();
+    }
+
+    /**
      * @param nodeId Node ID.
      * @param msg Message.
      */
@@ -221,7 +247,7 @@ public class GridMapQueryExecutor {
         MapNodeResults nodeRess = qryRess.get(nodeId);
 
         if (nodeRess == null) {
-            nodeRess = new MapNodeResults();
+            nodeRess = new MapNodeResults(nodeId);
 
             MapNodeResults old = qryRess.putIfAbsent(nodeId, nodeRess);
 
@@ -416,6 +442,7 @@ public class GridMapQueryExecutor {
         final boolean enforceJoinOrder = req.isFlagSet(GridH2QueryRequest.FLAG_ENFORCE_JOIN_ORDER);
         final boolean explain = req.isFlagSet(GridH2QueryRequest.FLAG_EXPLAIN);
         final boolean replicated = req.isFlagSet(GridH2QueryRequest.FLAG_REPLICATED);
+        final boolean lazy = req.isFlagSet(GridH2QueryRequest.FLAG_LAZY);
 
         final List<Integer> cacheIds = req.caches();
 
@@ -429,30 +456,51 @@ public class GridMapQueryExecutor {
 
             final int segment = i;
 
-            ctx.closure().callLocal(
-                new Callable<Void>() {
-                    @Override public Void call() throws Exception {
-                        onQueryRequest0(node,
-                            req.requestId(),
-                            segment,
-                            req.schemaName(),
-                            req.queries(),
-                            cacheIds,
-                            req.topologyVersion(),
-                            partsMap,
-                            parts,
-                            req.tables(),
-                            req.pageSize(),
-                            joinMode,
-                            enforceJoinOrder,
-                            false,
-                            req.timeout(),
-                            params);
-
-                        return null;
+            if (lazy) {
+                onQueryRequest0(node,
+                    req.requestId(),
+                    segment,
+                    req.schemaName(),
+                    req.queries(),
+                    cacheIds,
+                    req.topologyVersion(),
+                    partsMap,
+                    parts,
+                    req.pageSize(),
+                    joinMode,
+                    enforceJoinOrder,
+                    false, // Replicated is always false here (see condition above).
+                    req.timeout(),
+                    params,
+                    true); // Lazy = true.
+            }
+            else {
+                ctx.closure().callLocal(
+                    new Callable<Void>() {
+                        @Override
+                        public Void call() throws Exception {
+                            onQueryRequest0(node,
+                                req.requestId(),
+                                segment,
+                                req.schemaName(),
+                                req.queries(),
+                                cacheIds,
+                                req.topologyVersion(),
+                                partsMap,
+                                parts,
+                                req.pageSize(),
+                                joinMode,
+                                enforceJoinOrder,
+                                false,
+                                req.timeout(),
+                                params,
+                                false); // Lazy = false.
+
+                            return null;
+                        }
                     }
-                }
-                , QUERY_POOL);
+                    , QUERY_POOL);
+            }
         }
 
         onQueryRequest0(node,
@@ -464,13 +512,13 @@ public class GridMapQueryExecutor {
             req.topologyVersion(),
             partsMap,
             parts,
-            req.tables(),
             req.pageSize(),
             joinMode,
             enforceJoinOrder,
             replicated,
             req.timeout(),
-            params);
+            params,
+            lazy);
     }
 
     /**
@@ -483,28 +531,61 @@ public class GridMapQueryExecutor {
      * @param topVer Topology version.
      * @param partsMap Partitions map for unstable topology.
      * @param parts Explicit partitions for current node.
-     * @param tbls Tables.
      * @param pageSize Page size.
      * @param distributedJoinMode Query distributed join mode.
+     * @param lazy Streaming flag.
      */
     private void onQueryRequest0(
-        ClusterNode node,
-        long reqId,
-        int segmentId,
-        String schemaName,
-        Collection<GridCacheSqlQuery> qrys,
-        List<Integer> cacheIds,
-        AffinityTopologyVersion topVer,
-        Map<UUID, int[]> partsMap,
-        int[] parts,
-        Collection<QueryTable> tbls,
-        int pageSize,
-        DistributedJoinMode distributedJoinMode,
-        boolean enforceJoinOrder,
-        boolean replicated,
-        int timeout,
-        Object[] params
+        final ClusterNode node,
+        final long reqId,
+        final int segmentId,
+        final String schemaName,
+        final Collection<GridCacheSqlQuery> qrys,
+        final List<Integer> cacheIds,
+        final AffinityTopologyVersion topVer,
+        final Map<UUID, int[]> partsMap,
+        final int[] parts,
+        final int pageSize,
+        final DistributedJoinMode distributedJoinMode,
+        final boolean enforceJoinOrder,
+        final boolean replicated,
+        final int timeout,
+        final Object[] params,
+        boolean lazy
     ) {
+        if (lazy && MapQueryLazyWorker.currentWorker() == null) {
+            // Lazy queries must be re-submitted to dedicated workers.
+            MapQueryLazyWorkerKey key = new MapQueryLazyWorkerKey(node.id(), reqId, segmentId);
+            MapQueryLazyWorker worker = new MapQueryLazyWorker(ctx.igniteInstanceName(), key, log, this);
+
+            worker.submit(new Runnable() {
+                @Override public void run() {
+                    onQueryRequest0(node, reqId, segmentId, schemaName, qrys, cacheIds, topVer, partsMap, parts,
+                        pageSize, distributedJoinMode, enforceJoinOrder, replicated, timeout, params, true);
+                }
+            });
+
+            if (lazyWorkerBusyLock.enterBusy()) {
+                try {
+                    MapQueryLazyWorker oldWorker = lazyWorkers.put(key, worker);
+
+                    if (oldWorker != null)
+                        oldWorker.stop();
+
+                    IgniteThread thread = new IgniteThread(worker);
+
+                    thread.start();
+                }
+                finally {
+                    lazyWorkerBusyLock.leaveBusy();
+                }
+            }
+            else
+                log.info("Ignored query request (node is stopping) [nodeId=" + node.id() + ", reqId=" + reqId + ']');
+
+            return;
+        }
+
         // Prepare to run queries.
         GridCacheContext<?, ?> mainCctx =
             !F.isEmpty(cacheIds) ? ctx.cache().context().cacheContext(cacheIds.get(0)) : null;
@@ -519,13 +600,18 @@ public class GridMapQueryExecutor {
             if (topVer != null) {
                 // Reserve primary for topology version or explicit partitions.
                 if (!reservePartitions(cacheIds, topVer, parts, reserved)) {
+                    // Unregister lazy worker because re-try may never reach this node again.
+                    if (lazy)
+                        stopAndUnregisterCurrentLazyWorker();
+
                     sendRetry(node, reqId, segmentId);
 
                     return;
                 }
             }
 
-            qr = new MapQueryResults(h2, reqId, qrys.size(), mainCctx != null ? mainCctx.name() : null);
+            qr = new MapQueryResults(h2, reqId, qrys.size(), mainCctx != null ? mainCctx.name() : null,
+                MapQueryLazyWorker.currentWorker());
 
             if (nodeRess.put(reqId, segmentId, qr) != null)
                 throw new IllegalStateException();
@@ -570,8 +656,7 @@ public class GridMapQueryExecutor {
                     ResultSet rs = null;
 
                     // If we are not the target node for this replicated query, just ignore it.
-                    if (qry.node() == null ||
-                        (segmentId == 0 && qry.node().equals(ctx.localNodeId()))) {
+                    if (qry.node() == null || (segmentId == 0 && qry.node().equals(ctx.localNodeId()))) {
                         rs = h2.executeSqlQueryWithTimer(conn, qry.query(),
                             F.asList(qry.parameters(params)), true,
                             timeout,
@@ -624,6 +709,10 @@ public class GridMapQueryExecutor {
                 qr.cancel(false);
             }
 
+            // Unregister worker after possible cancellation.
+            if (lazy)
+                stopAndUnregisterCurrentLazyWorker();
+
             if (X.hasCause(e, GridH2RetryException.class))
                 sendRetry(node, reqId, segmentId);
             else {
@@ -672,27 +761,39 @@ public class GridMapQueryExecutor {
      * @param node Node.
      * @param req Request.
      */
-    private void onNextPageRequest(ClusterNode node, GridQueryNextPageRequest req) {
-        MapNodeResults nodeRess = qryRess.get(node.id());
+    private void onNextPageRequest(final ClusterNode node, final GridQueryNextPageRequest req) {
+        final MapNodeResults nodeRess = qryRess.get(node.id());
 
         if (nodeRess == null) {
             sendError(node, req.queryRequestId(), new CacheException("No node result found for request: " + req));
 
             return;
-        } else if (nodeRess.cancelled(req.queryRequestId())) {
+        }
+        else if (nodeRess.cancelled(req.queryRequestId())) {
             sendError(node, req.queryRequestId(), new QueryCancelledException());
 
             return;
         }
 
-        MapQueryResults qr = nodeRess.get(req.queryRequestId(), req.segmentId());
+        final MapQueryResults qr = nodeRess.get(req.queryRequestId(), req.segmentId());
 
         if (qr == null)
             sendError(node, req.queryRequestId(), new CacheException("No query result found for request: " + req));
         else if (qr.cancelled())
             sendError(node, req.queryRequestId(), new QueryCancelledException());
-        else
-            sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize());
+        else {
+            MapQueryLazyWorker lazyWorker = qr.lazyWorker();
+
+            if (lazyWorker != null) {
+                lazyWorker.submit(new Runnable() {
+                    @Override public void run() {
+                        sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize());
+                    }
+                });
+            }
+            else
+                sendNextPage(nodeRess, node, qr, req.query(), req.segmentId(), req.pageSize());
+        }
     }
 
     /**
@@ -784,4 +885,34 @@ public class GridMapQueryExecutor {
                 reservations.remove(grpKey);
         }
     }
+
+    /**
+     * Unregister lazy worker if needed (i.e. if we are currently in lazy worker thread).
+     */
+    public void stopAndUnregisterCurrentLazyWorker() {
+        MapQueryLazyWorker worker = MapQueryLazyWorker.currentWorker();
+
+        if (worker != null) {
+            worker.stop();
+
+            // Just stop is not enough as worker may be registered, but not started due to exception.
+            unregisterLazyWorker(worker);
+        }
+    }
+
+    /**
+     * Unregister lazy worker.
+     *
+     * @param worker Worker.
+     */
+    public void unregisterLazyWorker(MapQueryLazyWorker worker) {
+        lazyWorkers.remove(worker.key(), worker);
+    }
+
+    /**
+     * @return Number of registered lazy workers.
+     */
+    public int registeredLazyWorkers() {
+        return lazyWorkers.size();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 0e9d1a2..8638794 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -506,6 +506,7 @@ public class GridReduceQueryExecutor {
      * @param cancel Query cancel.
      * @param params Query parameters.
      * @param parts Partitions.
+     * @param lazy Lazy execution flag.
      * @return Rows iterator.
      */
     public Iterator<List<?>> query(
@@ -516,7 +517,8 @@ public class GridReduceQueryExecutor {
         int timeoutMillis,
         GridQueryCancel cancel,
         Object[] params,
-        final int[] parts
+        final int[] parts,
+        boolean lazy
     ) {
         if (F.isEmpty(params))
             params = EMPTY_PARAMS;
@@ -712,6 +714,9 @@ public class GridReduceQueryExecutor {
                 if (isReplicatedOnly)
                     flags |= GridH2QueryRequest.FLAG_REPLICATED;
 
+                if (lazy && mapQrys.size() == 1)
+                    flags |= GridH2QueryRequest.FLAG_LAZY;
+
                 GridH2QueryRequest req = new GridH2QueryRequest()
                     .requestId(qryReqId)
                     .topologyVersion(topVer)

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java
index d5ea357..2d20c8d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapNodeResults.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.query.h2.twostep;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.jsr166.ConcurrentHashMap8;
 
+import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 
 import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.PER_SEGMENT_Q;
@@ -35,6 +36,18 @@ class MapNodeResults {
     private final GridBoundedConcurrentLinkedHashMap<Long, Boolean> qryHist =
         new GridBoundedConcurrentLinkedHashMap<>(1024, 1024, 0.75f, 64, PER_SEGMENT_Q);
 
+    /** Node ID. */
+    private final UUID nodeId;
+
+    /**
+     * Constructor.
+     *
+     * @param nodeId Node ID.
+     */
+    public MapNodeResults(UUID nodeId) {
+        this.nodeId = nodeId;
+    }
+
     /**
      * @param reqId Query Request ID.
      * @return {@code False} if query was already cancelled.
@@ -59,7 +72,7 @@ class MapNodeResults {
      * @return query partial results.
      */
     public MapQueryResults get(long reqId, int segmentId) {
-        return res.get(new MapRequestKey(reqId, segmentId));
+        return res.get(new MapRequestKey(nodeId, reqId, segmentId));
     }
 
     /**
@@ -84,7 +97,7 @@ class MapNodeResults {
      * @return {@code True} if removed.
      */
     public boolean remove(long reqId, int segmentId, MapQueryResults qr) {
-        return res.remove(new MapRequestKey(reqId, segmentId), qr);
+        return res.remove(new MapRequestKey(nodeId, reqId, segmentId), qr);
     }
 
     /**
@@ -94,7 +107,7 @@ class MapNodeResults {
      * @return previous value.
      */
     public MapQueryResults put(long reqId, int segmentId, MapQueryResults qr) {
-        return res.put(new MapRequestKey(reqId, segmentId), qr);
+        return res.put(new MapRequestKey(nodeId, reqId, segmentId), qr);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
new file mode 100644
index 0000000..5158035
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
@@ -0,0 +1,176 @@
+/*
+ * 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.twostep;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.LongAdder8;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingDeque;
+
+/**
+ * Worker for lazy query execution.
+ */
+public class MapQueryLazyWorker extends GridWorker {
+    /** Lazy thread flag. */
+    private static final ThreadLocal<MapQueryLazyWorker> LAZY_WORKER = new ThreadLocal<>();
+
+    /** Active lazy worker count (for testing purposes). */
+    private static final LongAdder8 ACTIVE_CNT = new LongAdder8();
+
+    /** Task to be executed. */
+    private final BlockingQueue<Runnable> tasks = new LinkedBlockingDeque<>();
+
+    /** Key. */
+    private final MapQueryLazyWorkerKey key;
+
+    /** Map query executor. */
+    private final GridMapQueryExecutor exec;
+
+    /** Latch decremented when worker finishes. */
+    private final CountDownLatch stopLatch = new CountDownLatch(1);
+
+    /** Map query result. */
+    private volatile MapQueryResult res;
+
+    /**
+     * Constructor.
+     *
+     * @param instanceName Instance name.
+     * @param key Lazy worker key.
+     * @param log Logger.
+     * @param exec Map query executor.
+     */
+    public MapQueryLazyWorker(@Nullable String instanceName, MapQueryLazyWorkerKey key, IgniteLogger log,
+        GridMapQueryExecutor exec) {
+        super(instanceName, workerName(instanceName, key), log);
+
+        this.key = key;
+        this.exec = exec;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+        LAZY_WORKER.set(this);
+
+        ACTIVE_CNT.increment();
+
+        try {
+            while (!isCancelled()) {
+                Runnable task = tasks.take();
+
+                if (task != null)
+                    task.run();
+            }
+        }
+        finally {
+            if (res != null)
+                res.close();
+
+            LAZY_WORKER.set(null);
+
+            ACTIVE_CNT.decrement();
+
+            exec.unregisterLazyWorker(this);
+        }
+    }
+
+    /**
+     * Submit task to worker.
+     *
+     * @param task Task to be executed.
+     */
+    public void submit(Runnable task) {
+        tasks.add(task);
+    }
+
+    /**
+     * @return Worker key.
+     */
+    public MapQueryLazyWorkerKey key() {
+        return key;
+    }
+
+    /**
+     * Stop the worker.
+     */
+    public void stop() {
+        if (MapQueryLazyWorker.currentWorker() == null)
+            submit(new Runnable() {
+                @Override public void run() {
+                    stop();
+                }
+            });
+        else {
+            isCancelled = true;
+
+            stopLatch.countDown();
+        }
+    }
+
+    /**
+     * Await worker stop.
+     */
+    public void awaitStop() {
+        try {
+            U.await(stopLatch);
+        }
+        catch (IgniteInterruptedCheckedException e) {
+            throw new IgniteException("Failed to wait for lazy worker stop (interrupted): " + name(), e);
+        }
+    }
+
+    /**
+     * @param res Map query result.
+     */
+    public void result(MapQueryResult res) {
+        this.res = res;
+    }
+
+    /**
+     * @return Current worker or {@code null} if call is performed not from lazy worker thread.
+     */
+    @Nullable public static MapQueryLazyWorker currentWorker() {
+        return LAZY_WORKER.get();
+    }
+
+    /**
+     * @return Active workers count.
+     */
+    public static int activeCount() {
+        return ACTIVE_CNT.intValue();
+    }
+
+    /**
+     * Construct worker name.
+     *
+     * @param instanceName Instance name.
+     * @param key Key.
+     * @return Name.
+     */
+    private static String workerName(String instanceName, MapQueryLazyWorkerKey key) {
+        return "query-lazy-worker_" + instanceName + "_" + key.nodeId() + "_" + key.queryRequestId() + "_" +
+            key.segment();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorkerKey.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorkerKey.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorkerKey.java
new file mode 100644
index 0000000..a0f5ebb
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorkerKey.java
@@ -0,0 +1,97 @@
+/*
+ * 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.twostep;
+
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+import java.util.UUID;
+
+/**
+ * Key to identify lazy worker.
+ */
+public class MapQueryLazyWorkerKey {
+    /** Client node ID. */
+    private final UUID nodeId;
+
+    /** Query request ID. */
+    private final long qryReqId;
+
+    /** Segment. */
+    private final int segment;
+
+    /**
+     * Constructor.
+     *
+     * @param nodeId Node ID.
+     * @param qryReqId Query request ID.
+     * @param segment Segment.
+     */
+    public MapQueryLazyWorkerKey(UUID nodeId, long qryReqId, int segment) {
+        this.nodeId = nodeId;
+        this.qryReqId = qryReqId;
+        this.segment = segment;
+    }
+
+    /**
+     * @return Node id.
+     */
+    public UUID nodeId() {
+        return nodeId;
+    }
+
+    /**
+     * @return Query request ID.
+     */
+    public long queryRequestId() {
+        return qryReqId;
+    }
+
+    /**
+     * @return Segment.
+     */
+    public int segment() {
+        return segment;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = nodeId.hashCode();
+
+        res = 31 * res + (int)(qryReqId ^ (qryReqId >>> 32));
+        res = 31 * res + segment;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (obj != null && obj instanceof MapQueryLazyWorkerKey) {
+            MapQueryLazyWorkerKey other = (MapQueryLazyWorkerKey)obj;
+
+            return F.eq(qryReqId, other.qryReqId) && F.eq(nodeId, other.nodeId) && F.eq(segment, other.segment);
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MapQueryLazyWorkerKey.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java
index 4799e03..e54c784d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResult.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.jdbc.JdbcResultSet;
+import org.h2.result.LazyResult;
 import org.h2.result.ResultInterface;
 import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
@@ -41,7 +42,7 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_OBJECT_READ;
 /**
  * Mapper result for a single part of the query.
  */
-class MapQueryResult implements AutoCloseable {
+class MapQueryResult {
     /** */
     private static final Field RESULT_FIELD;
 
@@ -95,24 +96,30 @@ class MapQueryResult implements AutoCloseable {
     /** */
     private final Object[] params;
 
+    /** Lazy worker. */
+    private final MapQueryLazyWorker lazyWorker;
+
     /**
      * @param rs Result set.
      * @param cacheName Cache name.
      * @param qrySrcNodeId Query source node.
      * @param qry Query.
      * @param params Query params.
+     * @param lazyWorker Lazy worker.
      */
     MapQueryResult(IgniteH2Indexing h2, ResultSet rs, @Nullable String cacheName,
-        UUID qrySrcNodeId, GridCacheSqlQuery qry, Object[] params) {
+        UUID qrySrcNodeId, GridCacheSqlQuery qry, Object[] params, @Nullable MapQueryLazyWorker lazyWorker) {
         this.h2 = h2;
         this.cacheName = cacheName;
         this.qry = qry;
         this.params = params;
         this.qrySrcNodeId = qrySrcNodeId;
         this.cpNeeded = F.eq(h2.kernalContext().localNodeId(), qrySrcNodeId);
+        this.lazyWorker = lazyWorker;
 
         if (rs != null) {
             this.rs = rs;
+
             try {
                 res = (ResultInterface)RESULT_FIELD.get(rs);
             }
@@ -120,7 +127,7 @@ class MapQueryResult implements AutoCloseable {
                 throw new IllegalStateException(e); // Must not happen.
             }
 
-            rowCnt = res.getRowCount();
+            rowCnt = (res instanceof LazyResult) ? -1 : res.getRowCount();
             cols = res.getVisibleColumnCount();
         }
         else {
@@ -167,6 +174,8 @@ class MapQueryResult implements AutoCloseable {
      * @return {@code true} If there are no more rows available.
      */
     synchronized boolean fetchNextPage(List<Value[]> rows, int pageSize) {
+        assert lazyWorker == null || lazyWorker == MapQueryLazyWorker.currentWorker();
+
         if (closed)
             return true;
 
@@ -246,13 +255,34 @@ class MapQueryResult implements AutoCloseable {
         return res;
     }
 
-    /** {@inheritDoc} */
-    @Override public synchronized void close() {
-        if (closed)
+    /**
+     * Close the result.
+     */
+    public void close() {
+        if (lazyWorker != null && MapQueryLazyWorker.currentWorker() == null) {
+            lazyWorker.submit(new Runnable() {
+                @Override public void run() {
+                    close();
+                }
+            });
+
+            lazyWorker.awaitStop();
+
             return;
+        }
 
-        closed = true;
+        synchronized (this) {
+            assert lazyWorker == null || lazyWorker == MapQueryLazyWorker.currentWorker();
 
-        U.closeQuiet(rs);
+            if (closed)
+                return;
+
+            closed = true;
+
+            U.closeQuiet(rs);
+
+            if (lazyWorker != null)
+                lazyWorker.stop();
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java
index 7ad1d14..99f1966 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryResults.java
@@ -45,20 +45,27 @@ class MapQueryResults {
     /** */
     private final String cacheName;
 
+    /** Lazy worker. */
+    private final MapQueryLazyWorker lazyWorker;
+
     /** */
     private volatile boolean cancelled;
 
     /**
+     * Constructor.
+     *
      * @param qryReqId Query request ID.
      * @param qrys Number of queries.
      * @param cacheName Cache name.
+     * @param lazyWorker Lazy worker (if any).
      */
     @SuppressWarnings("unchecked")
-    MapQueryResults(IgniteH2Indexing h2, long qryReqId, int qrys,
-        @Nullable String cacheName) {
+    MapQueryResults(IgniteH2Indexing h2, long qryReqId, int qrys, @Nullable String cacheName,
+        @Nullable MapQueryLazyWorker lazyWorker) {
         this.h2 = h2;
         this.qryReqId = qryReqId;
         this.cacheName = cacheName;
+        this.lazyWorker = lazyWorker;
 
         results = new AtomicReferenceArray<>(qrys);
         cancels = new GridQueryCancel[qrys];
@@ -86,13 +93,25 @@ class MapQueryResults {
     }
 
     /**
+     * @return Lazy worker.
+     */
+    MapQueryLazyWorker lazyWorker() {
+        return lazyWorker;
+    }
+
+    /**
+     * Add result.
+     *
      * @param qry Query result index.
      * @param q Query object.
      * @param qrySrcNodeId Query source node.
      * @param rs Result set.
      */
     void addResult(int qry, GridCacheSqlQuery q, UUID qrySrcNodeId, ResultSet rs, Object[] params) {
-        MapQueryResult res = new MapQueryResult(h2, rs, cacheName, qrySrcNodeId, q, params);
+        MapQueryResult res = new MapQueryResult(h2, rs, cacheName, qrySrcNodeId, q, params, lazyWorker);
+
+        if (lazyWorker != null)
+            lazyWorker.result(res);
 
         if (!results.compareAndSet(qry, null, res))
             throw new IllegalStateException();
@@ -130,6 +149,7 @@ class MapQueryResults {
                 continue;
             }
 
+            // NB: Cancel is already safe even for lazy queries (see implementation of passed Runnable).
             if (forceQryCancel) {
                 GridQueryCancel cancel = cancels[i];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapRequestKey.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapRequestKey.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapRequestKey.java
index 6feb8ea..9d987db 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapRequestKey.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapRequestKey.java
@@ -17,18 +17,32 @@
 
 package org.apache.ignite.internal.processors.query.h2.twostep;
 
+import org.apache.ignite.internal.util.typedef.F;
+
+import java.util.UUID;
+
 /**
  * Mapper request key.
  */
 class MapRequestKey {
+    /** Node ID. */
+    private UUID nodeId;
+
     /** */
     private long reqId;
 
     /** */
     private int segmentId;
 
-    /** Constructor */
-    MapRequestKey(long reqId, int segmentId) {
+    /**
+     * Constructor.
+     *
+     * @param nodeId Node ID.
+     * @param reqId Request ID.
+     * @param segmentId Segment ID.
+     */
+    MapRequestKey(UUID nodeId, long reqId, int segmentId) {
+        this.nodeId = nodeId;
         this.reqId = reqId;
         this.segmentId = segmentId;
     }
@@ -50,14 +64,15 @@ class MapRequestKey {
 
         MapRequestKey other = (MapRequestKey)o;
 
-        return reqId == other.reqId && segmentId == other.segmentId;
+        return F.eq(nodeId, other.nodeId) && reqId == other.reqId && segmentId == other.segmentId;
 
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        int res = (int)(reqId ^ (reqId >>> 32));
+        int res = nodeId != null ? nodeId.hashCode() : 0;
 
+        res = 31 * res + (int)(reqId ^ (reqId >>> 32));
         res = 31 * res + segmentId;
 
         return res;

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
index 93a383c..4e1fadb 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2QueryRequest.java
@@ -78,6 +78,11 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
      */
     public static final int FLAG_REPLICATED = 1 << 4;
 
+    /**
+     * If lazy execution is enabled.
+     */
+    public static final int FLAG_LAZY = 1 << 5;
+
     /** */
     private long reqId;
 
@@ -185,6 +190,10 @@ public class GridH2QueryRequest implements Message, GridCacheQueryMarshallable {
     }
 
     /**
+     * Get tables.
+     * <p>
+     * N.B.: Was used in AI 1.9 for snapshots. Unused at the moment, but should be kept for compatibility reasons.
+     *
      * @return Tables.
      */
     public Collection<QueryTable> tables() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/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
new file mode 100644
index 0000000..d5cc0eb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/LazyQuerySelfTest.java
@@ -0,0 +1,389 @@
+/*
+ * 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;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+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.IgniteKernal;
+import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.processors.query.h2.twostep.MapQueryLazyWorker;
+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 ocunt. */
+    private static final int KEY_CNT = 200;
+
+    /** Base query argument. */
+    private static final int BASE_QRY_ARG = 50;
+
+    /** Size for small pages. */
+    private static final int PAGE_SIZE_SMALL = 12;
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Test local query execution.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSingleNode() throws Exception {
+        checkSingleNode(1);
+    }
+
+    /**
+     * Test local query execution.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSingleNodeWithParallelism() throws Exception {
+        checkSingleNode(4);
+    }
+
+    /**
+     * Test query execution with multiple topology nodes.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultipleNodes() throws Exception {
+        checkMultipleNodes(1);
+    }
+
+    /**
+     * Test query execution with multiple topology nodes with query parallelism.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultipleNodesWithParallelism() throws Exception {
+        checkMultipleNodes(4);
+    }
+
+    /**
+     * Check local query execution.
+     *
+     * @param parallelism Query parallelism.
+     * @throws Exception If failed.
+     */
+    public void checkSingleNode(int parallelism) throws Exception {
+        Ignite srv = startGrid();
+
+        srv.createCache(cacheConfiguration(parallelism));
+
+        populateBaseQueryData(srv);
+
+        checkBaseOperations(srv);
+    }
+
+    /**
+     * Check query execution with multiple topology nodes.
+     *
+     * @param parallelism Query parallelism.
+     * @throws Exception If failed.
+     */
+    public void checkMultipleNodes(int parallelism) throws Exception {
+        Ignite srv1 = startGrid(1);
+        Ignite srv2 = startGrid(2);
+
+        Ignite cli;
+
+        try {
+            Ignition.setClientMode(true);
+
+            cli = startGrid(3);
+        }
+        finally {
+            Ignition.setClientMode(false);
+        }
+
+        cli.createCache(cacheConfiguration(parallelism));
+
+        populateBaseQueryData(cli);
+
+        checkBaseOperations(srv1);
+        checkBaseOperations(srv2);
+        checkBaseOperations(cli);
+
+        // Test originating node leave.
+        FieldsQueryCursor<List<?>> cursor = execute(cli, baseQuery().setPageSize(PAGE_SIZE_SMALL));
+
+        Iterator<List<?>> iter = cursor.iterator();
+
+        for (int i = 0; i < 30; i++)
+            iter.next();
+
+        stopGrid(3);
+
+        assertNoWorkers();
+
+        // Test server node leave with active worker.
+        cursor = execute(srv1, baseQuery().setPageSize(PAGE_SIZE_SMALL));
+
+        try {
+            iter = cursor.iterator();
+
+            for (int i = 0; i < 30; i++)
+                iter.next();
+
+            stopGrid(2);
+        }
+        finally {
+            cursor.close();
+        }
+
+        assertNoWorkers();
+    }
+
+    /**
+     * Check base operations.
+     *
+     * @param node Node.
+     * @throws Exception If failed.
+     */
+    private void checkBaseOperations(Ignite node) throws Exception {
+        // Get full data.
+        List<List<?>> rows = execute(node, baseQuery()).getAll();
+
+        assertBaseQueryResults(rows);
+        assertNoWorkers();
+
+        // Get data in several pages.
+        rows = execute(node, baseQuery().setPageSize(PAGE_SIZE_SMALL)).getAll();
+
+        assertBaseQueryResults(rows);
+        assertNoWorkers();
+
+        // Test full iteration.
+        rows = new ArrayList<>();
+
+        FieldsQueryCursor<List<?>> cursor = execute(node, baseQuery().setPageSize(PAGE_SIZE_SMALL));
+
+        for (List<?> row : cursor)
+            rows.add(row);
+
+        assertBaseQueryResults(rows);
+        assertNoWorkers();
+
+        // Test partial iteration with cursor close.
+        try (FieldsQueryCursor<List<?>> partialCursor = execute(node, baseQuery().setPageSize(PAGE_SIZE_SMALL))) {
+            Iterator<List<?>> iter = partialCursor.iterator();
+
+            for (int i = 0; i < 30; i++)
+                iter.next();
+        }
+
+        assertNoWorkers();
+
+        // Test execution of multiple queries at a time.
+        List<Iterator<List<?>>> iters = new ArrayList<>();
+
+        for (int i = 0; i < 200; i++)
+            iters.add(execute(node, randomizedQuery().setPageSize(PAGE_SIZE_SMALL)).iterator());
+
+        while (!iters.isEmpty()) {
+            Iterator<Iterator<List<?>>> iterIter = iters.iterator();
+
+            while (iterIter.hasNext()) {
+                Iterator<List<?>> iter = iterIter.next();
+
+                int i = 0;
+
+                while (iter.hasNext() && i < 20) {
+                    iter.next();
+
+                    i++;
+                }
+
+                if (!iter.hasNext())
+                    iterIter.remove();
+            }
+        }
+
+        assertNoWorkers();
+    }
+
+    /**
+     * Populate base query data.
+     *
+     * @param node Node.
+     */
+    private static void populateBaseQueryData(Ignite node) {
+        IgniteCache<Long, Person> cache = cache(node);
+
+        for (long i = 0; i < KEY_CNT; i++)
+            cache.put(i, new Person(i));
+    }
+
+    /**
+     * @return Query with randomized argument.
+     */
+    private static SqlFieldsQuery randomizedQuery() {
+        return query(ThreadLocalRandom.current().nextInt(KEY_CNT / 2));
+    }
+
+    /**
+     * @return Base query.
+     */
+    private static SqlFieldsQuery baseQuery() {
+        return query(BASE_QRY_ARG);
+    }
+
+    /**
+     * @param parallelism Query parallelism.
+     * @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);
+    }
+
+    /**
+     * Default query.
+     *
+     * @param arg Argument.
+     * @return Query.
+     */
+    private static SqlFieldsQuery query(long arg) {
+        return new SqlFieldsQuery("SELECT id, name FROM Person WHERE id >= ?").setArgs(arg);
+    }
+
+    /**
+     * Assert base query results.
+     *
+     * @param rows Result rows.
+     */
+    private static void assertBaseQueryResults(List<List<?>> rows) {
+        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 >= BASE_QRY_ARG);
+            assertEquals(nameForId(id), name);
+        }
+    }
+
+    /**
+     * Get cache for node.
+     *
+     * @param node Node.
+     * @return Cache.
+     */
+    private static IgniteCache<Long, Person> cache(Ignite node) {
+        return node.cache(CACHE_NAME);
+    }
+
+    /**
+     * Execute query on the given cache.
+     *
+     * @param node Node.
+     * @param qry Query.
+     * @return Cursor.
+     */
+    @SuppressWarnings("unchecked")
+    private static FieldsQueryCursor<List<?>> execute(Ignite node, SqlFieldsQuery qry) {
+        return cache(node).query(qry.setLazy(true));
+    }
+
+    /**
+     * Make sure that are no active lazy workers.
+     *
+     * @throws Exception If failed.
+     */
+    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();
+
+                    if (idx.mapQueryExecutor().registeredLazyWorkers() != 0)
+                        return false;
+                }
+
+                return MapQueryLazyWorker.activeCount() == 0;
+            }
+        }, 1000L);
+    }
+
+    /**
+     * Get name for ID.
+     *
+     * @param id ID.
+     * @return Name.
+     */
+    private static String nameForId(long id) {
+        return "name-" + id;
+    }
+
+    /**
+     * Person class.
+     */
+    private static class Person {
+        /** ID. */
+        @QuerySqlField(index = true)
+        private long id;
+
+        /** Name. */
+        @QuerySqlField
+        private String name;
+
+        /**
+         * Constructor.
+         *
+         * @param id ID.
+         */
+        public Person(long id) {
+            this.id = id;
+            this.name = nameForId(id);
+        }
+
+        /**
+         * @return ID.
+         */
+        public long id() {
+            return id;
+        }
+
+        /**
+         * @return Name.
+         */
+        public String name() {
+            return name;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/136075ae/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 99b0370..5ac0655f 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
@@ -127,6 +127,7 @@ import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
+import org.apache.ignite.internal.processors.query.LazyQuerySelfTest;
 import org.apache.ignite.internal.processors.query.SqlSchemaSelfTest;
 import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest;
 import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest;
@@ -184,6 +185,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IncorrectQueryEntityTest.class);
 
         // Queries tests.
+        suite.addTestSuite(LazyQuerySelfTest.class);
         suite.addTestSuite(IgniteSqlSplitterSelfTest.class);
         suite.addTestSuite(IgniteSqlSegmentedIndexSelfTest.class);
         suite.addTestSuite(IgniteSqlSegmentedIndexMultiNodeSelfTest.class);


[11/19] ignite git commit: IGNITE-6100 Fixed memory leak, IgnitePdsRecoveryAfterFileCorruptionTest.testPageRecoveryAfterFileCorruption - Fixes #2466.

Posted by ag...@apache.org.
IGNITE-6100 Fixed memory leak, IgnitePdsRecoveryAfterFileCorruptionTest.testPageRecoveryAfterFileCorruption - Fixes #2466.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-5901
Commit: 06ad010e92541e4e0f0cdcfe6c99ea8d4af69b70
Parents: 071c24c
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Thu Aug 17 17:07:29 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 17:07:29 2017 +0300

----------------------------------------------------------------------
 .../persistence/pagemem/PageMemoryImpl.java     | 71 ++++++++++----------
 1 file changed, 37 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/06ad010e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 1b4cf81..b6e5f46 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -47,6 +47,7 @@ import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
+import org.apache.ignite.internal.pagemem.wal.WALIterator;
 import org.apache.ignite.internal.pagemem.wal.WALPointer;
 import org.apache.ignite.internal.pagemem.wal.record.CheckpointRecord;
 import org.apache.ignite.internal.pagemem.wal.record.PageSnapshot;
@@ -670,55 +671,57 @@ public class PageMemoryImpl implements PageMemoryEx {
             ByteBuffer curPage = null;
             ByteBuffer lastValidPage = null;
 
-            for (IgniteBiTuple<WALPointer, WALRecord> tuple : walMgr.replay(null)) {
-                switch (tuple.getValue().type()) {
-                    case PAGE_RECORD:
-                        PageSnapshot snapshot = (PageSnapshot)tuple.getValue();
+            try (WALIterator it = walMgr.replay(null)) {
+                for (IgniteBiTuple<WALPointer, WALRecord> tuple : it) {
+                    switch (tuple.getValue().type()) {
+                        case PAGE_RECORD:
+                            PageSnapshot snapshot = (PageSnapshot)tuple.getValue();
 
-                        if (snapshot.fullPageId().equals(fullId)) {
-                            if (tmpAddr == null) {
-                                assert snapshot.pageData().length <= pageSize() : snapshot.pageData().length;
+                            if (snapshot.fullPageId().equals(fullId)) {
+                                if (tmpAddr == null) {
+                                    assert snapshot.pageData().length <= pageSize() : snapshot.pageData().length;
 
-                                tmpAddr = GridUnsafe.allocateMemory(pageSize());
-                            }
+                                    tmpAddr = GridUnsafe.allocateMemory(pageSize());
+                                }
 
-                            if (curPage == null)
-                                curPage = wrapPointer(tmpAddr, pageSize());
+                                if (curPage == null)
+                                    curPage = wrapPointer(tmpAddr, pageSize());
 
-                            PageUtils.putBytes(tmpAddr, 0, snapshot.pageData());
-                        }
+                                PageUtils.putBytes(tmpAddr, 0, snapshot.pageData());
+                            }
 
-                        break;
+                            break;
 
-                    case CHECKPOINT_RECORD:
-                        CheckpointRecord rec = (CheckpointRecord)tuple.getValue();
+                        case CHECKPOINT_RECORD:
+                            CheckpointRecord rec = (CheckpointRecord)tuple.getValue();
 
-                        assert !rec.end();
+                            assert !rec.end();
 
-                        if (curPage != null) {
-                            lastValidPage = curPage;
-                            curPage = null;
-                        }
+                            if (curPage != null) {
+                                lastValidPage = curPage;
+                                curPage = null;
+                            }
 
-                        break;
+                            break;
 
-                    case MEMORY_RECOVERY: // It means that previous checkpoint was broken.
-                        curPage = null;
+                        case MEMORY_RECOVERY: // It means that previous checkpoint was broken.
+                            curPage = null;
 
-                        break;
+                            break;
 
-                    default:
-                        if (tuple.getValue() instanceof PageDeltaRecord) {
-                            PageDeltaRecord deltaRecord = (PageDeltaRecord)tuple.getValue();
+                        default:
+                            if (tuple.getValue() instanceof PageDeltaRecord) {
+                                PageDeltaRecord deltaRecord = (PageDeltaRecord)tuple.getValue();
 
-                            if (curPage != null
-                                && deltaRecord.pageId() == fullId.pageId()
-                                && deltaRecord.groupId() == fullId.groupId()) {
-                                assert tmpAddr != null;
+                                if (curPage != null
+                                    && deltaRecord.pageId() == fullId.pageId()
+                                    && deltaRecord.groupId() == fullId.groupId()) {
+                                    assert tmpAddr != null;
 
-                                deltaRecord.applyDelta(this, tmpAddr);
+                                    deltaRecord.applyDelta(this, tmpAddr);
+                                }
                             }
-                        }
+                    }
                 }
             }
 


[19/19] ignite git commit: IGNITE-5901 - Reverted debug changes

Posted by ag...@apache.org.
IGNITE-5901 - Reverted debug changes


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

Branch: refs/heads/ignite-5901
Commit: ed1da133dcff02df1f7112f73b2392b75e3635b1
Parents: f7aa561
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Aug 17 19:51:33 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 19:51:33 2017 +0300

----------------------------------------------------------------------
 .../wal/FileWriteAheadLogManager.java           | 19 +-----------
 .../db/wal/IgniteWalRecoveryTest.java           | 32 ++++++++++----------
 .../db/wal/IgniteWalRecoveryTest2.java          | 24 ---------------
 .../db/wal/IgniteWalRecoveryTest3.java          | 24 ---------------
 .../db/wal/IgniteWalRecoveryTest4.java          | 24 ---------------
 .../db/wal/IgniteWalRecoveryTest5.java          | 24 ---------------
 .../IgnitePdsWithIndexingCoreTestSuite.java     | 28 ++++++-----------
 7 files changed, 27 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ed1da133/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 9a03e9e..a9327a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -24,7 +24,6 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
-import java.nio.channels.AsynchronousCloseException;
 import java.nio.file.Files;
 import java.sql.Time;
 import java.util.Arrays;
@@ -70,7 +69,6 @@ import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -1673,9 +1671,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
          */
         private final Condition nextSegment = lock.newCondition();
 
-        /** */
-        private volatile Exception closeStack;
-
         /**
          * @param fileIO I/O file interface to use
          * @param idx Absolute WAL segment file index for easy access.
@@ -2038,10 +2033,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                         fileIO.force();
                     }
                     catch (IOException e) {
-                        if (X.hasCause(e, AsynchronousCloseException.class)) {
-                            log.error("Was concurrently closed: ", closeStack);
-                        }
-
                         throw new StorageException(e);
                     }
 
@@ -2073,9 +2064,7 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                 try {
                     flushOrWait(null, true);
 
-                    WALRecord hd = head.get();
-
-                    assert stopped() : "Segment is not closed after close flush: " + hd;
+                    assert stopped() : "Segment is not closed after close flush: " + head.get();
 
                     try {
                         int switchSegmentRecSize = RecordV1Serializer.REC_TYPE_SIZE + RecordV1Serializer.FILE_WAL_POINTER_SIZE;
@@ -2108,8 +2097,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                             lastFsyncPos = written;
                         }
 
-                        closeStack = new Exception("written=" + written + ", head=" + hd);
-
                         fileIO.close();
                     }
                     catch (IOException e) {
@@ -2241,10 +2228,6 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
                     assert written == fileIO.position();
                 }
                 catch (IOException e) {
-                    if (X.hasCause(e, AsynchronousCloseException.class)) {
-                        log.error("Was concurrently closed: ", closeStack);
-                    }
-
                     invalidateEnvironmentLocked(e);
 
                     throw new StorageException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed1da133/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
index 3d4f59f..399e36d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest.java
@@ -183,7 +183,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if failed.
      */
-    public void _testWalBig() throws Exception {
+    public void testWalBig() throws Exception {
         IgniteEx ignite = startGrid(1);
 
         ignite.active(true);
@@ -225,7 +225,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if failed.
      */
-    public void _testWalBigObjectNodeCancel() throws Exception {
+    public void testWalBigObjectNodeCancel() throws Exception {
         final int MAX_SIZE_POWER = 21;
 
         IgniteEx ignite = startGrid(1);
@@ -263,7 +263,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If fail.
      */
-    public void _testSwitchClassLoader() throws Exception {
+    public void testSwitchClassLoader() throws Exception {
         try {
             final IgniteEx igniteEx = startGrid(1);
 
@@ -307,7 +307,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if failed.
      */
-    public void _testWalSimple() throws Exception {
+    public void testWalSimple() throws Exception {
         try {
             IgniteEx ignite = startGrid(1);
 
@@ -387,7 +387,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If fail.
      */
-    public void _testWalLargeValue() throws Exception {
+    public void testWalLargeValue() throws Exception {
         try {
             IgniteEx ignite = startGrid(1);
 
@@ -469,7 +469,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if failed.
      */
-    public void _testHugeCheckpointRecord() throws Exception {
+    public void testHugeCheckpointRecord() throws Exception {
         try {
             final IgniteEx ignite = startGrid(1);
 
@@ -521,7 +521,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     private void checkWalRolloverMultithreaded() throws Exception {
         walSegmentSize = 2 * 1024 * 1024;
 
-        final long endTime = System.currentTimeMillis() + 3 * 60 * 1000;
+        final long endTime = System.currentTimeMillis() + 2 * 60 * 1000;
 
         try {
             IgniteEx ignite = startGrid(1);
@@ -549,7 +549,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If fail.
      */
-    public void _testWalRenameDirSimple() throws Exception {
+    public void testWalRenameDirSimple() throws Exception {
         try {
             IgniteEx ignite = startGrid(1);
 
@@ -611,7 +611,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if failed.
      */
-    public void _testRecoveryNoCheckpoint() throws Exception {
+    public void testRecoveryNoCheckpoint() throws Exception {
         try {
             IgniteEx ctrlGrid = startGrid(0);
 
@@ -661,7 +661,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if failed.
      */
-    public void _testRecoveryLargeNoCheckpoint() throws Exception {
+    public void testRecoveryLargeNoCheckpoint() throws Exception {
         try {
             IgniteEx ctrlGrid = startGrid(0);
 
@@ -712,13 +712,13 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
-        return TimeUnit.MINUTES.toMillis(200);
+        return TimeUnit.MINUTES.toMillis(20);
     }
 
     /**
      * @throws Exception if failed.
      */
-    public void _testRandomCrash() throws Exception {
+    public void testRandomCrash() throws Exception {
         try {
             IgniteEx ctrlGrid = startGrid(0);
 
@@ -758,7 +758,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if failed.
      */
-    public void _testLargeRandomCrash() throws Exception {
+    public void testLargeRandomCrash() throws Exception {
         try {
             IgniteEx ctrlGrid = startGrid(0);
 
@@ -808,7 +808,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void _testDestroyCache() throws Exception {
+    public void testDestroyCache() throws Exception {
         try {
             IgniteEx ignite = startGrid(1);
 
@@ -833,7 +833,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If fail.
      */
-    public void _testEvictPartition() throws Exception {
+    public void testEvictPartition() throws Exception {
         try {
             Ignite ignite1 = startGrid("node1");
 
@@ -877,7 +877,7 @@ public class IgniteWalRecoveryTest extends GridCommonAbstractTest {
     /**
      * @throws Exception if failed.
      */
-    public void _testApplyDeltaRecords() throws Exception {
+    public void testApplyDeltaRecords() throws Exception {
         try {
             IgniteEx ignite0 = (IgniteEx)startGrid("node0");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed1da133/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest2.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest2.java
deleted file mode 100644
index 8296f40..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest2.java
+++ /dev/null
@@ -1,24 +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.cache.persistence.db.wal;
-
-/**
- *
- */
-public class IgniteWalRecoveryTest2 extends IgniteWalRecoveryTest {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed1da133/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest3.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest3.java
deleted file mode 100644
index d37ba5b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest3.java
+++ /dev/null
@@ -1,24 +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.cache.persistence.db.wal;
-
-/**
- *
- */
-public class IgniteWalRecoveryTest3 extends IgniteWalRecoveryTest {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed1da133/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest4.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest4.java
deleted file mode 100644
index ded580d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest4.java
+++ /dev/null
@@ -1,24 +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.cache.persistence.db.wal;
-
-/**
- *
- */
-public class IgniteWalRecoveryTest4 extends IgniteWalRecoveryTest {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed1da133/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest5.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest5.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest5.java
deleted file mode 100644
index 70b4cc0..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalRecoveryTest5.java
+++ /dev/null
@@ -1,24 +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.cache.persistence.db.wal;
-
-/**
- *
- */
-public class IgniteWalRecoveryTest5 extends IgniteWalRecoveryTest {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ed1da133/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
index 019d84c..bb9c9d1 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingCoreTestSuite.java
@@ -26,10 +26,6 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IgnitePdsPageE
 import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsCacheIntegrationTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.file.IgnitePdsNoActualWalHistoryTest;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest;
-import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest2;
-import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest3;
-import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest4;
-import org.apache.ignite.internal.processors.cache.persistence.db.wal.IgniteWalRecoveryTest5;
 import org.apache.ignite.internal.processors.cache.persistence.db.wal.WalRecoveryTxLogicalRecordsTest;
 
 /**
@@ -43,22 +39,18 @@ public class IgnitePdsWithIndexingCoreTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Ignite Persistent Store With Indexing Test Suite");
 
-//        suite.addTestSuite(IgnitePdsCacheIntegrationTest.class);
-//        suite.addTestSuite(IgnitePdsPageEvictionTest.class);
-//        suite.addTestSuite(IgnitePdsMultiNodePutGetRestartTest.class);
-//        suite.addTestSuite(IgnitePersistentStoreCacheGroupsTest.class);
-//        suite.addTestSuite(WalRecoveryTxLogicalRecordsTest.class);
+        suite.addTestSuite(IgnitePdsCacheIntegrationTest.class);
+        suite.addTestSuite(IgnitePdsPageEvictionTest.class);
+        suite.addTestSuite(IgnitePdsMultiNodePutGetRestartTest.class);
+        suite.addTestSuite(IgnitePersistentStoreCacheGroupsTest.class);
+        suite.addTestSuite(WalRecoveryTxLogicalRecordsTest.class);
 
         suite.addTestSuite(IgniteWalRecoveryTest.class);
-        suite.addTestSuite(IgniteWalRecoveryTest2.class);
-        suite.addTestSuite(IgniteWalRecoveryTest3.class);
-        suite.addTestSuite(IgniteWalRecoveryTest4.class);
-        suite.addTestSuite(IgniteWalRecoveryTest5.class);
-//        suite.addTestSuite(IgnitePdsNoActualWalHistoryTest.class);
-//        suite.addTestSuite(IgnitePdsAtomicCacheRebalancingTest.class);
-//        suite.addTestSuite(IgnitePdsTxCacheRebalancingTest.class);
-//
-//        suite.addTestSuite(IgnitePdsBinaryMetadataOnClusterRestartTest.class);
+        suite.addTestSuite(IgnitePdsNoActualWalHistoryTest.class);
+        suite.addTestSuite(IgnitePdsAtomicCacheRebalancingTest.class);
+        suite.addTestSuite(IgnitePdsTxCacheRebalancingTest.class);
+
+        suite.addTestSuite(IgnitePdsBinaryMetadataOnClusterRestartTest.class);
 
         return suite;
     }


[14/19] ignite git commit: IGNITE-6103 - Handle missed partition ID during WAL replay

Posted by ag...@apache.org.
IGNITE-6103 - Handle missed partition ID during WAL replay


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

Branch: refs/heads/ignite-5901
Commit: 15710a86977b803dbecc6c53b4036801c21fdcd9
Parents: c47c365
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Aug 17 18:09:50 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 18:10:14 2017 +0300

----------------------------------------------------------------------
 .../persistence/GridCacheDatabaseSharedManager.java    | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/15710a86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 3c7ba28..19af87b 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -1621,7 +1621,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
      * @param dataEntry Data entry to apply.
      */
     private void applyUpdate(GridCacheContext cacheCtx, DataEntry dataEntry) throws IgniteCheckedException {
-        GridDhtLocalPartition locPart = cacheCtx.topology().forceCreatePartition(dataEntry.partitionId());
+        int partId = dataEntry.partitionId();
+
+        if (partId == -1)
+            partId = cacheCtx.affinity().partition(dataEntry.key());
+
+        GridDhtLocalPartition locPart = cacheCtx.topology().forceCreatePartition(partId);
 
         switch (dataEntry.op()) {
             case CREATE:
@@ -1636,15 +1641,15 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                     null);
 
                 if (dataEntry.partitionCounter() != 0)
-                    cacheCtx.offheap().onPartitionInitialCounterUpdated(dataEntry.partitionId(), dataEntry.partitionCounter());
+                    cacheCtx.offheap().onPartitionInitialCounterUpdated(partId, dataEntry.partitionCounter());
 
                 break;
 
             case DELETE:
-                cacheCtx.offheap().remove(cacheCtx, dataEntry.key(), dataEntry.partitionId(), locPart);
+                cacheCtx.offheap().remove(cacheCtx, dataEntry.key(), partId, locPart);
 
                 if (dataEntry.partitionCounter() != 0)
-                    cacheCtx.offheap().onPartitionInitialCounterUpdated(dataEntry.partitionId(), dataEntry.partitionCounter());
+                    cacheCtx.offheap().onPartitionInitialCounterUpdated(partId, dataEntry.partitionCounter());
 
                 break;
 


[07/19] ignite git commit: IGNITE-5738: JDBC: add batch support. This closes #2393.

Posted by ag...@apache.org.
IGNITE-5738: JDBC: add batch support. This closes #2393.


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

Branch: refs/heads/ignite-5901
Commit: 7781823d4552bb30efa48758b7473d07c9e8aee3
Parents: 9cfb050
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Thu Aug 17 14:58:38 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 17 14:58:38 2017 +0300

----------------------------------------------------------------------
 .../jdbc2/JdbcDeleteStatementSelfTest.java      |  22 ++
 .../jdbc2/JdbcInsertStatementSelfTest.java      | 159 ++++++++++++++
 .../jdbc2/JdbcMergeStatementSelfTest.java       |  41 ++++
 .../jdbc2/JdbcStatementBatchingSelfTest.java    | 133 ++++++++++++
 .../jdbc2/JdbcUpdateStatementSelfTest.java      |  24 +++
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   3 +
 .../internal/jdbc2/JdbcBatchUpdateTask.java     | 215 +++++++++++++++++++
 .../internal/jdbc2/JdbcDatabaseMetadata.java    |   2 +-
 .../internal/jdbc2/JdbcPreparedStatement.java   |  25 ++-
 .../ignite/internal/jdbc2/JdbcStatement.java    |  59 ++++-
 10 files changed, 675 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java
index d55c979..3eec5a0 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcDeleteStatementSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.jdbc2;
 
+import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.HashSet;
@@ -46,4 +47,25 @@ public class JdbcDeleteStatementSelfTest extends JdbcAbstractUpdateStatementSelf
         assertFalse(jcache(0).containsKey("p2"));
         assertTrue(jcache(0).containsKeys(new HashSet<Object>(Arrays.asList("p1", "p3"))));
     }
+
+    /**
+     *
+     */
+    public void testBatch() throws SQLException {
+        PreparedStatement ps = conn.prepareStatement("delete from Person where firstName = ?");
+
+        ps.setString(1, "John");
+
+        ps.addBatch();
+
+        ps.setString(1, "Harry");
+
+        ps.addBatch();
+
+        int[] res = ps.executeBatch();
+
+        assertFalse(jcache(0).containsKey("p1"));
+        assertTrue(jcache(0).containsKeys(new HashSet<Object>(Arrays.asList("p2", "p3"))));
+        assertTrue(Arrays.equals(new int[] {1, 0}, res));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
index 0e7539f..407d6e2 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcInsertStatementSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.jdbc2;
 
+import java.sql.BatchUpdateException;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
@@ -26,6 +27,7 @@ import java.util.HashSet;
 import java.util.concurrent.Callable;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.testframework.GridTestUtils;
 
 /**
@@ -174,4 +176,161 @@ public class JdbcInsertStatementSelfTest extends JdbcAbstractDmlStatementSelfTes
 
         assertEquals(3, jcache(0).withKeepBinary().getAll(new HashSet<>(Arrays.asList("p1", "p2", "p3"))).size());
     }
+
+    /**
+     * @throws SQLException if failed.
+     */
+    public void testBatch() throws SQLException {
+        formBatch(1, 2);
+        formBatch(3, 4);
+
+        int[] res = prepStmt.executeBatch();
+
+        assertTrue(Arrays.equals(new int[] {2, 2}, res));
+    }
+
+    /**
+     * @throws SQLException if failed.
+     */
+    public void testSingleItemBatch() throws SQLException {
+        formBatch(1, 2);
+
+        int[] res = prepStmt.executeBatch();
+
+        assertTrue(Arrays.equals(new int[] {2}, res));
+    }
+
+    /**
+     * @throws SQLException if failed.
+     */
+    public void testSingleItemBatchError() throws SQLException {
+        formBatch(1, 2);
+
+        prepStmt.executeBatch();
+
+        formBatch(1, 2); // Duplicate key
+
+        BatchUpdateException reason = (BatchUpdateException)
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    return prepStmt.executeBatch();
+                }
+            },
+            BatchUpdateException.class,
+            "Failed to INSERT some keys because they are already in cache");
+
+        // Check update counts in the exception.
+        assertTrue(F.isEmpty(reason.getUpdateCounts()));
+    }
+
+    /**
+     * @throws SQLException if failed.
+     */
+    public void testErrorAmidstBatch() throws SQLException {
+        formBatch(1, 2);
+        formBatch(3, 1); // Duplicate key
+
+        BatchUpdateException reason = (BatchUpdateException)
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    return prepStmt.executeBatch();
+                }
+            },
+            BatchUpdateException.class,
+            "Failed to INSERT some keys because they are already in cache");
+
+        // Check update counts in the exception.
+        int[] counts = reason.getUpdateCounts();
+
+        assertNotNull(counts);
+
+        assertEquals(1, counts.length);
+        assertEquals(2, counts[0]);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClearBatch() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws SQLException {
+                return prepStmt.executeBatch();
+            }
+        }, SQLException.class, "Batch is empty");
+
+        formBatch(1, 2);
+
+        prepStmt.clearBatch();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws SQLException {
+                return prepStmt.executeBatch();
+            }
+        }, SQLException.class, "Batch is empty");
+    }
+
+    /**
+     * Form batch on prepared statement.
+     *
+     * @param id1 id for first row.
+     * @param id2 id for second row.
+     * @throws SQLException if failed.
+     */
+    private void formBatch(int id1, int id2) throws SQLException {
+        int[] ids = new int[] { id1, id2 };
+
+        int arg = 0;
+        for (int id: ids) {
+            String key = "p" + id;
+
+            switch (id) {
+                case 1:
+                    prepStmt.setString(arg + 1, key);
+                    prepStmt.setInt(arg + 2, 1);
+                    prepStmt.setString(arg + 3, "John");
+                    prepStmt.setString(arg + 4, "White");
+                    prepStmt.setInt(arg + 5, 25);
+                    prepStmt.setBytes(arg + 6, getBytes("White"));
+
+                    break;
+
+                case 2:
+                    prepStmt.setString(arg + 1, key);
+                    prepStmt.setInt(arg + 2, 2);
+                    prepStmt.setString(arg + 3, "Joe");
+                    prepStmt.setString(arg + 4, "Black");
+                    prepStmt.setInt(arg + 5, 35);
+                    prepStmt.setBytes(arg + 6, getBytes("Black"));
+
+                    break;
+
+                case 3:
+                    prepStmt.setString(arg + 1, key);
+                    prepStmt.setInt(arg + 2, 3);
+                    prepStmt.setString(arg + 3, "Mike");
+                    prepStmt.setString(arg + 4, "Green");
+                    prepStmt.setInt(arg + 5, 40);
+                    prepStmt.setBytes(arg + 6, getBytes("Green"));
+
+                    break;
+
+                case 4:
+                    prepStmt.setString(arg + 1, key);
+                    prepStmt.setInt(arg + 2, 4);
+                    prepStmt.setString(arg + 3, "Leah");
+                    prepStmt.setString(arg + 4, "Grey");
+                    prepStmt.setInt(arg + 5, 22);
+                    prepStmt.setBytes(arg + 6, getBytes("Grey"));
+
+                    break;
+
+                default:
+                    assert false;
+            }
+
+            arg += 6;
+        }
+
+        prepStmt.addBatch();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
index 1432a78..489bacd 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMergeStatementSelfTest.java
@@ -21,6 +21,7 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.Arrays;
 import org.apache.ignite.cache.CachePeekMode;
 
 /**
@@ -143,4 +144,44 @@ public class JdbcMergeStatementSelfTest extends JdbcAbstractDmlStatementSelfTest
 
         assertEquals(false, res);
     }
+
+    /**
+     * @throws SQLException if failed.
+     */
+    public void testBatch() throws SQLException {
+        prepStmt.setString(1, "p1");
+        prepStmt.setInt(2, 1);
+        prepStmt.setString(3, "John");
+        prepStmt.setString(4, "White");
+        prepStmt.setInt(5, 25);
+        prepStmt.setBytes(6, getBytes("White"));
+
+        prepStmt.setString(7, "p2");
+        prepStmt.setInt(8, 2);
+        prepStmt.setString(9, "Joe");
+        prepStmt.setString(10, "Black");
+        prepStmt.setInt(11, 35);
+        prepStmt.setBytes(12, getBytes("Black"));
+        prepStmt.addBatch();
+
+        prepStmt.setString(1, "p3");
+        prepStmt.setInt(2, 3);
+        prepStmt.setString(3, "Mike");
+        prepStmt.setString(4, "Green");
+        prepStmt.setInt(5, 40);
+        prepStmt.setBytes(6, getBytes("Green"));
+
+        prepStmt.setString(7, "p4");
+        prepStmt.setInt(8, 4);
+        prepStmt.setString(9, "Leah");
+        prepStmt.setString(10, "Grey");
+        prepStmt.setInt(11, 22);
+        prepStmt.setBytes(12, getBytes("Grey"));
+
+        prepStmt.addBatch();
+
+        int[] res = prepStmt.executeBatch();
+
+        assertTrue(Arrays.equals(new int[] {2, 2}, res));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementBatchingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementBatchingSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementBatchingSelfTest.java
new file mode 100644
index 0000000..c9169b9
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcStatementBatchingSelfTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.jdbc2;
+
+import java.sql.BatchUpdateException;
+import java.sql.DatabaseMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.concurrent.Callable;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Statement batch test.
+ */
+public class JdbcStatementBatchingSelfTest extends JdbcAbstractDmlStatementSelfTest {
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        jcache(0).clear();
+    }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    public void testDatabaseMetadataBatchSupportFlag() throws SQLException {
+        DatabaseMetaData meta = conn.getMetaData();
+
+        assertNotNull(meta);
+
+        assertTrue(meta.supportsBatchUpdates());
+    }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    public void testBatch() throws SQLException {
+        try (Statement stmt = conn.createStatement()) {
+            stmt.addBatch("INSERT INTO Person(_key, id, firstName, lastName, age, data) " +
+                "VALUES ('p1', 0, 'J', 'W', 250, RAWTOHEX('W'))");
+
+            stmt.addBatch("MERGE INTO Person(_key, id, firstName, lastName, age, data) VALUES " +
+                "('p1', 1, 'John', 'White', 25, RAWTOHEX('White')), " +
+                "('p2', 2, 'Joe', 'Black', 35, RAWTOHEX('Black')), " +
+                "('p3', 0, 'M', 'G', 4, RAWTOHEX('G'))");
+
+            stmt.addBatch("UPDATE Person SET id = 3, firstName = 'Mike', lastName = 'Green', " +
+                "age = 40, data = RAWTOHEX('Green') WHERE _key = 'p3'");
+
+            stmt.addBatch("DELETE FROM Person WHERE _key = 'p1'");
+
+            int[] res = stmt.executeBatch();
+
+            assertEquals(4, res.length);
+            assertEquals(1, res[0]);
+            assertEquals(3, res[1]);
+            assertEquals(1, res[2]);
+            assertEquals(1, res[3]);
+        }
+    }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    public void testErrorAmidstBatch() throws SQLException {
+        BatchUpdateException reason = (BatchUpdateException)
+            GridTestUtils.assertThrows(log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    try (Statement stmt = conn.createStatement()) {
+                        stmt.addBatch("INSERT INTO Person(_key, id, firstName, lastName, age, data) " +
+                            "VALUES ('p1', 0, 'J', 'W', 250, RAWTOHEX('W'))");
+
+                        stmt.addBatch("UPDATE Person SET id = 3, firstName = 'Mike', lastName = 'Green', " +
+                            "age = 40, data = RAWTOHEX('Green') WHERE _key = 'p3'");
+
+                        stmt.addBatch("SELECT id FROM Person WHERE _key = 'p1'");
+
+                        return stmt.executeBatch();
+                    }
+                }
+            },
+            BatchUpdateException.class,
+            "Given statement type does not match that declared by JDBC driver");
+
+        // Check update counts in the exception.
+        int[] counts = reason.getUpdateCounts();
+
+        assertEquals(2, counts.length);
+        assertEquals(1, counts[0]);
+        assertEquals(0, counts[1]);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClearBatch() throws Exception {
+        try (Statement stmt = conn.createStatement()) {
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws SQLException {
+                    return stmt.executeBatch();
+                }
+            }, SQLException.class, "Batch is empty");
+
+            stmt.addBatch("INSERT INTO Person(_key, id, firstName, lastName, age, data) " +
+                "VALUES ('p1', 0, 'J', 'W', 250, RAWTOHEX('W'))");
+
+            stmt.clearBatch();
+
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws SQLException {
+                    return stmt.executeBatch();
+                }
+            }, SQLException.class, "Batch is empty");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcUpdateStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcUpdateStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcUpdateStatementSelfTest.java
index 8ae0e90..07b5587 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcUpdateStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcUpdateStatementSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.jdbc2;
 
+import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.Arrays;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
@@ -47,4 +48,27 @@ public class JdbcUpdateStatementSelfTest extends JdbcAbstractUpdateStatementSelf
         assertEquals(Arrays.asList(F.asList("John"), F.asList("Jack"), F.asList("Mike")),
                 jcache(0).query(new SqlFieldsQuery("select firstName from Person order by _key")).getAll());
     }
+
+    /**
+     * @throws SQLException If failed.
+     */
+    public void testBatch() throws SQLException {
+        PreparedStatement ps = conn.prepareStatement("update Person set lastName = concat(firstName, 'son') " +
+            "where firstName = ?");
+
+        ps.setString(1, "John");
+
+        ps.addBatch();
+
+        ps.setString(1, "Harry");
+
+        ps.addBatch();
+
+        int[] res = ps.executeBatch();
+
+        assertEquals(Arrays.asList(F.asList("Johnson"), F.asList("Black"), F.asList("Green")),
+            jcache(0).query(new SqlFieldsQuery("select lastName from Person order by _key")).getAll());
+
+        assertTrue(Arrays.equals(new int[] {1, 0}, res));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index cf7ee8f..a20002b 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -93,9 +93,12 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDefaultNoOpCacheTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcMergeStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerMergeStatementSelfTest.class));
+        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcUpdateStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcInsertStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcBinaryMarshallerInsertStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcDeleteStatementSelfTest.class));
+        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStatementBatchingSelfTest.class));
+
         suite.addTest(new TestSuite(JdbcBlobTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStreamingSelfTest.class));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBatchUpdateTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBatchUpdateTask.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBatchUpdateTask.java
new file mode 100644
index 0000000..7b4846c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcBatchUpdateTask.java
@@ -0,0 +1,215 @@
+/*
+ * 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.jdbc2;
+
+import java.sql.BatchUpdateException;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteJdbcDriver;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.resources.IgniteInstanceResource;
+
+import static java.sql.Statement.SUCCESS_NO_INFO;
+
+/**
+ * Task for SQL batched update statements execution through {@link IgniteJdbcDriver}.
+ */
+class JdbcBatchUpdateTask implements IgniteCallable<int[]> {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Ignite. */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** Cache name. */
+    private final String cacheName;
+
+    /** Schema name. */
+    private final String schemaName;
+
+    /** SQL command for argument batching. */
+    private final String sql;
+
+    /** Batch of statements. */
+    private final List<String> sqlBatch;
+
+    /** Batch of arguments. */
+    private final List<List<Object>> batchArgs;
+
+    /** Fetch size. */
+    private final int fetchSize;
+
+    /** Local execution flag. */
+    private final boolean loc;
+
+    /** Local query flag. */
+    private final boolean locQry;
+
+    /** Collocated query flag. */
+    private final boolean collocatedQry;
+
+    /** Distributed joins flag. */
+    private final boolean distributedJoins;
+
+    /**
+     * @param ignite Ignite.
+     * @param cacheName Cache name.
+     * @param schemaName Schema name.
+     * @param sql SQL query. {@code null} in case of statement batching.
+     * @param sqlBatch Batch of SQL statements. {@code null} in case of parameter batching.
+     * @param batchArgs Batch of SQL parameters. {@code null} in case of statement batching.
+     * @param loc Local execution flag.
+     * @param fetchSize Fetch size.
+     * @param locQry Local query flag.
+     * @param collocatedQry Collocated query flag.
+     * @param distributedJoins Distributed joins flag.
+     */
+    public JdbcBatchUpdateTask(Ignite ignite, String cacheName, String schemaName, String sql,
+        List<String> sqlBatch, List<List<Object>> batchArgs, boolean loc, int fetchSize,
+        boolean locQry, boolean collocatedQry, boolean distributedJoins) {
+        this.ignite = ignite;
+        this.cacheName = cacheName;
+        this.schemaName = schemaName;
+        this.sql = sql;
+        this.sqlBatch = sqlBatch;
+        this.batchArgs = batchArgs;
+        this.fetchSize = fetchSize;
+        this.loc = loc;
+        this.locQry = locQry;
+        this.collocatedQry = collocatedQry;
+        this.distributedJoins = distributedJoins;
+
+        assert (!F.isEmpty(sql) && !F.isEmpty(batchArgs)) ^ !F.isEmpty(sqlBatch);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] call() throws Exception {
+        IgniteCache<?, ?> cache = ignite.cache(cacheName);
+
+        // Don't create caches on server nodes in order to avoid of data rebalancing.
+        boolean start = ignite.configuration().isClientMode();
+
+        if (cache == null && cacheName == null)
+            cache = ((IgniteKernal)ignite).context().cache().getOrStartPublicCache(start, !loc && locQry);
+
+        if (cache == null) {
+            if (cacheName == null)
+                throw new SQLException("Failed to execute query. No suitable caches found.");
+            else
+                throw new SQLException("Cache not found [cacheName=" + cacheName + ']');
+        }
+
+        int batchSize = F.isEmpty(sql) ? sqlBatch.size() : batchArgs.size();
+
+        int[] updCntrs = new int[batchSize];
+
+        int idx = 0;
+
+        try {
+            if (F.isEmpty(sql)) {
+                for (; idx < batchSize; idx++)
+                    updCntrs[idx] = doSingleUpdate(cache, sqlBatch.get(idx), null);
+            }
+            else {
+                for (; idx < batchSize; idx++)
+                    updCntrs[idx] = doSingleUpdate(cache, sql, batchArgs.get(idx));
+            }
+        }
+        catch (Exception ex) {
+            throw new BatchUpdateException(Arrays.copyOf(updCntrs, idx), ex);
+        }
+
+        return updCntrs;
+    }
+
+    /**
+     * Performs update.
+     *
+     * @param cache Cache.
+     * @param sqlText SQL text.
+     * @param args Parameters.
+     * @return Update counter.
+     * @throws SQLException If failed.
+     */
+    private Integer doSingleUpdate(IgniteCache<?, ?> cache, String sqlText, List<Object> args) throws SQLException {
+        SqlFieldsQuery qry = new JdbcSqlFieldsQuery(sqlText, false);
+
+        qry.setPageSize(fetchSize);
+        qry.setLocal(locQry);
+        qry.setCollocated(collocatedQry);
+        qry.setDistributedJoins(distributedJoins);
+        qry.setSchema(schemaName);
+        qry.setArgs(args == null ? null : args.toArray());
+
+        QueryCursorImpl<List<?>> qryCursor = (QueryCursorImpl<List<?>>)cache.withKeepBinary().query(qry);
+
+        if (qryCursor.isQuery())
+            throw new SQLException(getError("Query produced result set", qry));
+
+        List<List<?>> rows = qryCursor.getAll();
+
+        if (F.isEmpty(rows))
+            return SUCCESS_NO_INFO;
+
+        if (rows.size() != 1)
+            throw new SQLException(getError("Expected single row for update operation result", qry));
+
+        List<?> row = rows.get(0);
+
+        if (F.isEmpty(row) || row.size() != 1)
+            throw new SQLException(getError("Expected row size of 1 for update operation", qry));
+
+        Object objRes = row.get(0);
+
+        if (!(objRes instanceof Long))
+            throw new SQLException(getError("Unexpected update result type", qry));
+
+        Long longRes = (Long)objRes;
+
+        if (longRes > Integer.MAX_VALUE) {
+            IgniteLogger log = ignite.log();
+
+            if (log != null)
+                log.warning(getError("Query updated row counter (" + longRes + ") exceeds integer range", qry));
+
+            return Integer.MAX_VALUE;
+        }
+
+        return longRes.intValue();
+    }
+
+    /**
+     * Formats error message with query details.
+     *
+     * @param msg Error message.
+     * @param qry Query.
+     * @return Result.
+     */
+    private String getError(String msg, SqlFieldsQuery qry) {
+        return msg + " [qry='" + qry.getSql() + "', params=" + Arrays.deepToString(qry.getArgs()) + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
index 98a2563..b369b0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcDatabaseMetadata.java
@@ -1063,7 +1063,7 @@ public class JdbcDatabaseMetadata implements DatabaseMetaData {
 
     /** {@inheritDoc} */
     @Override public boolean supportsBatchUpdates() throws SQLException {
-        return false;
+        return true;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
index 16030f7..38dfe02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatement.java
@@ -39,6 +39,7 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Calendar;
+import java.util.List;
 
 /**
  * JDBC prepared statement implementation.
@@ -50,6 +51,9 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     /** H2's parsed statement to retrieve metadata from. */
     PreparedStatement nativeStatement;
 
+    /** Batch arguments. */
+    private List<List<Object>> batchArgs;
+
     /**
      * Creates new prepared statement.
      *
@@ -66,7 +70,8 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     @Override public void addBatch(String sql) throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Adding new SQL command to batch not supported for prepared statement.");
+        throw new SQLFeatureNotSupportedException("Adding new SQL command to batch is not supported for prepared " +
+            "statement (use addBatch() to add new set of arguments)");
     }
 
     /** {@inheritDoc} */
@@ -185,7 +190,7 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     @Override public void clearBatch() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Batch statements are not supported yet.");
+        batchArgs = null;
     }
 
     /** {@inheritDoc} */
@@ -207,14 +212,26 @@ public class JdbcPreparedStatement extends JdbcStatement implements PreparedStat
     @Override public void addBatch() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Batch statements are not supported yet.");
+        if (batchArgs == null)
+            batchArgs = new ArrayList<>();
+
+        batchArgs.add(args);
+
+        args = null;
     }
 
     /** {@inheritDoc} */
     @Override public int[] executeBatch() throws SQLException {
-        throw new SQLFeatureNotSupportedException("Batch statements are not supported yet.");
+        ensureNotClosed();
+
+        List<List<Object>> batchArgs = this.batchArgs;
+
+        this.batchArgs = null;
+
+        return doBatchUpdate(sql, null, batchArgs);
     }
 
+
     /** {@inheritDoc} */
     @Override public void setCharacterStream(int paramIdx, Reader x, int len) throws SQLException {
         ensureNotClosed();

http://git-wip-us.apache.org/repos/asf/ignite/blob/7781823d/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
index 89a80ca..19c20a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcStatement.java
@@ -74,7 +74,7 @@ public class JdbcStatement implements Statement {
     /** Current updated items count. */
     long updateCnt = -1;
 
-    /** Batch statements. */
+    /** Batch of statements. */
     private List<String> batch;
 
     /**
@@ -187,7 +187,7 @@ public class JdbcStatement implements Statement {
 
     /**
      * @param rows query result.
-     * @return update counter, if found
+     * @return update counter, if found.
      * @throws SQLException if getting an update counter from result proved to be impossible.
      */
     private static long updateCounterFromQueryResult(List<List<?>> rows) throws SQLException {
@@ -461,7 +461,60 @@ public class JdbcStatement implements Statement {
     @Override public int[] executeBatch() throws SQLException {
         ensureNotClosed();
 
-        throw new SQLFeatureNotSupportedException("Batch statements are not supported yet.");
+        List<String> batch = this.batch;
+
+        this.batch = null;
+
+        return doBatchUpdate(null, batch, null);
+    }
+
+    /**
+     * Runs batch of update commands.
+     *
+     * @param command SQL command.
+     * @param batch Batch of SQL commands.
+     * @param batchArgs Batch of SQL parameters.
+     * @return Number of affected rows.
+     * @throws SQLException If failed.
+     */
+    protected int[] doBatchUpdate(String command, List<String> batch, List<List<Object>> batchArgs)
+        throws SQLException {
+        rs = null;
+
+        updateCnt = -1;
+
+        if ((F.isEmpty(command) || F.isEmpty(batchArgs)) && F.isEmpty(batch))
+            throw new SQLException("Batch is empty.");
+
+        Ignite ignite = conn.ignite();
+
+        UUID nodeId = conn.nodeId();
+
+        boolean loc = nodeId == null;
+
+        if (!conn.isDmlSupported())
+            throw new SQLException("Failed to query Ignite: DML operations are supported in versions 1.8.0 and newer");
+
+        JdbcBatchUpdateTask task = new JdbcBatchUpdateTask(loc ? ignite : null, conn.cacheName(),
+            conn.schemaName(), command, batch, batchArgs, loc, getFetchSize(), conn.isLocalQuery(),
+            conn.isCollocatedQuery(), conn.isDistributedJoins());
+
+        try {
+            int[] res = loc ? task.call() : ignite.compute(ignite.cluster().forNodeId(nodeId)).call(task);
+
+            updateCnt = F.isEmpty(res)? -1 : res[res.length - 1];
+
+            return res;
+        }
+        catch (IgniteSQLException e) {
+            throw e.toJdbcException();
+        }
+        catch (SQLException e) {
+            throw e;
+        }
+        catch (Exception e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
     }
 
     /** {@inheritDoc} */


[04/19] ignite git commit: IGNITE-5233: JDBC thind driver: implemented metadata methods. This closes #2079.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinParameterMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinParameterMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinParameterMetadata.java
new file mode 100644
index 0000000..8647258
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinParameterMetadata.java
@@ -0,0 +1,115 @@
+/*
+ * 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.jdbc.thin;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import java.util.List;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcParameterMeta;
+
+/**
+ * JDBC SQL query's parameters metadata.
+ */
+public class JdbcThinParameterMetadata implements ParameterMetaData {
+    /** Parameters metadata. */
+    private final List<JdbcParameterMeta> meta;
+
+    /**
+     * @param meta Parameters metadata.
+     */
+    public JdbcThinParameterMetadata(List<JdbcParameterMeta> meta) {
+        assert meta != null;
+
+        this.meta = meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getParameterCount() throws SQLException {
+        return meta.size();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("MagicConstant")
+    @Override public int isNullable(int param) throws SQLException {
+        return parameter(param).isNullable();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isSigned(int param) throws SQLException {
+        return parameter(param).isSigned();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getPrecision(int param) throws SQLException {
+        return parameter(param).precision();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getScale(int param) throws SQLException {
+        return parameter(param).scale();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getParameterType(int param) throws SQLException {
+        return parameter(param).type();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getParameterTypeName(int param) throws SQLException {
+        return parameter(param).typeName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getParameterClassName(int param) throws SQLException {
+        return parameter(param).typeClass();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("MagicConstant")
+    @Override public int getParameterMode(int param) throws SQLException {
+        return parameter(param).mode();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!isWrapperFor(iface))
+            throw new SQLException("Parameters metadata is not a wrapper for " + iface.getName());
+
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface != null && iface.isAssignableFrom(JdbcThinParameterMetadata.class);
+    }
+
+    /**
+     * Bounds checks the parameter index.
+     *
+     * @param param Parameter index.
+     * @return Parameter.
+     * @throws SQLException If failed.
+     */
+    private JdbcParameterMeta parameter(int param) throws SQLException {
+        if (param <= 0 || param > meta.size())
+            throw new SQLException("Invalid parameter number");
+
+        return meta.get(param - 1);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
index 455c80f..e6dfa59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinPreparedStatement.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.jdbc.thin;
 
+import java.io.IOException;
 import java.io.InputStream;
 import java.io.Reader;
 import java.math.BigDecimal;
@@ -39,7 +40,9 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Calendar;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsResult;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery;
 
 /**
@@ -52,6 +55,9 @@ public class JdbcThinPreparedStatement extends JdbcThinStatement implements Prep
     /** Query arguments. */
     protected ArrayList<Object> args;
 
+    /** Parameters metadata. */
+    private JdbcThinParameterMetadata metaData;
+
     /**
      * Creates new prepared statement.
      *
@@ -322,8 +328,27 @@ public class JdbcThinPreparedStatement extends JdbcThinStatement implements Prep
     /** {@inheritDoc} */
     @Override public ParameterMetaData getParameterMetaData() throws SQLException {
         ensureNotClosed();
+        try {
+            if (conn.isClosed())
+                throw new SQLException("Connection is closed.");
 
-        throw new SQLFeatureNotSupportedException("Meta data for prepared statement is not supported.");
+            if (metaData != null)
+                return metaData;
+
+            JdbcMetaParamsResult res = conn.io().parametersMeta(conn.getSchema(), sql);
+
+            metaData = new JdbcThinParameterMetadata(res.meta());
+
+            return metaData;
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
index 5c61e23..c4be5bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinResultSet.java
@@ -100,6 +100,34 @@ public class JdbcThinResultSet implements ResultSet {
     /** Update count. */
     private long updCnt;
 
+    /** Jdbc metadata. Cache the JDBC object on the first access */
+    private JdbcThinResultSetMetadata jdbcMeta;
+
+    /**
+     * Constructs static result set.
+     *
+     * @param fields Fields.
+     * @param meta Columns metadata.
+     */
+    JdbcThinResultSet(List<List<Object>> fields, List<JdbcColumnMeta> meta) {
+        stmt = null;
+        fetchSize = 0;
+        qryId = -1L;
+        finished = true;
+        isQuery = true;
+        updCnt = -1;
+
+        this.rows = fields;
+
+        rowsIter = fields.iterator();
+
+        this.meta = meta;
+
+        metaInit = true;
+
+        initColumnOrder();
+    }
+
     /**
      * Creates new result set.
      *
@@ -180,7 +208,7 @@ public class JdbcThinResultSet implements ResultSet {
 
     /** {@inheritDoc} */
     @Override public void close() throws SQLException {
-        if (closed || stmt.connection().isClosed())
+        if (closed || stmt == null || stmt.connection().isClosed())
             return;
 
         try {
@@ -497,7 +525,10 @@ public class JdbcThinResultSet implements ResultSet {
     @Override public ResultSetMetaData getMetaData() throws SQLException {
         ensureNotClosed();
 
-        return new JdbcThinResultSetMetadata(meta());
+        if (jdbcMeta == null)
+            jdbcMeta = new JdbcThinResultSetMetadata(meta());
+
+        return jdbcMeta;
     }
 
     /** {@inheritDoc} */
@@ -1648,7 +1679,6 @@ public class JdbcThinResultSet implements ResultSet {
     }
 
     /**
-     * Init column order map.
      * @throws SQLException On error.
      * @return Column order map.
      */
@@ -1659,6 +1689,15 @@ public class JdbcThinResultSet implements ResultSet {
         if(!metaInit)
             meta();
 
+        initColumnOrder();
+
+        return colOrder;
+    }
+
+    /**
+     * Init column order map.
+     */
+    private void initColumnOrder() {
         colOrder = new HashMap<>(meta.size());
 
         for (int i = 0; i < meta.size(); ++i) {
@@ -1667,8 +1706,6 @@ public class JdbcThinResultSet implements ResultSet {
             if(!colOrder.containsKey(colName))
                 colOrder.put(colName, i);
         }
-
-        return colOrder;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
index b01350a..3772b83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java
@@ -46,8 +46,8 @@ public class JdbcThinStatement implements Statement {
     /** Default queryPage size. */
     private static final int DFLT_PAGE_SIZE = SqlQuery.DFLT_PAGE_SIZE;
 
-    /** Ignite endpoint and I/O protocol implementation. */
-    private JdbcThinConnection conn;
+    /** JDBC Connection implementation. */
+    protected JdbcThinConnection conn;
 
     /** Closed flag. */
     private boolean closed;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
index f54d5fd..e124921 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java
@@ -34,6 +34,18 @@ import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
 import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteRequest;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcBatchExecuteResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaColumnsRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaColumnsResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaIndexesRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaIndexesResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaParamsResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaPrimaryKeysRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaPrimaryKeysResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaSchemasRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaSchemasResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaTablesRequest;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaTablesResult;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQuery;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryCloseRequest;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcQueryExecuteRequest;
@@ -47,6 +59,7 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResponse;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult;
 import org.apache.ignite.internal.util.ipc.loopback.IpcClientTcpEndpoint;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteProductVersion;
 
 /**
  * JDBC IO layer implementation based on blocking IPC streams.
@@ -59,7 +72,7 @@ public class JdbcThinTcpIo {
     private static final int HANDSHAKE_MSG_SIZE = 13;
 
     /** Initial output for query message. */
-    private static final int QUERY_EXEC_MSG_INIT_CAP = 256;
+    private static final int DYNAMIC_SIZE_MSG_CAP = 256;
 
     /** Maximum batch query count. */
     private static final int MAX_BATCH_QRY_CNT = 32;
@@ -115,6 +128,9 @@ public class JdbcThinTcpIo {
     /** Closed flag. */
     private boolean closed;
 
+    /** Ignite server version. */
+    private IgniteProductVersion igniteVer;
+
     /**
      * Constructor.
      *
@@ -202,8 +218,20 @@ public class JdbcThinTcpIo {
 
         boolean accepted = reader.readBoolean();
 
-        if (accepted)
+        if (accepted) {
+            byte maj = reader.readByte();
+            byte min = reader.readByte();
+            byte maintenance = reader.readByte();
+
+            String stage = reader.readString();
+
+            long ts = reader.readLong();
+            byte[] hash = reader.readByteArray();
+
+            igniteVer = new IgniteProductVersion(maj, min, maintenance, stage, ts, hash);
+
             return;
+        }
 
         short maj = reader.readShort();
         short min = reader.readShort();
@@ -231,7 +259,7 @@ public class JdbcThinTcpIo {
         String sql, List<Object> args)
         throws IOException, IgniteCheckedException {
         return sendRequest(new JdbcQueryExecuteRequest(cache, fetchSize, maxRows, sql,
-            args == null ? null : args.toArray(new Object[args.size()])), QUERY_EXEC_MSG_INIT_CAP);
+            args == null ? null : args.toArray(new Object[args.size()])), DYNAMIC_SIZE_MSG_CAP);
     }
 
     /**
@@ -295,21 +323,89 @@ public class JdbcThinTcpIo {
     }
 
     /**
-     * @param schema Schema.
+     * @param schemaName Schema.
      * @param batch Batch queries.
      * @return Result.
      * @throws IOException On error.
      * @throws IgniteCheckedException On error.
      */
-    public JdbcBatchExecuteResult batchExecute(String schema, List<JdbcQuery> batch)
+    public JdbcBatchExecuteResult batchExecute(String schemaName, List<JdbcQuery> batch)
         throws IOException, IgniteCheckedException {
         int cnt = Math.min(MAX_BATCH_QRY_CNT, batch.size());
 
-        return sendRequest(new JdbcBatchExecuteRequest(schema, batch), QUERY_EXEC_MSG_INIT_CAP * cnt);
+        return sendRequest(new JdbcBatchExecuteRequest(schemaName, batch), DYNAMIC_SIZE_MSG_CAP * cnt);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @param tablePtrn Table name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaTablesResult tablesMeta(String schemaPtrn, String tablePtrn)
+        throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaTablesRequest(schemaPtrn, tablePtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @param tablePtrn Table name pattern.
+     * @param columnPtrn Column name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaColumnsResult columnsMeta(String schemaPtrn, String tablePtrn, String columnPtrn)
+        throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaColumnsRequest(schemaPtrn, tablePtrn, columnPtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @param tablePtrn Table name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaIndexesResult indexMeta(String schemaPtrn, String tablePtrn) throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaIndexesRequest(schemaPtrn, tablePtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @param sql SQL query.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaParamsResult parametersMeta(String schemaPtrn, String sql) throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaParamsRequest(schemaPtrn, sql), DYNAMIC_SIZE_MSG_CAP);
     }
 
     /**
-     * @param req ODBC request.
+     * @param schemaPtrn Schema name pattern.
+     * @param tablePtrn Table name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaPrimaryKeysResult primaryKeysMeta(String schemaPtrn, String tablePtrn) throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaPrimaryKeysRequest(schemaPtrn, tablePtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param schemaPtrn Schema name pattern.
+     * @return Result.
+     * @throws IOException On error.
+     * @throws IgniteCheckedException On error.
+     */
+    public JdbcMetaSchemasResult schemasMeta(String schemaPtrn) throws IOException, IgniteCheckedException {
+        return sendRequest(new JdbcMetaSchemasRequest(schemaPtrn), DYNAMIC_SIZE_MSG_CAP);
+    }
+
+    /**
+     * @param req JDBC request bytes.
      * @throws IOException On error.
      */
     private void send(byte[] req) throws IOException {
@@ -434,4 +530,11 @@ public class JdbcThinTcpIo {
     public boolean tcpNoDelay() {
         return tcpNoDelay;
     }
+
+    /**
+     * @return Ignnite server version.
+     */
+    IgniteProductVersion igniteVersion() {
+        return igniteVer;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
index a879796..5a49e3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerNioListener.java
@@ -198,9 +198,11 @@ public class SqlListenerNioListener extends GridNioServerListenerAdapter<byte[]>
 
         String errMsg = null;
 
+        SqlListenerConnectionContext connCtx = null;
+
         if (SUPPORTED_VERS.contains(ver)) {
             // Prepare context.
-            SqlListenerConnectionContext connCtx = prepareContext(ver, reader);
+            connCtx = prepareContext(ver, reader);
 
             ses.addMeta(CONN_CTX_META_KEY, connCtx);
         }
@@ -213,9 +215,10 @@ public class SqlListenerNioListener extends GridNioServerListenerAdapter<byte[]>
         // Send response.
         BinaryWriterExImpl writer = new BinaryWriterExImpl(null, new BinaryHeapOutputStream(8), null, null);
 
-        if (errMsg == null)
-            writer.writeBoolean(true);
+        if (connCtx != null)
+            connCtx.handler().writeHandshake(writer);
         else {
+            // Failed handshake response
             writer.writeBoolean(false);
             writer.writeShort(CURRENT_VER.major());
             writer.writeShort(CURRENT_VER.minor());

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
index 98dc039..348054f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/SqlListenerRequestHandler.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.odbc;
 
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+
 /**
  * SQL listener request handler.
  */
@@ -36,4 +38,11 @@ public interface SqlListenerRequestHandler {
      * @return Error response.
      */
     public SqlListenerResponse handleException(Exception e);
+
+    /**
+     * Write successful handshake response.
+     *
+     * @param writer Binary writer.
+     */
+    public void writeHandshake(BinaryWriterExImpl writer);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java
index 9f71bff..25e1049 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteRequest.java
@@ -31,8 +31,8 @@ import org.jetbrains.annotations.Nullable;
  * JDBC batch execute request.
  */
 public class JdbcBatchExecuteRequest extends JdbcRequest {
-    /** Cache name. */
-    private String schema;
+    /** Schema name. */
+    private String schemaName;
 
     /** Sql query. */
     @GridToStringInclude(sensitive = true)
@@ -46,23 +46,23 @@ public class JdbcBatchExecuteRequest extends JdbcRequest {
     }
 
     /**
-     * @param schema Schema.
+     * @param schemaName Schema name.
      * @param queries Queries.
      */
-    public JdbcBatchExecuteRequest(String schema, List<JdbcQuery> queries) {
+    public JdbcBatchExecuteRequest(String schemaName, List<JdbcQuery> queries) {
         super(BATCH_EXEC);
 
         assert !F.isEmpty(queries);
 
-        this.schema = schema;
+        this.schemaName = schemaName;
         this.queries = queries;
     }
 
     /**
-     * @return Schema.
+     * @return Schema name.
      */
-    @Nullable public String schema() {
-        return schema;
+    @Nullable public String schemaName() {
+        return schemaName;
     }
 
     /**
@@ -76,7 +76,7 @@ public class JdbcBatchExecuteRequest extends JdbcRequest {
     @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
         super.writeBinary(writer);
 
-        writer.writeString(schema);
+        writer.writeString(schemaName);
         writer.writeInt(queries.size());
 
         for (JdbcQuery q : queries)
@@ -87,7 +87,7 @@ public class JdbcBatchExecuteRequest extends JdbcRequest {
     @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
         super.readBinary(reader);
 
-        schema = reader.readString();
+        schemaName = reader.readString();
 
         int n = reader.readInt();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java
index 7977c22..917e60a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcBatchExecuteResult.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.odbc.jdbc;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * JDBC batch execute result.
@@ -93,4 +94,9 @@ public class JdbcBatchExecuteResult extends JdbcResult {
         errMsg = reader.readString();
         updateCnts = reader.readIntArray();
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcBatchExecuteResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
index 07cbabe..9f145e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
@@ -21,19 +21,21 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.jdbc.thin.JdbcThinUtils;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener column metadata.
+ * JDBC column metadata.
  */
 public class JdbcColumnMeta implements JdbcRawBinarylizable {
     /** Cache name. */
     private String schemaName;
 
     /** Table name. */
-    private String tableName;
+    private String tblName;
 
     /** Column name. */
-    private String columnName;
+    private String colName;
 
     /** Data type. */
     private int dataType;
@@ -47,16 +49,17 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     /**
      * Default constructor is used for serialization.
      */
-    public JdbcColumnMeta() {
+    JdbcColumnMeta() {
+        // No-op.
     }
 
     /**
      * @param info Field metadata.
      */
-    public JdbcColumnMeta(GridQueryFieldMetadata info) {
+    JdbcColumnMeta(GridQueryFieldMetadata info) {
         this.schemaName = info.schemaName();
-        this.tableName = info.typeName();
-        this.columnName = info.fieldName();
+        this.tblName = info.typeName();
+        this.colName = info.fieldName();
 
         dataType = JdbcThinUtils.type(info.fieldTypeName());
         dataTypeName = JdbcThinUtils.typeName(info.fieldTypeName());
@@ -64,6 +67,24 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     }
 
     /**
+     * @param schemaName Schema.
+     * @param tblName Table.
+     * @param colName Column.
+     * @param cls Type.
+     */
+    public JdbcColumnMeta(String schemaName, String tblName, String colName, Class<?> cls) {
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+        this.colName = colName;
+
+        String type = cls.getName();
+
+        dataType = JdbcThinUtils.type(type);
+        dataTypeName = JdbcThinUtils.typeName(type);
+        dataTypeClass = type;
+    }
+
+    /**
      * @return Schema name.
      */
     public String schemaName() {
@@ -74,14 +95,14 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
      * @return Table name.
      */
     public String tableName() {
-        return tableName;
+        return tblName;
     }
 
     /**
      * @return Column name.
      */
     public String columnName() {
-        return columnName;
+        return colName;
     }
 
     /**
@@ -108,8 +129,8 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     /** {@inheritDoc} */
     @Override public void writeBinary(BinaryWriterExImpl writer) {
         writer.writeString(schemaName);
-        writer.writeString(tableName);
-        writer.writeString(columnName);
+        writer.writeString(tblName);
+        writer.writeString(colName);
 
         writer.writeInt(dataType);
         writer.writeString(dataTypeName);
@@ -119,11 +140,39 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     /** {@inheritDoc} */
     @Override public void readBinary(BinaryReaderExImpl reader) {
         schemaName = reader.readString();
-        tableName = reader.readString();
-        columnName = reader.readString();
+        tblName = reader.readString();
+        colName = reader.readString();
 
         dataType = reader.readInt();
         dataTypeName = reader.readString();
         dataTypeClass = reader.readString();
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        JdbcColumnMeta meta = (JdbcColumnMeta)o;
+
+        return F.eq(schemaName, meta.schemaName) && F.eq(tblName, meta.tblName) && F.eq(colName, meta.colName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = schemaName != null ? schemaName.hashCode() : 0;
+
+        result = 31 * result + (tblName != null ? tblName.hashCode() : 0);
+        result = 31 * result + colName.hashCode();
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcColumnMeta.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java
new file mode 100644
index 0000000..d33f887
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcIndexMeta.java
@@ -0,0 +1,192 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC index metadata.
+ */
+public class JdbcIndexMeta implements JdbcRawBinarylizable {
+    /** Index schema name. */
+    private String schemaName;
+
+    /** Index table name. */
+    private String tblName;
+
+    /** Index name. */
+    private String idxName;
+
+    /** Index type. */
+    private QueryIndexType type;
+
+    /** Index fields */
+    private List<String> fields;
+
+    /** Index fields is ascending. */
+    private List<Boolean> fieldsAsc;
+
+    /**
+     * Default constructor is used for binary serialization.
+     */
+    JdbcIndexMeta() {
+        // No-op.
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param idx Index info.
+     */
+    JdbcIndexMeta(String schemaName, String tblName, GridQueryIndexDescriptor idx) {
+        assert tblName != null;
+        assert idx != null;
+        assert idx.fields() != null;
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+
+        idxName = idx.name();
+        type = idx.type();
+        fields = new ArrayList(idx.fields());
+
+        fieldsAsc = new ArrayList<>(fields.size());
+
+        for (int i = 0; i < fields.size(); ++i)
+            fieldsAsc.add(!idx.descending(fields.get(i)));
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @return Index name.
+     */
+    public String indexName() {
+        return idxName;
+    }
+
+    /**
+     * @return Index type.
+     */
+    public QueryIndexType type() {
+        return type;
+    }
+
+    /**
+     * @return Index fields
+     */
+    public List<String> fields() {
+        return fields;
+    }
+
+    /**
+     * @return Index fields is ascending.
+     */
+    public List<Boolean> fieldsAsc() {
+        return fieldsAsc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+        writer.writeString(idxName);
+        writer.writeByte((byte)type.ordinal());
+
+        JdbcUtils.writeStringCollection(writer, fields);
+
+        if (fieldsAsc == null)
+            writer.writeInt(0);
+        else {
+            writer.writeInt(fieldsAsc.size());
+
+            for (Boolean b : fieldsAsc)
+                writer.writeBoolean(b.booleanValue());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        schemaName = reader.readString();
+        tblName = reader.readString();
+        idxName = reader.readString();
+        type = QueryIndexType.fromOrdinal(reader.readByte());
+
+        fields = JdbcUtils.readStringList(reader);
+
+        int size = reader.readInt();
+
+        if (size > 0) {
+            fieldsAsc = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i)
+                fieldsAsc .add(reader.readBoolean());
+        }
+        else
+            fieldsAsc = Collections.emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        JdbcIndexMeta meta = (JdbcIndexMeta)o;
+
+        return F.eq(schemaName, meta.schemaName) && F.eq(tblName, meta.tblName) && F.eq(idxName, meta.idxName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = schemaName != null ? schemaName.hashCode() : 0;
+
+        result = 31 * result + tblName.hashCode();
+        result = 31 * result + idxName.hashCode();
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcIndexMeta.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java
new file mode 100644
index 0000000..fca1bf7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsRequest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * JDBC get columns metadata request.
+ */
+public class JdbcMetaColumnsRequest extends JdbcRequest {
+    /** Schema name pattern. */
+    private String schemaName;
+
+    /** Table name pattern. */
+    private String tblName;
+
+    /** Column name pattern. */
+    private String colName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaColumnsRequest() {
+        super(META_COLUMNS);
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param colName Column name.
+     */
+    public JdbcMetaColumnsRequest(String schemaName, String tblName, String colName) {
+        super(META_COLUMNS);
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+        this.colName = colName;
+    }
+
+    /**
+     * @return Schema name pattern.
+     */
+    @Nullable public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name pattern.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @return Column name pattern.
+     */
+    public String columnName() {
+        return colName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+        writer.writeString(colName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemaName = reader.readString();
+        tblName = reader.readString();
+        colName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaColumnsRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
new file mode 100644
index 0000000..da270de
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResult.java
@@ -0,0 +1,99 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC columns metadata result.
+ */
+public class JdbcMetaColumnsResult extends JdbcResult {
+    /** Columns metadata. */
+    private List<JdbcColumnMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaColumnsResult() {
+        super(META_COLUMNS);
+    }
+
+    /**
+     * @param meta Columns metadata.
+     */
+    JdbcMetaColumnsResult(Collection<JdbcColumnMeta> meta) {
+        super(META_COLUMNS);
+
+        this.meta = new ArrayList<>(meta);
+    }
+
+    /**
+     * @return Columns metadata.
+     */
+    public List<JdbcColumnMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcColumnMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcColumnMeta m = new JdbcColumnMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaColumnsResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java
new file mode 100644
index 0000000..d4a53d8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesRequest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * JDBC indexes metadata request.
+ */
+public class JdbcMetaIndexesRequest extends JdbcRequest {
+    /** Schema name pattern. */
+    private String schemaName;
+
+    /** Table name pattern. */
+    private String tblName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaIndexesRequest() {
+        super(META_INDEXES);
+    }
+
+    /**
+     * @param schemaName Cache name.
+     * @param tblName Table name.
+     */
+    public JdbcMetaIndexesRequest(String schemaName, String tblName) {
+        super(META_INDEXES);
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    @Nullable public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemaName = reader.readString();
+        tblName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaIndexesRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java
new file mode 100644
index 0000000..2316dfc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaIndexesResult.java
@@ -0,0 +1,98 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC indexes metadata result.
+ */
+public class JdbcMetaIndexesResult extends JdbcResult {
+    /** Indexes metadata. */
+    private List<JdbcIndexMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaIndexesResult() {
+        super(META_INDEXES);
+    }
+
+    /**
+     * @param meta Indexes metadata.
+     */
+    JdbcMetaIndexesResult(Collection<JdbcIndexMeta> meta) {
+        super(META_INDEXES);
+        this.meta = new ArrayList<>(meta);
+    }
+
+    /**
+     * @return Indexes metadata.
+     */
+    public List<JdbcIndexMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcIndexMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcIndexMeta m = new JdbcIndexMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaIndexesResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java
new file mode 100644
index 0000000..6b955f9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsRequest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC SQL query parameters metadata request.
+ */
+public class JdbcMetaParamsRequest extends JdbcRequest {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Query. */
+    private String sql;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaParamsRequest() {
+        super(META_PARAMS);
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param sql SQL Query.
+     */
+    public JdbcMetaParamsRequest(String schemaName, String sql) {
+        super(META_PARAMS);
+
+        this.schemaName = schemaName;
+        this.sql = sql;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return SQL Query.
+     */
+    public String sql() {
+        return sql;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(sql);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemaName = reader.readString();
+        sql = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaParamsRequest.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java
new file mode 100644
index 0000000..7563e01
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaParamsResult.java
@@ -0,0 +1,97 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC SQL query parameters metadata result.
+ */
+public class JdbcMetaParamsResult extends JdbcResult {
+    /** Parameters meta results. */
+    private List<JdbcParameterMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaParamsResult() {
+        super(META_PARAMS);
+    }
+
+    /**
+     * @param meta Column metadata.
+     */
+    JdbcMetaParamsResult(List<JdbcParameterMeta> meta) {
+        super(META_PARAMS);
+        this.meta = meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcParameterMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcParameterMeta m = new JdbcParameterMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /**
+     * @return SQL query parameters metadata.
+     */
+    public List<JdbcParameterMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaParamsResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java
new file mode 100644
index 0000000..957225a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysRequest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * JDBC get primary keys metadata request.
+ */
+public class JdbcMetaPrimaryKeysRequest extends JdbcRequest {
+    /** Schema name pattern. */
+    private String schemaName;
+
+    /** Table name pattern. */
+    private String tblName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaPrimaryKeysRequest() {
+        super(META_PRIMARY_KEYS);
+    }
+
+    /**
+     * @param schemaName Cache name.
+     * @param tblName Table name.
+     */
+    public JdbcMetaPrimaryKeysRequest(String schemaName, String tblName) {
+        super(META_PRIMARY_KEYS);
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema name pattern.
+     */
+    @Nullable public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name pattern.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemaName = reader.readString();
+        tblName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaPrimaryKeysRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java
new file mode 100644
index 0000000..bd0dd90
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaPrimaryKeysResult.java
@@ -0,0 +1,99 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC primary keys metadata result.
+ */
+public class JdbcMetaPrimaryKeysResult extends JdbcResult {
+    /** Query result rows. */
+    private List<JdbcPrimaryKeyMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaPrimaryKeysResult() {
+        super(META_PRIMARY_KEYS);
+    }
+
+    /**
+     * @param meta Column metadata.
+     */
+    JdbcMetaPrimaryKeysResult(Collection<JdbcPrimaryKeyMeta> meta) {
+        super(META_PRIMARY_KEYS);
+
+        this.meta = new ArrayList<>(meta);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcPrimaryKeyMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcPrimaryKeyMeta m = new JdbcPrimaryKeyMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /**
+     * @return Primary keys metadata.
+     */
+    public List<JdbcPrimaryKeyMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaPrimaryKeysResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java
new file mode 100644
index 0000000..43bbe5d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasRequest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC tables metadata request.
+ */
+public class JdbcMetaSchemasRequest extends JdbcRequest {
+    /** Schema search pattern. */
+    private String schemaName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaSchemasRequest() {
+        super(META_SCHEMAS);
+    }
+
+    /**
+     * @param schemaName Schema search pattern.
+     */
+    public JdbcMetaSchemasRequest(String schemaName) {
+        super(META_SCHEMAS);
+
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return Schema search pattern.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        this.schemaName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaSchemasRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java
new file mode 100644
index 0000000..48b6aae
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaSchemasResult.java
@@ -0,0 +1,73 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.Collection;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC tables metadata result.
+ */
+public class JdbcMetaSchemasResult extends JdbcResult {
+    /** Found schemas. */
+    private Collection<String> schemas;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaSchemasResult() {
+        super(META_SCHEMAS);
+    }
+
+    /**
+     * @param schemas Found schemas.
+     */
+    JdbcMetaSchemasResult(Collection<String> schemas) {
+        super(META_SCHEMAS);
+        this.schemas = schemas;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        JdbcUtils.writeStringCollection(writer, schemas);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        schemas = JdbcUtils.readStringList(reader);
+    }
+
+    /**
+     * @return Found schemas.
+     */
+    public Collection<String> schemas() {
+        return schemas;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaSchemasResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java
new file mode 100644
index 0000000..740b656
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesRequest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC tables metadata request.
+ */
+public class JdbcMetaTablesRequest extends JdbcRequest {
+    /** Schema search pattern. */
+    private String schemaName;
+
+    /** Table search pattern. */
+    private String tblName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaTablesRequest() {
+        super(META_TABLES);
+    }
+
+    /**
+     * @param schemaName Schema search pattern.
+     * @param tblName Table search pattern.
+     */
+    public JdbcMetaTablesRequest(String schemaName, String tblName) {
+        super(META_TABLES);
+
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema search pattern.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table search pattern.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        this.schemaName = reader.readString();
+        this.tblName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaTablesRequest.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java
new file mode 100644
index 0000000..585667e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaTablesResult.java
@@ -0,0 +1,97 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC tables metadata result.
+ */
+public class JdbcMetaTablesResult extends JdbcResult {
+    /** Tables metadata. */
+    private List<JdbcTableMeta> meta;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaTablesResult() {
+        super(META_TABLES);
+    }
+
+    /**
+     * @param meta Tables metadata.
+     */
+    JdbcMetaTablesResult(List<JdbcTableMeta> meta) {
+        super(META_TABLES);
+        this.meta = meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        super.writeBinary(writer);
+
+        if (F.isEmpty(meta))
+            writer.writeInt(0);
+        else {
+            writer.writeInt(meta.size());
+
+            for(JdbcTableMeta m : meta)
+                m.writeBinary(writer);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        super.readBinary(reader);
+
+        int size = reader.readInt();
+
+        if (size == 0)
+            meta = Collections.emptyList();
+        else {
+            meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i) {
+                JdbcTableMeta m = new JdbcTableMeta();
+
+                m.readBinary(reader);
+
+                meta.add(m);
+            }
+        }
+    }
+
+    /**
+     * @return Tables metadata.
+     */
+    public List<JdbcTableMeta> meta() {
+        return meta;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaTablesResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
new file mode 100644
index 0000000..dd3b18b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
@@ -0,0 +1,165 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.sql.ParameterMetaData;
+import java.sql.SQLException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC SQL query parameter metadata.
+ *
+ * {@see java.sql.ParameterMetaData}.
+ */
+public class JdbcParameterMeta implements JdbcRawBinarylizable {
+    /** Null value is allow for the param. */
+    private int isNullable;
+
+    /** Signed flag. */
+    private boolean signed;
+
+    /** Precision. */
+    private int precision;
+
+    /** Scale. */
+    private int scale;
+
+    /** SQL type ID. */
+    private int type;
+
+    /** SQL type name. */
+    private String typeName;
+
+    /** Java type class name. */
+    private String typeClass;
+
+    /** Mode. */
+    private int mode;
+
+
+    /**
+     * Default constructor is used for binary serialization.
+     */
+    public JdbcParameterMeta() {
+        // No-op.
+    }
+
+    /**
+     * @param meta Param metadata.
+     * @param order Param order.
+     * @throws SQLException On errror.
+     */
+    public JdbcParameterMeta(ParameterMetaData meta, int order) throws SQLException {
+        isNullable = meta.isNullable(order);
+        signed = meta.isSigned(order);
+        precision = meta.getPrecision(order);
+        scale = meta.getScale(order);
+        type = meta.getParameterType(order);
+        typeName = meta.getParameterTypeName(order);
+        typeClass = meta.getParameterClassName(order);
+        mode = meta.getParameterMode(order);
+    }
+
+    /**
+     * @return Nullable mode.
+     */
+    public int isNullable() {
+        return isNullable;
+    }
+
+    /**
+     * @return Signed flag.
+     */
+    public boolean isSigned() {
+        return signed;
+    }
+
+    /**
+     * @return Precision.
+     */
+    public int precision() {
+        return precision;
+    }
+
+    /**
+     * @return Scale.
+     */
+    public int scale() {
+        return scale;
+    }
+
+    /**
+     * @return SQL type.
+     */
+    public int type() {
+        return type;
+    }
+
+    /**
+     * @return SQL type name.
+     */
+    public String typeName() {
+        return typeName;
+    }
+
+    /**
+     * @return Java type class name.
+     */
+    public String typeClass() {
+        return typeClass;
+    }
+
+    /**
+     * @return Mode.
+     */
+    public int mode() {
+        return mode;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        writer.writeInt(isNullable);
+        writer.writeBoolean(signed);
+        writer.writeInt(precision);
+        writer.writeInt(scale);
+        writer.writeInt(type);
+        writer.writeString(typeName);
+        writer.writeString(typeClass);
+        writer.writeInt(mode);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        isNullable = reader.readInt();
+        signed = reader.readBoolean();
+        precision = reader.readInt();
+        scale = reader.readInt();
+        type = reader.readInt();
+        typeName = reader.readString();
+        typeClass = reader.readString();
+        mode = reader.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcParameterMeta.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java
new file mode 100644
index 0000000..6b9bf70
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcPrimaryKeyMeta.java
@@ -0,0 +1,131 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.List;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * JDBC primary key metadata.
+ */
+public class JdbcPrimaryKeyMeta implements JdbcRawBinarylizable {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /** Primary key name. */
+    private String name;
+
+    /** Primary key fields. */
+    private List<String> fields;
+
+    /**
+     * Default constructor is used for binary serialization.
+     */
+    JdbcPrimaryKeyMeta() {
+        // No-op.
+    }
+
+    /**
+     * @param schemaName Schema.
+     * @param tblName Table.
+     * @param name Name.
+     * @param fields Primary key fields.
+     */
+    JdbcPrimaryKeyMeta(String schemaName, String tblName, String name, List<String> fields) {
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+        this.name = name;
+        this.fields = fields;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @return Primary key name.
+     */
+    public String name() {
+        return name;
+    }
+
+    /**
+     * @return Key fields.
+     */
+    public List<String> fields() {
+        return fields;
+    }
+
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+        writer.writeString(name);
+
+        JdbcUtils.writeStringCollection(writer, fields);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        schemaName = reader.readString();
+        tblName = reader.readString();
+        name = reader.readString();
+
+        fields = JdbcUtils.readStringList(reader);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        JdbcPrimaryKeyMeta meta = (JdbcPrimaryKeyMeta)o;
+
+        return F.eq(schemaName, meta.schemaName) && F.eq(tblName, meta.tblName) && F.eq(name, meta.name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = schemaName != null ? schemaName.hashCode() : 0;
+
+        result = 31 * result + tblName.hashCode();
+        result = 31 * result + name.hashCode();
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java
index 411d1e0..872889c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryCloseRequest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query close request.
+ * JDBC query close request.
  */
 public class JdbcQueryCloseRequest extends JdbcRequest {
     /** Query ID. */
@@ -31,7 +31,7 @@ public class JdbcQueryCloseRequest extends JdbcRequest {
 
     /**
      */
-    public JdbcQueryCloseRequest() {
+    JdbcQueryCloseRequest() {
         super(QRY_CLOSE);
     }
 


[10/19] ignite git commit: IGNITE-6098 Fixed IgniteDataIntegrityTests.testExpandBuffer - Fixes #2465.

Posted by ag...@apache.org.
IGNITE-6098 Fixed IgniteDataIntegrityTests.testExpandBuffer - Fixes #2465.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-5901
Commit: 071c24c415a222186cd7edf3ce3018de6bafdb24
Parents: 69e6f8b
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Thu Aug 17 17:05:50 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 17:05:50 2017 +0300

----------------------------------------------------------------------
 .../db/wal/crc/IgniteDataIntegrityTests.java    | 24 ++++++++++++++++----
 1 file changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/071c24c4/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
index 270c560..e4874d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/crc/IgniteDataIntegrityTests.java
@@ -122,7 +122,7 @@ public class IgniteDataIntegrityTests extends TestCase {
      *
      */
     public void testExpandBuffer() {
-        ByteBufferExpander expBuf = new ByteBufferExpander(16, ByteOrder.nativeOrder());
+        ByteBufferExpander expBuf = new ByteBufferExpander(24, ByteOrder.nativeOrder());
 
         ByteBuffer b1 = expBuf.buffer();
 
@@ -131,21 +131,32 @@ public class IgniteDataIntegrityTests extends TestCase {
         b1.putLong(3L);
 
         assertEquals(13, b1.position());
-        assertEquals(16, b1.limit());
+        assertEquals(24, b1.limit());
 
         ByteBuffer b2 = expBuf.expand(32);
 
+        assertEquals(13, b2.position());
+        assertEquals(24, b2.limit());
+
+        b2.rewind();
+
         assertEquals(0, b2.position());
         assertEquals((byte)1, b2.get());
         assertEquals(2, b2.getInt());
         assertEquals(3L, b2.getLong());
         assertEquals(13, b2.position());
-        assertEquals(32, b2.limit());
+        assertEquals(24, b2.limit());
+        assertEquals(32, b2.capacity());
+
+        b2.limit(b2.capacity());
 
         b2.putInt(4);
+        b2.putInt(5);
+        b2.putInt(6);
 
-        assertEquals(17, b2.position());
+        assertEquals(25, b2.position());
         assertEquals(32, b2.limit());
+        assertEquals(32, b2.capacity());
 
         b2.flip();
 
@@ -154,7 +165,10 @@ public class IgniteDataIntegrityTests extends TestCase {
         assertEquals(2, b2.getInt());
         assertEquals(3L, b2.getLong());
         assertEquals(4, b2.getInt());
-        assertEquals(17, b2.limit());
+        assertEquals(5, b2.getInt());
+        assertEquals(6, b2.getInt());
+        assertEquals(25, b2.limit());
+        assertEquals(32, b2.capacity());
     }
 
     /**


[13/19] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ag...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-5901
Commit: c47c365acb57dd586af0409bd925d3858fc61b54
Parents: 4402a1c 06ad010
Author: devozerov <vo...@gridgain.com>
Authored: Thu Aug 17 17:56:25 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 17 17:56:25 2017 +0300

----------------------------------------------------------------------
 .../configuration/CheckpointWriteOrder.java     | 33 ++++++++
 .../PersistentStoreConfiguration.java           | 26 +++++++
 .../GridCacheDatabaseSharedManager.java         | 82 +++++++++++++++-----
 .../persistence/pagemem/PageMemoryImpl.java     | 71 +++++++++--------
 ...nitePersistenceSequentialCheckpointTest.java | 44 +++++++++++
 .../IgnitePersistentStoreCacheGroupsTest.java   | 31 ++++----
 .../db/wal/crc/IgniteDataIntegrityTests.java    | 24 ++++--
 7 files changed, 239 insertions(+), 72 deletions(-)
----------------------------------------------------------------------



[08/19] ignite git commit: IGNITE-5837: Minor fix to DynamicIndexAbstractConcurrentSelfTest. This closes #2345.

Posted by ag...@apache.org.
IGNITE-5837: Minor fix to DynamicIndexAbstractConcurrentSelfTest. This closes #2345.


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

Branch: refs/heads/ignite-5901
Commit: b417a36c88625bc5dd1df1ad719a52f79e5f1d74
Parents: 7781823
Author: devozerov <vo...@gridgain.com>
Authored: Thu Aug 17 15:39:27 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 17 15:39:27 2017 +0300

----------------------------------------------------------------------
 .../DynamicIndexAbstractConcurrentSelfTest.java | 115 ++++++++++---------
 1 file changed, 63 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b417a36c/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
index 00fd413..7b53f73 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicIndexAbstractConcurrentSelfTest.java
@@ -66,7 +66,8 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
     private static final int LARGE_CACHE_SIZE = 100_000;
 
     /** Latches to block certain index operations. */
-    private static final ConcurrentHashMap<UUID, T3<CountDownLatch, AtomicBoolean, CountDownLatch>> BLOCKS = new ConcurrentHashMap<>();
+    private static final ConcurrentHashMap<UUID, T3<CountDownLatch, AtomicBoolean, CountDownLatch>> BLOCKS =
+        new ConcurrentHashMap<>();
 
     /** Cache mode. */
     private final CacheMode cacheMode;
@@ -88,8 +89,6 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
-
-        GridQueryProcessor.idxCls = BlockingIndexing.class;
     }
 
     /** {@inheritDoc} */
@@ -139,16 +138,16 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     public void testCoordinatorChange() throws Exception {
         // Start servers.
-        Ignite srv1 = Ignition.start(serverConfiguration(1));
-        Ignite srv2 = Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
-        Ignition.start(serverConfiguration(4));
+        Ignite srv1 = ignitionStart(serverConfiguration(1));
+        Ignite srv2 = ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
+        ignitionStart(serverConfiguration(4));
 
         UUID srv1Id = srv1.cluster().localNode().id();
         UUID srv2Id = srv2.cluster().localNode().id();
 
         // Start client which will execute operations.
-        Ignite cli = Ignition.start(clientConfiguration(5));
+        Ignite cli = ignitionStart(clientConfiguration(5));
 
         createSqlCache(cli);
 
@@ -203,11 +202,11 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      * @throws Exception If failed.
      */
     public void testOperationChaining() throws Exception {
-        Ignite srv1 = Ignition.start(serverConfiguration(1));
+        Ignite srv1 = ignitionStart(serverConfiguration(1));
 
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
-        Ignition.start(clientConfiguration(4));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
+        ignitionStart(clientConfiguration(4));
 
         createSqlCache(srv1);
 
@@ -223,9 +222,9 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
             queryProcessor(srv1).dynamicIndexCreate(CACHE_NAME, CACHE_NAME, TBL_NAME, idx2, false);
 
         // Start even more nodes of different flavors
-        Ignition.start(serverConfiguration(5));
-        Ignition.start(serverConfiguration(6, true));
-        Ignition.start(clientConfiguration(7));
+        ignitionStart(serverConfiguration(5));
+        ignitionStart(serverConfiguration(6, true));
+        ignitionStart(clientConfiguration(7));
 
         assert !idxFut1.isDone();
         assert !idxFut2.isDone();
@@ -255,7 +254,7 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      * @throws Exception If failed.
      */
     public void testNodeJoinOnPendingOperation() throws Exception {
-        Ignite srv1 = Ignition.start(serverConfiguration(1));
+        Ignite srv1 = ignitionStart(serverConfiguration(1));
 
         createSqlCache(srv1);
 
@@ -266,9 +265,9 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
         IgniteInternalFuture<?> idxFut =
             queryProcessor(srv1).dynamicIndexCreate(CACHE_NAME, CACHE_NAME, TBL_NAME, idx, false);
 
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
-        Ignition.start(clientConfiguration(4));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
+        ignitionStart(clientConfiguration(4));
 
         assert !idxFut.isDone();
 
@@ -293,10 +292,10 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     public void testConcurrentPutRemove() throws Exception {
         // Start several nodes.
-        Ignite srv1 = Ignition.start(serverConfiguration(1));
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3));
-        Ignition.start(serverConfiguration(4));
+        Ignite srv1 = ignitionStart(serverConfiguration(1));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3));
+        ignitionStart(serverConfiguration(4));
 
         awaitPartitionMapExchange();
 
@@ -390,8 +389,8 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     public void testConcurrentRebalance() throws Exception {
         // Start cache and populate it with data.
-        Ignite srv1 = Ignition.start(serverConfiguration(1));
-        Ignite srv2 = Ignition.start(serverConfiguration(2));
+        Ignite srv1 = ignitionStart(serverConfiguration(1));
+        Ignite srv2 = ignitionStart(serverConfiguration(2));
 
         createSqlCache(srv1);
 
@@ -412,12 +411,12 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
         idxLatch2.countDown();
 
         // Start two more nodes and unblock index operation in the middle.
-        Ignition.start(serverConfiguration(3));
+        ignitionStart(serverConfiguration(3));
 
         unblockIndexing(srv1);
         unblockIndexing(srv2);
 
-        Ignition.start(serverConfiguration(4));
+        ignitionStart(serverConfiguration(4));
 
         awaitPartitionMapExchange();
 
@@ -437,12 +436,12 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     public void testConcurrentCacheDestroy() throws Exception {
         // Start complex topology.
-        Ignite srv1 = Ignition.start(serverConfiguration(1));
+        Ignite srv1 = ignitionStart(serverConfiguration(1));
 
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
 
-        Ignite cli = Ignition.start(clientConfiguration(4));
+        Ignite cli = ignitionStart(clientConfiguration(4));
 
         // Start cache and populate it with data.
         createSqlCache(cli);
@@ -482,11 +481,11 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     public void testConcurrentOperationsMultithreaded() throws Exception {
         // Start complex topology.
-        Ignition.start(serverConfiguration(1));
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
+        ignitionStart(serverConfiguration(1));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
 
-        Ignite cli = Ignition.start(clientConfiguration(4));
+        Ignite cli = ignitionStart(clientConfiguration(4));
 
         createSqlCache(cli);
 
@@ -556,11 +555,11 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     public void testQueryConsistencyMultithreaded() throws Exception {
         // Start complex topology.
-        Ignition.start(serverConfiguration(1));
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
+        ignitionStart(serverConfiguration(1));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
 
-        Ignite cli = Ignition.start(clientConfiguration(4));
+        Ignite cli = ignitionStart(clientConfiguration(4));
 
         createSqlCache(cli);
 
@@ -654,11 +653,11 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     private void checkClientReconnect(final boolean restartCache) throws Exception {
         // Start complex topology.
-        final Ignite srv = Ignition.start(serverConfiguration(1));
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
+        final Ignite srv = ignitionStart(serverConfiguration(1));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
 
-        final Ignite cli = Ignition.start(clientConfiguration(4));
+        final Ignite cli = ignitionStart(clientConfiguration(4));
 
         createSqlCache(cli);
 
@@ -752,11 +751,11 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     public void testConcurrentOperationsAndNodeStartStopMultithreaded() throws Exception {
         // Start several stable nodes.
-        Ignition.start(serverConfiguration(1));
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
+        ignitionStart(serverConfiguration(1));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
 
-        final Ignite cli = Ignition.start(clientConfiguration(4));
+        final Ignite cli = ignitionStart(clientConfiguration(4));
 
         createSqlCache(cli);
 
@@ -798,7 +797,7 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
                                 cfg = clientConfiguration(lastIdx);
                         }
 
-                        Ignition.start(cfg);
+                        ignitionStart(cfg);
 
                         exists = true;
                     }
@@ -877,11 +876,11 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
      */
     public void testConcurrentOperationsAndCacheStartStopMultithreaded() throws Exception {
         // Start complex topology.
-        Ignition.start(serverConfiguration(1));
-        Ignition.start(serverConfiguration(2));
-        Ignition.start(serverConfiguration(3, true));
+        ignitionStart(serverConfiguration(1));
+        ignitionStart(serverConfiguration(2));
+        ignitionStart(serverConfiguration(3, true));
 
-        Ignite cli = Ignition.start(clientConfiguration(4));
+        Ignite cli = ignitionStart(clientConfiguration(4));
 
         final AtomicBoolean stopped = new AtomicBoolean();
 
@@ -1091,4 +1090,16 @@ public abstract class DynamicIndexAbstractConcurrentSelfTest extends DynamicInde
     private IgniteCache<?, ?> createSqlCache(Ignite node) throws IgniteCheckedException {
         return createSqlCache(node, cacheConfiguration());
     }
+
+    /**
+     * Start a node.
+     * 
+     * @param cfg Configuration.
+     * @return Ignite instance.
+     */
+    private static Ignite ignitionStart(IgniteConfiguration cfg) {
+        GridQueryProcessor.idxCls = BlockingIndexing.class;
+
+        return Ignition.start(cfg);
+    }
 }


[05/19] ignite git commit: IGNITE-5233: JDBC thind driver: implemented metadata methods. This closes #2079.

Posted by ag...@apache.org.
IGNITE-5233: JDBC thind driver: implemented metadata methods. This closes #2079.


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

Branch: refs/heads/ignite-5901
Commit: 0e8031444b2f0d68fd3fb5a9ba03ca4d6a0c4e2d
Parents: 37e58ba
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Aug 17 13:41:37 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 17 13:41:37 2017 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  337 +++-
 .../internal/jdbc/thin/JdbcThinConnection.java  |   20 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     | 1587 ++++++++++++++++++
 .../jdbc/thin/JdbcThinParameterMetadata.java    |  115 ++
 .../jdbc/thin/JdbcThinPreparedStatement.java    |   27 +-
 .../internal/jdbc/thin/JdbcThinResultSet.java   |   47 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   |    4 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |  117 +-
 .../processors/odbc/SqlListenerNioListener.java |    9 +-
 .../odbc/SqlListenerRequestHandler.java         |    9 +
 .../odbc/jdbc/JdbcBatchExecuteRequest.java      |   20 +-
 .../odbc/jdbc/JdbcBatchExecuteResult.java       |    6 +
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |   75 +-
 .../processors/odbc/jdbc/JdbcIndexMeta.java     |  192 +++
 .../odbc/jdbc/JdbcMetaColumnsRequest.java       |  102 ++
 .../odbc/jdbc/JdbcMetaColumnsResult.java        |   99 ++
 .../odbc/jdbc/JdbcMetaIndexesRequest.java       |   88 +
 .../odbc/jdbc/JdbcMetaIndexesResult.java        |   98 ++
 .../odbc/jdbc/JdbcMetaParamsRequest.java        |   87 +
 .../odbc/jdbc/JdbcMetaParamsResult.java         |   97 ++
 .../odbc/jdbc/JdbcMetaPrimaryKeysRequest.java   |   88 +
 .../odbc/jdbc/JdbcMetaPrimaryKeysResult.java    |   99 ++
 .../odbc/jdbc/JdbcMetaSchemasRequest.java       |   73 +
 .../odbc/jdbc/JdbcMetaSchemasResult.java        |   73 +
 .../odbc/jdbc/JdbcMetaTablesRequest.java        |   87 +
 .../odbc/jdbc/JdbcMetaTablesResult.java         |   97 ++
 .../processors/odbc/jdbc/JdbcParameterMeta.java |  165 ++
 .../odbc/jdbc/JdbcPrimaryKeyMeta.java           |  131 ++
 .../odbc/jdbc/JdbcQueryCloseRequest.java        |    4 +-
 .../odbc/jdbc/JdbcQueryExecuteRequest.java      |    8 +-
 .../odbc/jdbc/JdbcQueryExecuteResult.java       |   12 +-
 .../odbc/jdbc/JdbcQueryFetchRequest.java        |    4 +-
 .../odbc/jdbc/JdbcQueryFetchResult.java         |   12 +-
 .../odbc/jdbc/JdbcQueryMetadataRequest.java     |   18 +-
 .../odbc/jdbc/JdbcQueryMetadataResult.java      |   14 +-
 .../processors/odbc/jdbc/JdbcRequest.java       |   67 +-
 .../odbc/jdbc/JdbcRequestHandler.java           |  273 ++-
 .../processors/odbc/jdbc/JdbcResult.java        |   58 +-
 .../processors/odbc/jdbc/JdbcTableMeta.java     |   82 +
 .../processors/odbc/jdbc/JdbcUtils.java         |   37 +-
 .../odbc/odbc/OdbcRequestHandler.java           |    6 +
 .../processors/query/GridQueryProcessor.java    |    2 +-
 .../query/GridQueryTypeDescriptor.java          |    7 +
 .../query/QueryTypeDescriptorImpl.java          |   15 +
 .../internal/processors/query/QueryUtils.java   |    7 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   18 +-
 46 files changed, 4472 insertions(+), 121 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index 2dae107..01b2e8a 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -21,16 +21,27 @@ import java.io.Serializable;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.Statement;
+import java.sql.Types;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Set;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.affinity.AffinityKey;
-import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -66,15 +77,18 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     }
 
     /**
+     * @param qryEntity Query entity.
      * @return Cache configuration.
      */
-    protected CacheConfiguration cacheConfiguration() {
+    protected CacheConfiguration cacheConfiguration(QueryEntity qryEntity) {
         CacheConfiguration<?,?> cache = defaultCacheConfiguration();
 
         cache.setCacheMode(PARTITIONED);
         cache.setBackups(1);
         cache.setWriteSynchronizationMode(FULL_SYNC);
 
+        cache.setQueryEntities(Collections.singletonList(qryEntity));
+
         return cache;
     }
 
@@ -84,22 +98,49 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
 
         startGridsMultiThreaded(3);
 
-        IgniteCache<String, Organization> orgCache = jcache(grid(0), cacheConfiguration(), "org",
-            String.class, Organization.class);
+        IgniteCache<String, Organization> orgCache = jcache(grid(0),
+            cacheConfiguration(new QueryEntity(String.class.getName(), Organization.class.getName())
+                .addQueryField("id", Integer.class.getName(), null)
+                .addQueryField("name", String.class.getName(), null)
+                .setIndexes(Arrays.asList(
+                    new QueryIndex("id"),
+                    new QueryIndex("name", false, "org_name_index")
+                ))), "org");
 
         assert orgCache != null;
 
         orgCache.put("o1", new Organization(1, "A"));
         orgCache.put("o2", new Organization(2, "B"));
 
-        IgniteCache<AffinityKey, Person> personCache = jcache(grid(0), cacheConfiguration(), "pers",
-            AffinityKey.class, Person.class);
+        LinkedHashMap<String, Boolean> persFields = new LinkedHashMap<>();
+
+        persFields.put("name", true);
+        persFields.put("age", false);
+
+        IgniteCache<AffinityKey, Person> personCache = jcache(grid(0), cacheConfiguration(
+            new QueryEntity(AffinityKey.class.getName(), Person.class.getName())
+                .addQueryField("name", String.class.getName(), null)
+                .addQueryField("age", Integer.class.getName(), null)
+                .addQueryField("orgId", Integer.class.getName(), null)
+                .setIndexes(Arrays.asList(
+                    new QueryIndex("orgId"),
+                    new QueryIndex().setFields(persFields)))
+            ), "pers");
 
         assert personCache != null;
 
         personCache.put(new AffinityKey<>("p1", "o1"), new Person("John White", 25, 1));
         personCache.put(new AffinityKey<>("p2", "o1"), new Person("Joe Black", 35, 1));
         personCache.put(new AffinityKey<>("p3", "o2"), new Person("Mike Green", 40, 2));
+
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            Statement stmt = conn.createStatement();
+
+            stmt.execute("CREATE TABLE TEST (ID INT, NAME VARCHAR(50), VAL VARCHAR(50), PRIMARY KEY (ID, NAME))");
+            stmt.execute("CREATE TABLE \"Quoted\" (\"Id\" INT primary key, \"Name\" VARCHAR(50))");
+            stmt.execute("CREATE INDEX \"MyTestIndex quoted\" on \"Quoted\" (\"Id\" DESC)");
+            stmt.execute("CREATE INDEX IDX ON TEST (ID ASC)");
+        }
     }
 
     /** {@inheritDoc} */
@@ -147,8 +188,6 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testGetTables() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-5233");
-
         try (Connection conn = DriverManager.getConnection(URL)) {
             DatabaseMetaData meta = conn.getMetaData();
 
@@ -184,15 +223,43 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
+    public void testGetAllTables() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            DatabaseMetaData meta = conn.getMetaData();
+
+            ResultSet rs = meta.getTables(null, null, null, null);
+
+            Set<String> expectedTbls = new HashSet<>(Arrays.asList(
+                "org.ORGANIZATION",
+                "pers.PERSON",
+                "PUBLIC.TEST",
+                "PUBLIC.Quoted"));
+
+            Set<String> actualTbls = new HashSet<>(expectedTbls.size());
+
+            while(rs.next()) {
+                actualTbls.add(rs.getString("TABLE_SCHEM") + '.'
+                    + rs.getString("TABLE_NAME"));
+            }
+
+            assert expectedTbls.equals(actualTbls) : "expectedTbls=" + expectedTbls +
+                ", actualTbls" + actualTbls;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetColumns() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-5233");
+        final boolean primitivesInformationIsLostAfterStore = ignite(0).configuration().getMarshaller()
+            instanceof BinaryMarshaller;
 
         try (Connection conn = DriverManager.getConnection(URL)) {
             conn.setSchema("pers");
 
             DatabaseMetaData meta = conn.getMetaData();
 
-            ResultSet rs = meta.getColumns("", "pers", "Person", "%");
+            ResultSet rs = meta.getColumns("", "pers", "PERSON", "%");
 
             assert rs != null;
 
@@ -216,7 +283,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 } else if ("AGE".equals(name) || "ORGID".equals(name)) {
                     assert rs.getInt("DATA_TYPE") == INTEGER;
                     assert "INTEGER".equals(rs.getString("TYPE_NAME"));
-                    assert rs.getInt("NULLABLE") == 0;
+                    assertEquals(primitivesInformationIsLostAfterStore ? 1 : 0, rs.getInt("NULLABLE"));
                 }
                 if ("_KEY".equals(name)) {
                     assert rs.getInt("DATA_TYPE") == OTHER;
@@ -235,7 +302,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
             assert names.isEmpty();
             assert cnt == 3;
 
-            rs = meta.getColumns("", "org", "Organization", "%");
+            rs = meta.getColumns("", "org", "ORGANIZATION", "%");
 
             assert rs != null;
 
@@ -280,22 +347,243 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
-    public void testMetadataResultSetClose() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-5233");
+    public void testGetAllColumns() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            DatabaseMetaData meta = conn.getMetaData();
+
+            ResultSet rs = meta.getColumns(null, null, null, null);
+
+            Set<String> expectedCols = new HashSet<>(Arrays.asList(
+                "org.ORGANIZATION.ID",
+                "org.ORGANIZATION.NAME",
+                "pers.PERSON.ORGID",
+                "pers.PERSON.AGE",
+                "pers.PERSON.NAME",
+                "PUBLIC.TEST.ID",
+                "PUBLIC.TEST.NAME",
+                "PUBLIC.TEST.VAL",
+                "PUBLIC.Quoted.Id",
+                "PUBLIC.Quoted.Name"));
+
+            Set<String> actualCols = new HashSet<>(expectedCols.size());
+
+            while(rs.next()) {
+                actualCols.add(rs.getString("TABLE_SCHEM") + '.'
+                    + rs.getString("TABLE_NAME") + "."
+                    + rs.getString("COLUMN_NAME"));
+            }
+
+            assert expectedCols.equals(actualCols) : "expectedCols=" + expectedCols +
+                ", actualCols" + actualCols;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvalidCatalog() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            DatabaseMetaData meta = conn.getMetaData();
+
+            ResultSet rs = meta.getSchemas("q", null);
+
+            assert !rs.next() : "Results must be empty";
+
+            rs = meta.getTables("q", null, null, null);
+
+            assert !rs.next() : "Results must be empty";
+
+            rs = meta.getColumns("q", null, null, null);
+
+            assert !rs.next() : "Results must be empty";
+
+            rs = meta.getIndexInfo("q", null, null, false, false);
+
+            assert !rs.next() : "Results must be empty";
+
+            rs = meta.getPrimaryKeys("q", null, null);
+
+            assert !rs.next() : "Results must be empty";
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIndexMetadata() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL);
+             ResultSet rs = conn.getMetaData().getIndexInfo(null, "pers", "PERSON", false, false)) {
+
+            int cnt = 0;
+
+            while (rs.next()) {
+                String idxName = rs.getString("INDEX_NAME");
+                String field = rs.getString("COLUMN_NAME");
+                String ascOrDesc = rs.getString("ASC_OR_DESC");
+
+                assert rs.getShort("TYPE") == DatabaseMetaData.tableIndexOther;
+
+                if ("PERSON_ORGID_ASC_IDX".equals(idxName)) {
+                    assert "ORGID".equals(field);
+                    assert "A".equals(ascOrDesc);
+                }
+                else if ("PERSON_NAME_ASC_AGE_DESC_IDX".equals(idxName)) {
+                    if ("NAME".equals(field))
+                        assert "A".equals(ascOrDesc);
+                    else if ("AGE".equals(field))
+                        assert "D".equals(ascOrDesc);
+                    else
+                        fail("Unexpected field: " + field);
+                }
+                else
+                    fail("Unexpected index: " + idxName);
+
+                cnt++;
+            }
+
+            assert cnt == 3;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAllIndexes() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            ResultSet rs = conn.getMetaData().getIndexInfo(null, null, null, false, false);
+
+            Set<String> expectedIdxs = new HashSet<>(Arrays.asList(
+                "org.ORGANIZATION.ORGANIZATION_ID_ASC_IDX",
+                "org.ORGANIZATION.ORG_NAME_INDEX",
+                "pers.PERSON.PERSON_ORGID_ASC_IDX",
+                "pers.PERSON.PERSON_NAME_ASC_AGE_DESC_IDX",
+                "PUBLIC.TEST.IDX",
+                "PUBLIC.Quoted.MyTestIndex quoted"));
+
+            Set<String> actualIdxs = new HashSet<>(expectedIdxs.size());
+
+            while(rs.next()) {
+                actualIdxs.add(rs.getString("TABLE_SCHEM") +
+                    '.' + rs.getString("TABLE_NAME") +
+                    '.' + rs.getString("INDEX_NAME"));
+            }
+
+            assert expectedIdxs.equals(actualIdxs) : "expectedIdxs=" + expectedIdxs +
+                ", actualIdxs" + actualIdxs;
+        }
+    }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimaryKeyMetadata() throws Exception {
         try (Connection conn = DriverManager.getConnection(URL);
-             ResultSet tbls = conn.getMetaData().getTables(null, null, "%", null)) {
-            int colCnt = tbls.getMetaData().getColumnCount();
+             ResultSet rs = conn.getMetaData().getPrimaryKeys(null, "pers", "PERSON")) {
+
+            int cnt = 0;
+
+            while (rs.next()) {
+                assert "_KEY".equals(rs.getString("COLUMN_NAME"));
+
+                cnt++;
+            }
+
+            assert cnt == 1;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAllPrimaryKeys() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            ResultSet rs = conn.getMetaData().getPrimaryKeys(null, null, null);
+
+            Set<String> expectedPks = new HashSet<>(Arrays.asList(
+                "org.ORGANIZATION.PK_org_ORGANIZATION._KEY",
+                "pers.PERSON.PK_pers_PERSON._KEY",
+                "PUBLIC.TEST.PK_PUBLIC_TEST.ID",
+                "PUBLIC.TEST.PK_PUBLIC_TEST.NAME",
+                "PUBLIC.Quoted.PK_PUBLIC_Quoted.Id"));
+
+            Set<String> actualPks = new HashSet<>(expectedPks.size());
+
+            while(rs.next()) {
+                actualPks.add(rs.getString("TABLE_SCHEM") +
+                    '.' + rs.getString("TABLE_NAME") +
+                    '.' + rs.getString("PK_NAME") +
+                    '.' + rs.getString("COLUMN_NAME"));
+            }
+
+            assert expectedPks.equals(actualPks) : "expectedPks=" + expectedPks +
+                ", actualPks" + actualPks;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testParametersMetadata() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            conn.setSchema("pers");
+
+            PreparedStatement stmt = conn.prepareStatement("select orgId from Person p where p.name > ? and p.orgId > ?");
+
+            ParameterMetaData meta = stmt.getParameterMetaData();
+
+            assert meta != null;
+
+            assert meta.getParameterCount() == 2;
+
+            assert meta.getParameterType(1) == Types.VARCHAR;
+            assert meta.isNullable(1) == ParameterMetaData.parameterNullableUnknown;
+            assert meta.getPrecision(1) == Integer.MAX_VALUE;
+
+            assert meta.getParameterType(2) == Types.INTEGER;
+            assert meta.isNullable(2) == ParameterMetaData.parameterNullableUnknown;
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSchemasMetadata() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            ResultSet rs = conn.getMetaData().getSchemas();
+
+            Set<String> expectedSchemas = new HashSet<>(Arrays.asList("PUBLIC", "pers", "org"));
 
-            while (tbls.next()) {
-                for (int i = 0; i < colCnt; i++)
-                    tbls.getObject(i + 1);
+            Set<String> schemas = new HashSet<>();
+
+            while (rs.next()) {
+                schemas.add(rs.getString(1));
+
+                assert rs.getString(2) == null;
             }
+
+            assert expectedSchemas.equals(schemas) : "Unexpected schemas: " + schemas +
+                ". Expected schemas: " + expectedSchemas;
         }
-        catch (Exception e) {
-            log.error("Unexpected exception", e);
+    }
 
-            fail();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEmptySchemasMetadata() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            ResultSet rs = conn.getMetaData().getSchemas(null, "qqq");
+
+            assert !rs.next() : "Empty result set is expected";
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testVersions() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            assert conn.getMetaData().getDatabaseProductVersion().equals(IgniteVersionUtils.VER.toString());
+            assert conn.getMetaData().getDriverVersion().equals(IgniteVersionUtils.VER.toString());
         }
     }
 
@@ -305,15 +593,12 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     @SuppressWarnings("UnusedDeclaration")
     private static class Person implements Serializable {
         /** Name. */
-        @QuerySqlField(index = false)
         private final String name;
 
         /** Age. */
-        @QuerySqlField
         private final int age;
 
         /** Organization ID. */
-        @QuerySqlField
         private final int orgId;
 
         /**
@@ -338,11 +623,9 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
     @SuppressWarnings("UnusedDeclaration")
     private static class Organization implements Serializable {
         /** ID. */
-        @QuerySqlField
         private final int id;
 
         /** Name. */
-        @QuerySqlField(index = false)
         private final String name;
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
index 89ef2fc..8836cd5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java
@@ -64,6 +64,9 @@ public class JdbcThinConnection implements Connection {
     /** Logger. */
     private static final Logger LOG = Logger.getLogger(JdbcThinConnection.class.getName());
 
+    /** Connection URL. */
+    private String url;
+
     /** Schema name. */
     private String schemaName;
 
@@ -88,6 +91,9 @@ public class JdbcThinConnection implements Connection {
     /** Ignite endpoint. */
     private JdbcThinTcpIo cliIo;
 
+    /** Jdbc metadata. Cache the JDBC object on the first access */
+    private JdbcThinDatabaseMetadata metadata;
+
     /**
      * Creates new connection.
      *
@@ -99,6 +105,8 @@ public class JdbcThinConnection implements Connection {
         assert url != null;
         assert props != null;
 
+        this.url = url;
+
         holdability = HOLD_CURSORS_OVER_COMMIT;
         autoCommit = true;
         txIsolation = Connection.TRANSACTION_NONE;
@@ -274,7 +282,10 @@ public class JdbcThinConnection implements Connection {
     @Override public DatabaseMetaData getMetaData() throws SQLException {
         ensureNotClosed();
 
-        return null;
+        if (metadata == null)
+            metadata = new JdbcThinDatabaseMetadata(this);
+
+        return metadata;
     }
 
     /** {@inheritDoc} */
@@ -665,4 +676,11 @@ public class JdbcThinConnection implements Connection {
                 ", value=" + strVal + ']');
         }
     }
+
+    /**
+     * @return Connection URL.
+     */
+    public String url() {
+        return url;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
new file mode 100644
index 0000000..583bcec
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
@@ -0,0 +1,1587 @@
+/*
+ * 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.jdbc.thin;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.RowIdLifetime;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.internal.jdbc2.JdbcUtils;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcColumnMeta;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcIndexMeta;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaColumnsResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaIndexesResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaPrimaryKeysResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaSchemasResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcMetaTablesResult;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcPrimaryKeyMeta;
+import org.apache.ignite.internal.processors.odbc.jdbc.JdbcTableMeta;
+import org.apache.ignite.internal.util.typedef.F;
+
+import static java.sql.Connection.TRANSACTION_NONE;
+import static java.sql.ResultSet.CONCUR_READ_ONLY;
+import static java.sql.ResultSet.HOLD_CURSORS_OVER_COMMIT;
+import static java.sql.ResultSet.TYPE_FORWARD_ONLY;
+import static java.sql.RowIdLifetime.ROWID_UNSUPPORTED;
+
+/**
+ * JDBC database metadata implementation.
+ */
+@SuppressWarnings("RedundantCast")
+public class JdbcThinDatabaseMetadata implements DatabaseMetaData {
+    /** Connection. */
+    private final JdbcThinConnection conn;
+
+    /**
+     * @param conn Connection.
+     */
+    JdbcThinDatabaseMetadata(JdbcThinConnection conn) {
+        this.conn = conn;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean allProceduresAreCallable() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean allTablesAreSelectable() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getURL() throws SQLException {
+        return conn.url();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getUserName() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isReadOnly() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullsAreSortedHigh() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullsAreSortedLow() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullsAreSortedAtStart() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullsAreSortedAtEnd() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDatabaseProductName() throws SQLException {
+        return "Ignite";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDatabaseProductVersion() throws SQLException {
+        return conn.io().igniteVersion().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDriverName() throws SQLException {
+        return "Ignite JDBC Thin Driver";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDriverVersion() throws SQLException {
+        return IgniteVersionUtils.VER.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDriverMajorVersion() {
+        return IgniteVersionUtils.VER.major();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDriverMinorVersion() {
+        return IgniteVersionUtils.VER.minor();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean usesLocalFiles() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean usesLocalFilePerTable() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMixedCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesUpperCaseIdentifiers() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesLowerCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesMixedCaseIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMixedCaseQuotedIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesUpperCaseQuotedIdentifiers() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesLowerCaseQuotedIdentifiers() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean storesMixedCaseQuotedIdentifiers() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getIdentifierQuoteString() throws SQLException {
+        return "\"";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQLKeywords() throws SQLException {
+        return "LIMIT,MINUS,ROWNUM,SYSDATE,SYSTIME,SYSTIMESTAMP,TODAY";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getNumericFunctions() throws SQLException {
+        // TODO: IGNITE-6028
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getStringFunctions() throws SQLException {
+        // TODO: IGNITE-6028
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSystemFunctions() throws SQLException {
+        // TODO: IGNITE-6028
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTimeDateFunctions() throws SQLException {
+        // TODO: IGNITE-6028
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSearchStringEscape() throws SQLException {
+        return "\\";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getExtraNameCharacters() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsAlterTableWithAddColumn() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsAlterTableWithDropColumn() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsColumnAliasing() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nullPlusNonNullIsNull() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsConvert() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsConvert(int fromType, int toType) throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsTableCorrelationNames() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsDifferentTableCorrelationNames() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsExpressionsInOrderBy() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOrderByUnrelated() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsGroupBy() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsGroupByUnrelated() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsGroupByBeyondSelect() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsLikeEscapeClause() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMultipleResultSets() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMultipleTransactions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsNonNullableColumns() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMinimumSQLGrammar() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCoreSQLGrammar() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsExtendedSQLGrammar() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsANSI92EntryLevelSQL() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsANSI92IntermediateSQL() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsANSI92FullSQL() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsIntegrityEnhancementFacility() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOuterJoins() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsFullOuterJoins() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsLimitedOuterJoins() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSchemaTerm() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getProcedureTerm() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCatalogTerm() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isCatalogAtStart() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCatalogSeparator() throws SQLException {
+        return "";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInDataManipulation() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInProcedureCalls() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInTableDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInIndexDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSchemasInPrivilegeDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInDataManipulation() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInProcedureCalls() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInTableDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInIndexDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCatalogsInPrivilegeDefinitions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsPositionedDelete() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsPositionedUpdate() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSelectForUpdate() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsStoredProcedures() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSubqueriesInComparisons() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSubqueriesInExists() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSubqueriesInIns() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSubqueriesInQuantifieds() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsCorrelatedSubqueries() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsUnion() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsUnionAll() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOpenCursorsAcrossCommit() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOpenCursorsAcrossRollback() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOpenStatementsAcrossCommit() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsOpenStatementsAcrossRollback() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxBinaryLiteralLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxCharLiteralLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInGroupBy() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInIndex() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInOrderBy() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInSelect() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxColumnsInTable() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxConnections() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxCursorNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxIndexLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxSchemaNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxProcedureNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxCatalogNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxRowSize() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean doesMaxRowSizeIncludeBlobs() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxStatementLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxStatements() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxTableNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxTablesInSelect() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMaxUserNameLength() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDefaultTransactionIsolation() throws SQLException {
+        return TRANSACTION_NONE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsTransactions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsTransactionIsolationLevel(int level) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsDataDefinitionAndDataManipulationTransactions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsDataManipulationTransactionsOnly() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean dataDefinitionCausesTransactionCommit() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean dataDefinitionIgnoredInTransactions() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getProcedures(String catalog, String schemaPtrn,
+        String procedureNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PROCEDURE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_TYPE", String.class),
+            new JdbcColumnMeta(null, null, "SPECIFIC_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getProcedureColumns(String catalog, String schemaPtrn, String procedureNamePtrn,
+        String colNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PROCEDURE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PROCEDURE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "COLUMN_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PRECISION", Integer.class),
+            new JdbcColumnMeta(null, null, "LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "SCALE", Short.class),
+            new JdbcColumnMeta(null, null, "RADIX", Short.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Short.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_DEF", String.class),
+            new JdbcColumnMeta(null, null, "SQL_DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "SQL_DATETIME_SUB", Integer.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class),
+            new JdbcColumnMeta(null, null, "SPECIFIC_NAME", String.class)
+            ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getTables(String catalog, String schemaPtrn, String tblNamePtrn,
+        String[] tblTypes) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_TYPE", String.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "SELF_REFERENCING_COL_NAME", String.class),
+            new JdbcColumnMeta(null, null, "REF_GENERATION", String.class));
+
+        boolean tblTypeMatch = false;
+
+        if (tblTypes == null)
+            tblTypeMatch = true;
+        else {
+            for (String type : tblTypes) {
+                if ("TABLE".equals(type)) {
+                    tblTypeMatch = true;
+
+                    break;
+                }
+            }
+        }
+
+        if (!validCatalogPattern(catalog) || !tblTypeMatch)
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaTablesResult res = conn.io().tablesMeta(schemaPtrn, tblNamePtrn);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (JdbcTableMeta tblMeta : res.meta())
+                rows.add(tableRow(tblMeta));
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param tblMeta Table metadata.
+     * @return Table metadata row.
+     */
+    private List<Object> tableRow(JdbcTableMeta tblMeta) {
+        List<Object> row = new ArrayList<>(10);
+
+        row.add(null);
+        row.add(tblMeta.schemaName());
+        row.add(tblMeta.tableName());
+        row.add("TABLE");
+        row.add(null);
+        row.add(null);
+        row.add(null);
+        row.add(null);
+        row.add(null);
+        row.add(null);
+
+        return row;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getSchemas() throws SQLException {
+        return getSchemas(null, "%");
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ArraysAsListWithZeroOrOneArgument")
+    @Override public ResultSet getCatalogs() throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(),
+            Arrays.asList(new JdbcColumnMeta(null, null, "TABLE_CAT", String.class)));
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ArraysAsListWithZeroOrOneArgument")
+    @Override public ResultSet getTableTypes() throws SQLException {
+        return new JdbcThinResultSet(Collections.singletonList(Collections.<Object>singletonList("TABLE")),
+            Arrays.asList(new JdbcColumnMeta(null, null, "TABLE_TYPE", String.class)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getColumns(String catalog, String schemaPtrn, String tblNamePtrn,
+        String colNamePtrn) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Integer.class),
+            new JdbcColumnMeta(null, null, "NUM_PREC_RADIX", Short.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Short.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_DEF", String.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_CATLOG", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_SCHEMA", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_TABLE", String.class),
+            new JdbcColumnMeta(null, null, "SOURCE_DATA_TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "IS_AUTOINCREMENT", String.class));
+
+        if (!validCatalogPattern(catalog))
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaColumnsResult res = conn.io().columnsMeta(schemaPtrn, tblNamePtrn, colNamePtrn);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (int i = 0; i < res.meta().size(); ++i)
+                rows.add(columnRow(res.meta().get(i), i + 1));
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param colMeta Column metadata.
+     * @param pos Ordinal position.
+     * @return Column metadata row.
+     */
+    private List<Object> columnRow(JdbcColumnMeta colMeta, int pos) {
+        List<Object> row = new ArrayList<>(20);
+
+        row.add((String)null);
+        row.add(colMeta.schemaName());
+        row.add(colMeta.tableName());
+        row.add(colMeta.columnName());
+        row.add(colMeta.dataType());
+        row.add(colMeta.dataTypeName());
+        row.add((Integer)null);
+        row.add((Integer)null);
+        row.add(10);
+        row.add(JdbcUtils.nullable(colMeta.columnName(), colMeta.dataTypeClass()) ? 1 : 0 );
+        row.add((String)null);
+        row.add((String)null);
+        row.add(Integer.MAX_VALUE);
+        row.add(pos);
+        row.add("YES");
+        row.add((String)null);
+        row.add((String)null);
+        row.add((String)null);
+        row.add((Short)null);
+        row.add("NO");
+
+        return row;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getColumnPrivileges(String catalog, String schema, String tbl,
+        String colNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "GRANTOR", String.class),
+            new JdbcColumnMeta(null, null, "GRANTEE", String.class),
+            new JdbcColumnMeta(null, null, "PRIVILEGE", String.class),
+            new JdbcColumnMeta(null, null, "IS_GRANTABLE", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getTablePrivileges(String catalog, String schemaPtrn,
+        String tblNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "GRANTOR", String.class),
+            new JdbcColumnMeta(null, null, "GRANTEE", String.class),
+            new JdbcColumnMeta(null, null, "PRIVILEGE", String.class),
+            new JdbcColumnMeta(null, null, "IS_GRANTABLE", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getBestRowIdentifier(String catalog, String schema, String tbl, int scope,
+        boolean nullable) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "SCOPE", Short.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "BUFFER_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Short.class),
+            new JdbcColumnMeta(null, null, "PSEUDO_COLUMN", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getVersionColumns(String catalog, String schema, String tbl) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "SCOPE", Short.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "BUFFER_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Short.class),
+            new JdbcColumnMeta(null, null, "PSEUDO_COLUMN", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getPrimaryKeys(String catalog, String schema, String tbl) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "KEY_SEQ", Short.class),
+            new JdbcColumnMeta(null, null, "PK_NAME", String.class));
+
+        if (!validCatalogPattern(catalog))
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaPrimaryKeysResult res = conn.io().primaryKeysMeta(schema, tbl);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (JdbcPrimaryKeyMeta pkMeta : res.meta())
+                rows.addAll(primaryKeyRows(pkMeta));
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param pkMeta Primary key metadata.
+     * @return Result set rows for primary key.
+     */
+    private List<List<Object>> primaryKeyRows(JdbcPrimaryKeyMeta pkMeta) {
+        List<List<Object>> rows = new ArrayList<>(pkMeta.fields().size());
+
+        for (int i = 0; i < pkMeta.fields().size(); ++i) {
+            List<Object> row = new ArrayList<>(6);
+
+            row.add((String)null); // table catalog
+            row.add(pkMeta.schemaName());
+            row.add(pkMeta.tableName());
+            row.add(pkMeta.fields().get(i));
+            row.add((Integer)i + 1); // sequence number
+            row.add(pkMeta.name());
+
+            rows.add(row);
+        }
+
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getImportedKeys(String catalog, String schema, String tbl) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "KEY_SEQ", Short.class),
+            new JdbcColumnMeta(null, null, "UPDATE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "DELETE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "FK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DEFERRABILITY", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getExportedKeys(String catalog, String schema, String tbl) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "KEY_SEQ", Short.class),
+            new JdbcColumnMeta(null, null, "UPDATE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "DELETE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "FK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DEFERRABILITY", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getCrossReference(String parentCatalog, String parentSchema, String parentTbl,
+        String foreignCatalog, String foreignSchema, String foreignTbl) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "PKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "PKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FKTABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "FKCOLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "KEY_SEQ", Short.class),
+            new JdbcColumnMeta(null, null, "UPDATE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "DELETE_RULE", Short.class),
+            new JdbcColumnMeta(null, null, "FK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PK_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DEFERRABILITY", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getTypeInfo() throws SQLException {
+        List<List<Object>> types = new ArrayList<>(21);
+
+        types.add(Arrays.<Object>asList("BOOLEAN", Types.BOOLEAN, 1, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "BOOLEAN", 0, 0,
+            Types.BOOLEAN, 0, 10));
+
+        types.add(Arrays.<Object>asList("TINYINT", Types.TINYINT, 3, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "TINYINT", 0, 0,
+            Types.TINYINT, 0, 10));
+
+        types.add(Arrays.<Object>asList("SMALLINT", Types.SMALLINT, 5, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "SMALLINT", 0, 0,
+            Types.SMALLINT, 0, 10));
+
+        types.add(Arrays.<Object>asList("INTEGER", Types.INTEGER, 10, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "INTEGER", 0, 0,
+            Types.INTEGER, 0, 10));
+
+        types.add(Arrays.<Object>asList("BIGINT", Types.BIGINT, 19, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "BIGINT", 0, 0,
+            Types.BIGINT, 0, 10));
+
+        types.add(Arrays.<Object>asList("FLOAT", Types.FLOAT, 17, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "FLOAT", 0, 0,
+            Types.FLOAT, 0, 10));
+
+        types.add(Arrays.<Object>asList("REAL", Types.REAL, 7, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "REAL", 0, 0,
+            Types.REAL, 0, 10));
+
+        types.add(Arrays.<Object>asList("DOUBLE", Types.DOUBLE, 17, null, null, null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "DOUBLE", 0, 0,
+            Types.DOUBLE, 0, 10));
+
+        types.add(Arrays.<Object>asList("NUMERIC", Types.NUMERIC, Integer.MAX_VALUE, null, null, "PRECISION,SCALE",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "NUMERIC", 0, 0,
+            Types.NUMERIC, 0, 10));
+
+        types.add(Arrays.<Object>asList("DECIMAL", Types.DECIMAL, Integer.MAX_VALUE, null, null, "PRECISION,SCALE",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "DECIMAL", 0, 0,
+            Types.DECIMAL, 0, 10));
+
+        types.add(Arrays.<Object>asList("DATE", Types.DATE, 8, "DATE '", "'", null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "DATE", 0, 0,
+            Types.DATE, 0, null));
+
+        types.add(Arrays.<Object>asList("TIME", Types.TIME, 6, "TIME '", "'", null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "TIME", 0, 0,
+            Types.TIME, 0, null));
+
+        types.add(Arrays.<Object>asList("TIMESTAMP", Types.TIMESTAMP, 23, "TIMESTAMP '", "'", null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "TIMESTAMP", 0, 10,
+            Types.TIMESTAMP, 0, null));
+
+        types.add(Arrays.<Object>asList("CHAR", Types.CHAR, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, true, (short)typeSearchable, false, false, false, "CHAR", 0, 0,
+            Types.CHAR, 0, null));
+
+        types.add(Arrays.<Object>asList("VARCHAR", Types.VARCHAR, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, true, (short)typeSearchable, false, false, false, "VARCHAR", 0, 0,
+            Types.VARCHAR, 0, null));
+
+        types.add(Arrays.<Object>asList("LONGVARCHAR", Types.LONGVARCHAR, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, true, (short)typeSearchable, false, false, false, "LONGVARCHAR", 0, 0,
+            Types.LONGVARCHAR, 0, null));
+
+        types.add(Arrays.<Object>asList("BINARY", Types.BINARY, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "BINARY", 0, 0,
+            Types.BINARY, 0, null));
+
+        types.add(Arrays.<Object>asList("VARBINARY", Types.VARBINARY, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "VARBINARY", 0, 0,
+            Types.VARBINARY, 0, null));
+
+        types.add(Arrays.<Object>asList("LONGVARBINARY", Types.LONGVARBINARY, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "LONGVARBINARY", 0, 0,
+            Types.LONGVARBINARY, 0, null));
+
+        types.add(Arrays.<Object>asList("OTHER", Types.OTHER, Integer.MAX_VALUE, "'", "'", "LENGTH",
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "OTHER", 0, 0,
+            Types.OTHER, 0, null));
+
+        types.add(Arrays.<Object>asList("ARRAY", Types.ARRAY, 0, "(", "')", null,
+            (short)typeNullable, false, (short)typeSearchable, false, false, false, "ARRAY", 0, 0,
+            Types.ARRAY, 0, null));
+
+        return new JdbcThinResultSet(types, Arrays.asList(
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "PRECISION", Integer.class),
+            new JdbcColumnMeta(null, null, "LITERAL_PREFIX", String.class),
+            new JdbcColumnMeta(null, null, "LITERAL_SUFFIX", String.class),
+            new JdbcColumnMeta(null, null, "CREATE_PARAMS", String.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Short.class),
+            new JdbcColumnMeta(null, null, "CASE_SENSITIVE", Boolean.class),
+            new JdbcColumnMeta(null, null, "SEARCHABLE", Short.class),
+            new JdbcColumnMeta(null, null, "UNSIGNED_ATTRIBUTE", Boolean.class),
+            new JdbcColumnMeta(null, null, "FIXED_PREC_SCALE", Boolean.class),
+            new JdbcColumnMeta(null, null, "AUTO_INCREMENT", Boolean.class),
+            new JdbcColumnMeta(null, null, "LOCAL_TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "MINIMUM_SCALE", Short.class),
+            new JdbcColumnMeta(null, null, "MAXIMUM_SCALE", Short.class),
+            new JdbcColumnMeta(null, null, "SQL_DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "SQL_DATETIME_SUB", Integer.class),
+            new JdbcColumnMeta(null, null, "NUM_PREC_RADIX", Integer.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getIndexInfo(String catalog, String schema, String tbl, boolean unique,
+        boolean approximate) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "NON_UNIQUE", Boolean.class),
+            new JdbcColumnMeta(null, null, "INDEX_QUALIFIER", String.class),
+            new JdbcColumnMeta(null, null, "INDEX_NAME", String.class),
+            new JdbcColumnMeta(null, null, "TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Short.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "ASC_OR_DESC", String.class),
+            new JdbcColumnMeta(null, null, "CARDINALITY", Integer.class),
+            new JdbcColumnMeta(null, null, "PAGES", Integer.class),
+            new JdbcColumnMeta(null, null, "FILTER_CONDITION", String.class));
+
+        if (!validCatalogPattern(catalog))
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaIndexesResult res = conn.io().indexMeta(schema, tbl);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (JdbcIndexMeta idxMeta : res.meta())
+                rows.addAll(indexRows(idxMeta));
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /**
+     * @param idxMeta Index metadata.
+     * @return List of result rows correspond to index.
+     */
+    private List<List<Object>> indexRows(JdbcIndexMeta idxMeta) {
+        List<List<Object>> rows = new ArrayList<>(idxMeta.fields().size());
+
+        for (int i = 0; i < idxMeta.fields().size(); ++i) {
+            List<Object> row = new ArrayList<>(13);
+
+            row.add((String)null); // table catalog
+            row.add(idxMeta.schemaName());
+            row.add(idxMeta.tableName());
+            row.add(true); // non unique
+            row.add(null); // index qualifier (index catalog)
+            row.add(idxMeta.indexName());
+            row.add((short)tableIndexOther); // type
+            row.add((Integer)i); // field ordinal position in index
+            row.add(idxMeta.fields().get(i));
+            row.add(idxMeta.fieldsAsc().get(i) ? "A" : "D");
+            row.add((Integer)0); // cardinality
+            row.add((Integer)0); // pages
+            row.add((String)null); // filer condition
+
+            rows.add(row);
+        }
+
+        return rows;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsResultSetType(int type) throws SQLException {
+        return type == TYPE_FORWARD_ONLY;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsResultSetConcurrency(int type, int concurrency) throws SQLException {
+        return supportsResultSetType(type) && concurrency == CONCUR_READ_ONLY;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean ownUpdatesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean ownDeletesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean ownInsertsAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean othersUpdatesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean othersDeletesAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean othersInsertsAreVisible(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean updatesAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean deletesAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean insertsAreDetected(int type) throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsBatchUpdates() throws SQLException {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getUDTs(String catalog, String schemaPtrn, String typeNamePtrn,
+        int[] types) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "CLASS_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "BASE_TYPE", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Connection getConnection() throws SQLException {
+        return conn;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSavepoints() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsNamedParameters() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsMultipleOpenResults() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsGetGeneratedKeys() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getSuperTypes(String catalog, String schemaPtrn,
+        String typeNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "SUPERTYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "SUPERTYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "SUPERTYPE_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getSuperTables(String catalog, String schemaPtrn,
+        String tblNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "SUPERTABLE_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getAttributes(String catalog, String schemaPtrn, String typeNamePtrn,
+        String attributeNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TYPE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "ATTR_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "ATTR_TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "ATTR_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Integer.class),
+            new JdbcColumnMeta(null, null, "NUM_PREC_RADIX", Integer.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Integer.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "ATTR_DEF", String.class),
+            new JdbcColumnMeta(null, null, "SQL_DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "SQL_DATETIME_SUB", Integer.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_CATALOG", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_SCHEMA", String.class),
+            new JdbcColumnMeta(null, null, "SCOPE_TABLE", String.class),
+            new JdbcColumnMeta(null, null, "SOURCE_DATA_TYPE", Short.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsResultSetHoldability(int holdability) throws SQLException {
+        return holdability == HOLD_CURSORS_OVER_COMMIT;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getResultSetHoldability() throws SQLException {
+        return HOLD_CURSORS_OVER_COMMIT;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDatabaseMajorVersion() throws SQLException {
+        return conn.io().igniteVersion().major();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getDatabaseMinorVersion() throws SQLException {
+        return conn.io().igniteVersion().minor();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getJDBCMajorVersion() throws SQLException {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getJDBCMinorVersion() throws SQLException {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getSQLStateType() throws SQLException {
+        return DatabaseMetaData.sqlStateSQL99;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean locatorsUpdateCopy() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsStatementPooling() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public RowIdLifetime getRowIdLifetime() throws SQLException {
+        return ROWID_UNSUPPORTED;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getSchemas(String catalog, String schemaPtrn) throws SQLException {
+        if (conn.isClosed())
+            throw new SQLException("Connection is closed.");
+
+        final List<JdbcColumnMeta> meta = Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_CATALOG", String.class)
+        );
+
+        if (!validCatalogPattern(catalog))
+            return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), meta);
+
+        try {
+            JdbcMetaSchemasResult res = conn.io().schemasMeta(schemaPtrn);
+
+            List<List<Object>> rows = new LinkedList<>();
+
+            for (String schema : res.schemas()) {
+                List<Object> row = new ArrayList<>(2);
+
+                row.add(schema);
+                row.add(null);
+
+                rows.add(row);
+            }
+
+            return new JdbcThinResultSet(rows, meta);
+        }
+        catch (IOException e) {
+            conn.close();
+
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+        catch (IgniteCheckedException e) {
+            throw new SQLException("Failed to query Ignite.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsStoredFunctionsUsingCallSyntax() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean autoCommitFailureClosesAllResultSets() throws SQLException {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getClientInfoProperties() throws SQLException {
+        // TODO: IGNITE-5425.
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "NAME", String.class),
+            new JdbcColumnMeta(null, null, "MAX_LEN", Integer.class),
+            new JdbcColumnMeta(null, null, "DEFAULT_VALUE", String.class),
+            new JdbcColumnMeta(null, null, "DESCRIPTION", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getFunctions(String catalog, String schemaPtrn,
+        String functionNamePtrn) throws SQLException {
+        // TODO: IGNITE-6028
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "FUNCTION_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_NAME", String.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_TYPE", String.class),
+            new JdbcColumnMeta(null, null, "SPECIFIC_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getFunctionColumns(String catalog, String schemaPtrn, String functionNamePtrn,
+        String colNamePtrn) throws SQLException {
+        // TODO: IGNITE-6028
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "FUNCTION_CAT", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "FUNCTION_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_TYPE", Short.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "TYPE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "PRECISION", Integer.class),
+            new JdbcColumnMeta(null, null, "LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "SCALE", Short.class),
+            new JdbcColumnMeta(null, null, "RADIX", Short.class),
+            new JdbcColumnMeta(null, null, "NULLABLE", Short.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "ORDINAL_POSITION", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class),
+            new JdbcColumnMeta(null, null, "SPECIFIC_NAME", String.class)
+        ));
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!isWrapperFor(iface))
+            throw new SQLException("Database meta data is not a wrapper for " + iface.getName());
+
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface != null && iface.isAssignableFrom(JdbcThinDatabaseMetadata.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public ResultSet getPseudoColumns(String catalog, String schemaPtrn, String tblNamePtrn,
+        String colNamePtrn) throws SQLException {
+        return new JdbcThinResultSet(Collections.<List<Object>>emptyList(), Arrays.asList(
+            new JdbcColumnMeta(null, null, "TABLE_CAT", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_SCHEM", String.class),
+            new JdbcColumnMeta(null, null, "TABLE_NAME", String.class),
+            new JdbcColumnMeta(null, null, "COLUMN_NAME", String.class),
+            new JdbcColumnMeta(null, null, "DATA_TYPE", Integer.class),
+            new JdbcColumnMeta(null, null, "COLUMN_SIZE", Integer.class),
+            new JdbcColumnMeta(null, null, "DECIMAL_DIGITS", Integer.class),
+            new JdbcColumnMeta(null, null, "NUM_PREC_RADIX", Integer.class),
+            new JdbcColumnMeta(null, null, "COLUMN_USAGE", Integer.class),
+            new JdbcColumnMeta(null, null, "REMARKS", String.class),
+            new JdbcColumnMeta(null, null, "CHAR_OCTET_LENGTH", Integer.class),
+            new JdbcColumnMeta(null, null, "IS_NULLABLE", String.class)
+        ));
+    }
+
+    /**
+     * @param catalog Catalog pattern.
+     * @return {@code true} If patter is valid for Ignite (null, empty, or '%' wildcard).
+     *  Otherwise returns {@code false}.
+     */
+    private static boolean validCatalogPattern(String catalog) {
+        return F.isEmpty(catalog) || "%".equals(catalog);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean generatedKeyAlwaysReturned() throws SQLException {
+        return false;
+    }
+}
\ No newline at end of file


[12/19] ignite git commit: IGNITE-5233: Fixed JavaDocs.

Posted by ag...@apache.org.
IGNITE-5233: Fixed JavaDocs.


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

Branch: refs/heads/ignite-5901
Commit: 4402a1c886c9138ebe14e0a76c89fb6035caa15d
Parents: b417a36
Author: devozerov <vo...@gridgain.com>
Authored: Thu Aug 17 17:56:02 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 17 17:56:02 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java    | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4402a1c8/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
index dd3b18b..c0cfc9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcParameterMeta.java
@@ -26,8 +26,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
  * JDBC SQL query parameter metadata.
- *
- * {@see java.sql.ParameterMetaData}.
  */
 public class JdbcParameterMeta implements JdbcRawBinarylizable {
     /** Null value is allow for the param. */


[09/19] ignite git commit: IGNITE-6033 Added sorted and multithreaded modes in checkpointing algorithm - Fixes #2441.

Posted by ag...@apache.org.
IGNITE-6033 Added sorted and multithreaded modes in checkpointing algorithm - Fixes #2441.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-5901
Commit: 69e6f8b201a13f1c780948237960267c42ac5d2d
Parents: b417a36
Author: Ivan Rakov <iv...@gmail.com>
Authored: Thu Aug 17 15:54:21 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 15:54:21 2017 +0300

----------------------------------------------------------------------
 .../configuration/CheckpointWriteOrder.java     | 33 ++++++++
 .../PersistentStoreConfiguration.java           | 26 +++++++
 .../GridCacheDatabaseSharedManager.java         | 82 +++++++++++++++-----
 ...nitePersistenceSequentialCheckpointTest.java | 44 +++++++++++
 .../IgnitePersistentStoreCacheGroupsTest.java   | 31 ++++----
 5 files changed, 183 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/69e6f8b2/modules/core/src/main/java/org/apache/ignite/configuration/CheckpointWriteOrder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CheckpointWriteOrder.java b/modules/core/src/main/java/org/apache/ignite/configuration/CheckpointWriteOrder.java
new file mode 100644
index 0000000..31feaf6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CheckpointWriteOrder.java
@@ -0,0 +1,33 @@
+/*
+* 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.configuration;
+
+/**
+ * This enum defines order of writing pages to disk storage during checkpoint.
+ */
+public enum CheckpointWriteOrder {
+    /**
+     * Pages are written in order provided by checkpoint pages collection iterator (which is basically a hashtable).
+     */
+    RANDOM,
+
+    /**
+     * All checkpoint pages are collected into single list and sorted by page index.
+     * Provides almost sequential disk writes, which can be much faster on some SSD models.
+     */
+    SEQUENTIAL
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69e6f8b2/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
index e8a0ff4..5b902ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/PersistentStoreConfiguration.java
@@ -47,6 +47,9 @@ public class PersistentStoreConfiguration implements Serializable {
     /** Default number of checkpointing threads. */
     public static final int DFLT_CHECKPOINTING_THREADS = 1;
 
+    /** Default checkpoint write order. */
+    public static final CheckpointWriteOrder DFLT_CHECKPOINT_WRITE_ORDER = CheckpointWriteOrder.RANDOM;
+
     /** Default number of checkpoints to be kept in WAL after checkpoint is finished */
     public static final int DFLT_WAL_HISTORY_SIZE = 20;
 
@@ -95,6 +98,9 @@ public class PersistentStoreConfiguration implements Serializable {
     /** */
     private int checkpointingThreads = DFLT_CHECKPOINTING_THREADS;
 
+    /** Checkpoint write order. */
+    private CheckpointWriteOrder checkpointWriteOrder = DFLT_CHECKPOINT_WRITE_ORDER;
+
     /** Number of checkpoints to keep */
     private int walHistSize = DFLT_WAL_HISTORY_SIZE;
 
@@ -587,6 +593,26 @@ public class PersistentStoreConfiguration implements Serializable {
         return walAutoArchiveAfterInactivity;
     }
 
+    /**
+     * This property defines order of writing pages to disk storage during checkpoint.
+     *
+     * @return Checkpoint write order.
+     */
+    public CheckpointWriteOrder getCheckpointWriteOrder() {
+        return checkpointWriteOrder;
+    }
+
+    /**
+     * This property defines order of writing pages to disk storage during checkpoint.
+     *
+     * @param checkpointWriteOrder Checkpoint write order.
+     */
+    public PersistentStoreConfiguration setCheckpointWriteOrder(CheckpointWriteOrder checkpointWriteOrder) {
+        this.checkpointWriteOrder = checkpointWriteOrder;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(PersistentStoreConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/69e6f8b2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 99e05dd..3c7ba28 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -49,7 +49,6 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
@@ -62,6 +61,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.PersistenceMetrics;
+import org.apache.ignite.configuration.CheckpointWriteOrder;
 import org.apache.ignite.configuration.DataPageEvictionMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
@@ -93,7 +93,6 @@ import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PageDeltaRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionDestroyRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.PartitionMetaStateRecord;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
@@ -135,6 +134,7 @@ import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.mxbean.PersistenceMetricsMXBean;
 import org.apache.ignite.thread.IgniteThread;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -382,11 +382,12 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         long cpBufSize = persistenceCfg.getCheckpointingPageBufferSize();
 
         if (persistenceCfg.getCheckpointingThreads() > 1)
-            asyncRunner = new ThreadPoolExecutor(
+            asyncRunner = new IgniteThreadPoolExecutor(
+                "checkpoint-runner",
+                cctx.igniteInstanceName(),
                 persistenceCfg.getCheckpointingThreads(),
                 persistenceCfg.getCheckpointingThreads(),
-                30L,
-                TimeUnit.SECONDS,
+                30_000,
                 new LinkedBlockingQueue<Runnable>()
             );
 
@@ -2084,10 +2085,10 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
             WALPointer cpPtr = null;
 
-            GridMultiCollectionWrapper<FullPageId> cpPages;
-
             final CheckpointProgress curr;
 
+            IgniteBiTuple<Collection<GridMultiCollectionWrapper<FullPageId>>, Integer> cpPagesTuple;
+
             tracker.onLockWaitStart();
 
             checkpointLock.writeLock().lock();
@@ -2152,19 +2153,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
                 if (curr.nextSnapshot)
                     snapshotMgr.onMarkCheckPointBegin(curr.snapshotOperation, map);
 
-                IgniteBiTuple<Collection<GridMultiCollectionWrapper<FullPageId>>, Integer> tup = beginAllCheckpoints();
+                cpPagesTuple = beginAllCheckpoints();
 
-                // Todo it maybe more optimally
-                Collection<FullPageId> cpPagesList = new ArrayList<>(tup.get2());
-
-                for (GridMultiCollectionWrapper<FullPageId> col : tup.get1()) {
-                    for (int i = 0; i < col.collectionsSize(); i++)
-                        cpPagesList.addAll(col.innerCollection(i));
-                }
-
-                cpPages = new GridMultiCollectionWrapper<>(cpPagesList);
-
-                if (!F.isEmpty(cpPages)) {
+                if (!F.isEmpty(cpPagesTuple.get1())) {
                     // No page updates for this checkpoint are allowed from now on.
                     cpPtr = cctx.wal().log(cpRec);
 
@@ -2180,7 +2171,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
             curr.cpBeginFut.onDone();
 
-            if (!F.isEmpty(cpPages)) {
+            if (!F.isEmpty(cpPagesTuple.get1())) {
                 assert cpPtr != null;
 
                 // Sync log outside the checkpoint write lock.
@@ -2198,6 +2189,8 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
                 checkpointHist.addCheckpointEntry(cpEntry);
 
+                GridMultiCollectionWrapper<FullPageId> cpPages = splitAndSortCpPagesIfNeeded(cpPagesTuple);
+
                 if (printCheckpointStats)
                     if (log.isInfoEnabled())
                         log.info(String.format("Checkpoint started [checkpointId=%s, startPtr=%s, checkpointLockWait=%dms, " +
@@ -2295,6 +2288,55 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         }
     }
 
+    /**
+     * Reorders list of checkpoint pages and splits them into needed number of sublists according to
+     * {@link PersistentStoreConfiguration#getCheckpointingThreads()} and
+     * {@link PersistentStoreConfiguration#getCheckpointWriteOrder()}.
+     *
+     * @param cpPagesTuple Checkpoint pages tuple.
+     */
+    private GridMultiCollectionWrapper<FullPageId> splitAndSortCpPagesIfNeeded(
+        IgniteBiTuple<Collection<GridMultiCollectionWrapper<FullPageId>>, Integer> cpPagesTuple) {
+        List<FullPageId> cpPagesList = new ArrayList<>(cpPagesTuple.get2());
+
+        for (GridMultiCollectionWrapper<FullPageId> col : cpPagesTuple.get1()) {
+            for (int i = 0; i < col.collectionsSize(); i++)
+                cpPagesList.addAll(col.innerCollection(i));
+        }
+
+        if (persistenceCfg.getCheckpointWriteOrder() == CheckpointWriteOrder.SEQUENTIAL) {
+            Collections.sort(cpPagesList, new Comparator<FullPageId>() {
+                @Override public int compare(FullPageId o1, FullPageId o2) {
+                    int cmp = Long.compare(o1.groupId(), o2.groupId());
+                    if (cmp != 0)
+                        return cmp;
+
+                    return Long.compare(PageIdUtils.effectivePageId(o1.pageId()),
+                        PageIdUtils.effectivePageId(o2.pageId()));
+                }
+            });
+        }
+
+        int cpThreads = persistenceCfg.getCheckpointingThreads();
+
+        int pagesSubLists = cpThreads == 1 ? 1 : cpThreads * 4;
+        // Splitting pages to (threads * 4) subtasks. If any thread will be faster, it will help slower threads.
+
+        Collection[] pagesSubListArr = new Collection[pagesSubLists];
+
+        for (int i = 0; i < pagesSubLists; i++) {
+            int totalSize = cpPagesList.size();
+
+            int from = totalSize * i / (pagesSubLists);
+
+            int to = totalSize * (i + 1) / (pagesSubLists);
+
+            pagesSubListArr[i] = cpPagesList.subList(from, to);
+        }
+
+        return new GridMultiCollectionWrapper<FullPageId>(pagesSubListArr);
+    }
+
     /** Pages write task */
     private class WriteCheckpointPages implements Runnable {
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/69e6f8b2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistenceSequentialCheckpointTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistenceSequentialCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistenceSequentialCheckpointTest.java
new file mode 100644
index 0000000..9295000
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistenceSequentialCheckpointTest.java
@@ -0,0 +1,44 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*      http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.ignite.internal.processors.cache.persistence;
+
+import org.apache.ignite.configuration.CheckpointWriteOrder;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+
+/**
+ *
+ */
+public class IgnitePersistenceSequentialCheckpointTest extends IgnitePersistentStoreCacheGroupsTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration()
+            .setWalMode(WALMode.LOG_ONLY)
+            .setCheckpointingThreads(4)
+            .setCheckpointWriteOrder(CheckpointWriteOrder.SEQUENTIAL));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int entriesCount() {
+        return 1000;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/69e6f8b2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java
index a945c73..b39b8cb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePersistentStoreCacheGroupsTest.java
@@ -87,7 +87,7 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
 
         MemoryConfiguration memCfg = new MemoryConfiguration();
         memCfg.setPageSize(1024);
-        memCfg.setDefaultMemoryPolicySize(10 * 1024 * 1024);
+        memCfg.setDefaultMemoryPolicySize(100 * 1024 * 1024);
 
         cfg.setMemoryConfiguration(memCfg);
 
@@ -115,6 +115,11 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
         super.afterTest();
     }
 
+    /** Entries count. */
+    protected int entriesCount() {
+        return 10;
+    }
+
     /**
      * @throws Exception If failed.
      */
@@ -236,7 +241,7 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
         for (String cacheName : caches) {
             IgniteCache<Object, Object> cache = node.cache(cacheName).withExpiryPolicy(plc);
 
-            for (int i = 0; i < 10; i++)
+            for (int i = 0; i < entriesCount(); i++)
                 cache.put(i, cacheName + i);
         }
 
@@ -253,10 +258,10 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
         for (String cacheName : caches) {
             IgniteCache<Object, Object> cache = node.cache(cacheName);
 
-            for (int i = 0; i < 10; i++)
+            for (int i = 0; i < entriesCount(); i++)
                 assertEquals(cacheName + i, cache.get(i));
 
-            assertEquals(10, cache.size());
+            assertEquals(entriesCount(), cache.size());
         }
 
         // Wait for expiration.
@@ -340,7 +345,7 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
         for (String cacheName : caches) {
             IgniteCache<Object, Object> cache = node.cache(cacheName);
 
-            for (int i = 0; i < 10; i++)
+            for (int i = 0; i < entriesCount(); i++)
                 cache.put(i, new Person("" + i, cacheName));
         }
     }
@@ -353,10 +358,10 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
         for (String cacheName : caches) {
             IgniteCache<Object, Object> cache = node.cache(cacheName);
 
-            for (int i = 0; i < 10; i++)
+            for (int i = 0; i < entriesCount(); i++)
                 assertEquals(new Person("" + i, cacheName), cache.get(i));
 
-            assertEquals(10, cache.size());
+            assertEquals(entriesCount(), cache.size());
         }
     }
 
@@ -373,10 +378,10 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
 
             List<Cache.Entry<Integer, Person>> persons = cache.query(qry.setArgs(cacheName)).getAll();
 
-            for (int i = 0; i < 10; i++)
+            for (int i = 0; i < entriesCount(); i++)
                 assertEquals(new Person("" + i, cacheName), persons.get(i).getValue());
 
-            assertEquals(10, persons.size());
+            assertEquals(entriesCount(), persons.size());
         }
     }
 
@@ -413,13 +418,13 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
         for (String cacheName : caches) {
             IgniteCache<Object, Object> cache = node.cache(cacheName);
 
-            for (int i = 0; i < 10; i++)  {
+            for (int i = 0; i < entriesCount(); i++)  {
                 cache.put(i, cacheName + i);
 
                 assertEquals(cacheName + i, cache.get(i));
             }
 
-            assertEquals(10, cache.size());
+            assertEquals(entriesCount(), cache.size());
         }
 
         stopAllGrids();
@@ -433,10 +438,10 @@ public class IgnitePersistentStoreCacheGroupsTest extends GridCommonAbstractTest
         for (String cacheName : caches) {
             IgniteCache<Object, Object> cache = node.cache(cacheName);
 
-            for (int i = 0; i < 10; i++)
+            for (int i = 0; i < entriesCount(); i++)
                 assertEquals(cacheName + i, cache.get(i));
 
-            assertEquals(10, cache.size());
+            assertEquals(entriesCount(), cache.size());
         }
     }
 


[17/19] ignite git commit: IGNITE-6102 - Implemented persistence configuration consistency check. Fixes #2470

Posted by ag...@apache.org.
IGNITE-6102 - Implemented persistence configuration consistency check. Fixes #2470


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

Branch: refs/heads/ignite-5901
Commit: 5f19c1bf92c8a97587cabb18ea62a2f3ecccdf14
Parents: b3ae58e
Author: Ivan Rakov <iv...@gmail.com>
Authored: Thu Aug 17 19:10:33 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 19:10:33 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  8 ++
 .../ignite/internal/IgniteNodeAttributes.java   |  3 +
 .../processors/cache/GridCacheProcessor.java    | 21 ++++++
 ...dMemoryConfigurationConsistencySelfTest.java | 79 ++++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |  2 +
 5 files changed, 113 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f19c1bf/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index bae8b74..e926653 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -228,6 +228,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MEMORY_CONFIG;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM;
@@ -235,6 +236,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PREFIX;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_RESTART_ENABLED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_PORT_RANGE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SPI_CLASS;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_CONFIG;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME;
 import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR;
 import static org.apache.ignite.internal.IgniteVersionUtils.BUILD_TSTAMP_STR;
@@ -1571,6 +1573,12 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         if (cfg.getConnectorConfiguration() != null)
             add(ATTR_REST_PORT_RANGE, cfg.getConnectorConfiguration().getPortRange());
 
+        // Save database configuration.
+        add(ATTR_MEMORY_CONFIG, cfg.getMemoryConfiguration());
+
+        // Save transactions configuration.
+        add(ATTR_TX_CONFIG, cfg.getTransactionConfiguration());
+
         // Stick in SPI versions and classes attributes.
         addSpiAttributes(cfg.getCollisionSpi());
         addSpiAttributes(cfg.getDiscoverySpi());

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f19c1bf/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index fe23f92..ab6c290 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -180,6 +180,9 @@ public final class IgniteNodeAttributes {
     /** */
     public static final String ATTR_DATA_STREAMER_POOL_SIZE = ATTR_PREFIX + ".data.streamer.pool.size";
 
+    /** Memory configuration. */
+    public static final String ATTR_MEMORY_CONFIG = ATTR_PREFIX + ".memory";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f19c1bf/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index ef27a14..306c595 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -52,6 +52,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.events.EventType;
@@ -811,6 +812,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             checkTransactionConfiguration(n);
 
+            checkMemoryConfiguration(n);
+
             DeploymentMode locDepMode = ctx.config().getDeploymentMode();
             DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
 
@@ -3034,6 +3037,24 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param rmt Remote node to check.
+     * @throws IgniteCheckedException If check failed.
+     */
+    private void checkMemoryConfiguration(ClusterNode rmt) throws IgniteCheckedException {
+        MemoryConfiguration memCfg = rmt.attribute(IgniteNodeAttributes.ATTR_MEMORY_CONFIG);
+
+        if (memCfg != null) {
+            MemoryConfiguration locMemCfg = ctx.config().getMemoryConfiguration();
+
+            if (memCfg.getPageSize() != locMemCfg.getPageSize()) {
+                throw new IgniteCheckedException("Memory configuration mismatch (fix configuration or set -D" +
+                    IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK + "=true system property) [rmtNodeId=" + rmt.id() +
+                    ", locPageSize = " + locMemCfg.getPageSize() + ", rmtPageSize = " + memCfg.getPageSize() + "]");
+            }
+        }
+    }
+
+    /**
      * @param cfg Cache configuration.
      * @return Query manager.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f19c1bf/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridMemoryConfigurationConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridMemoryConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridMemoryConfigurationConsistencySelfTest.java
new file mode 100644
index 0000000..bc71e33
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridMemoryConfigurationConsistencySelfTest.java
@@ -0,0 +1,79 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*      http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.ignite.internal.processors.cache;
+
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests a check of memory configuration consistency.
+ */
+public class GridMemoryConfigurationConsistencySelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        MemoryConfiguration memCfg = new MemoryConfiguration();
+
+        // Nodes will have different page size.
+        memCfg.setPageSize(MemoryConfiguration.DFLT_PAGE_SIZE * (1 + getTestIgniteInstanceIndex(gridName)));
+
+        cfg.setMemoryConfiguration(memCfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMemoryConfigurationConsistency() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            /** {@inheritDoc} */
+            @Override public Void call() throws Exception {
+                startGrids(2);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f19c1bf/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 42d9534..dea0eb0 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -77,6 +77,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheStoreValueBytesSelfT
 import org.apache.ignite.internal.processors.cache.GridCacheSwapPreloadSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheTtlManagerSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheTxPartitionedLocalStoreSelfTest;
+import org.apache.ignite.internal.processors.cache.GridMemoryConfigurationConsistencySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicInvokeTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicLocalInvokeTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAtomicLocalWithStoreInvokeTest;
@@ -223,6 +224,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         // suite.addTestSuite(GridCacheP2PUndeploySelfTest.class);
         suite.addTestSuite(GridCacheConfigurationValidationSelfTest.class);
         suite.addTestSuite(GridCacheConfigurationConsistencySelfTest.class);
+        suite.addTestSuite(GridMemoryConfigurationConsistencySelfTest.class);
         suite.addTestSuite(GridCacheJdbcBlobStoreSelfTest.class);
         suite.addTestSuite(GridCacheJdbcBlobStoreMultithreadedSelfTest.class);
         suite.addTestSuite(JdbcTypesDefaultTransformerTest.class);


[03/19] ignite git commit: IGNITE-5233: JDBC thind driver: implemented metadata methods. This closes #2079.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java
index f132366..0b26dce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java
@@ -27,10 +27,10 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * SQL listener query execute request.
+ * JDBC query execute request.
  */
 public class JdbcQueryExecuteRequest extends JdbcRequest {
-    /** Cache name. */
+    /** Schema name. */
     private String schemaName;
 
     /** Fetch size. */
@@ -49,7 +49,7 @@ public class JdbcQueryExecuteRequest extends JdbcRequest {
 
     /**
      */
-    public JdbcQueryExecuteRequest() {
+    JdbcQueryExecuteRequest() {
         super(QRY_EXEC);
     }
 
@@ -100,7 +100,7 @@ public class JdbcQueryExecuteRequest extends JdbcRequest {
     }
 
     /**
-     * @return Cache name.
+     * @return Schema name.
      */
     @Nullable public String schemaName() {
         return schemaName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java
index a935215..fdebdb8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java
@@ -21,9 +21,10 @@ import java.util.List;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query execute result.
+ * JDBC query execute result.
  */
 public class JdbcQueryExecuteResult extends JdbcResult {
     /** Query ID. */
@@ -44,7 +45,7 @@ public class JdbcQueryExecuteResult extends JdbcResult {
     /**
      * Condtructor.
      */
-    public JdbcQueryExecuteResult() {
+    JdbcQueryExecuteResult() {
         super(QRY_EXEC);
     }
 
@@ -53,7 +54,7 @@ public class JdbcQueryExecuteResult extends JdbcResult {
      * @param items Query result rows.
      * @param last Flag indicates the query has no unfetched results.
      */
-    public JdbcQueryExecuteResult(long queryId, List<List<Object>> items, boolean last) {
+    JdbcQueryExecuteResult(long queryId, List<List<Object>> items, boolean last) {
         super(QRY_EXEC);
 
         this.queryId = queryId;
@@ -147,4 +148,9 @@ public class JdbcQueryExecuteResult extends JdbcResult {
             updateCnt = reader.readLong();
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcQueryExecuteResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java
index 2e1f551..776c3bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchRequest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query fetch request.
+ * JDBC query fetch request.
  */
 public class JdbcQueryFetchRequest extends JdbcRequest {
     /** Query ID. */
@@ -35,7 +35,7 @@ public class JdbcQueryFetchRequest extends JdbcRequest {
     /**
      * Constructor.
      */
-    public JdbcQueryFetchRequest() {
+    JdbcQueryFetchRequest() {
         super(QRY_FETCH);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java
index 6735c6b..ac4a603 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryFetchResult.java
@@ -21,9 +21,10 @@ import java.util.List;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query fetch result.
+ * JDBC query fetch result.
  */
 public class JdbcQueryFetchResult extends JdbcResult {
     /** Query result rows. */
@@ -35,7 +36,7 @@ public class JdbcQueryFetchResult extends JdbcResult {
     /**
      * Default constructor is used for deserialization.
      */
-    public JdbcQueryFetchResult() {
+    JdbcQueryFetchResult() {
         super(QRY_FETCH);
     }
 
@@ -43,7 +44,7 @@ public class JdbcQueryFetchResult extends JdbcResult {
      * @param items Query result rows.
      * @param last Flag indicating the query has no unfetched results.
      */
-    public JdbcQueryFetchResult(List<List<Object>> items, boolean last){
+    JdbcQueryFetchResult(List<List<Object>> items, boolean last){
         super(QRY_FETCH);
 
         this.items = items;
@@ -81,4 +82,9 @@ public class JdbcQueryFetchResult extends JdbcResult {
 
         items = JdbcUtils.readItems(reader);
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcQueryFetchResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java
index d14c9df..bdef321 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataRequest.java
@@ -23,47 +23,47 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query metadata request.
+ * JDBC query metadata request.
  */
 public class JdbcQueryMetadataRequest extends JdbcRequest {
     /** Query ID. */
-    private long queryId;
+    private long qryId;
 
     /**
      * Constructor.
      */
-    public JdbcQueryMetadataRequest() {
+    JdbcQueryMetadataRequest() {
         super(QRY_META);
     }
 
     /**
-     * @param queryId Query ID.
+     * @param qryId Query ID.
      */
-    public JdbcQueryMetadataRequest(long queryId) {
+    public JdbcQueryMetadataRequest(long qryId) {
         super(QRY_META);
 
-        this.queryId = queryId;
+        this.qryId = qryId;
     }
 
     /**
      * @return Query ID.
      */
     public long queryId() {
-        return queryId;
+        return qryId;
     }
 
     /** {@inheritDoc} */
     @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
         super.writeBinary(writer);
 
-        writer.writeLong(queryId);
+        writer.writeLong(qryId);
     }
 
     /** {@inheritDoc} */
     @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
         super.readBinary(reader);
 
-        queryId = reader.readLong();
+        qryId = reader.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java
index cc193e3..c8c0991 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryMetadataResult.java
@@ -24,9 +24,10 @@ import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- * SQL listener query metadata result.
+ * JDBC query metadata result.
  */
 public class JdbcQueryMetadataResult extends JdbcResult {
     /** Fields metadata. */
@@ -35,7 +36,7 @@ public class JdbcQueryMetadataResult extends JdbcResult {
     /**
      * Default constructor is used for deserialization.
      */
-    public JdbcQueryMetadataResult() {
+    JdbcQueryMetadataResult() {
         super(QRY_META);
     }
 
@@ -43,14 +44,14 @@ public class JdbcQueryMetadataResult extends JdbcResult {
      * @param queryId Query ID.
      * @param meta Query metadata.
      */
-    public JdbcQueryMetadataResult(long queryId, List<JdbcColumnMeta> meta){
+    JdbcQueryMetadataResult(long queryId, List<JdbcColumnMeta> meta){
         super(QRY_META);
 
         this.meta = meta;
     }
 
     /**
-     * @return Query result rows.
+     * @return Query result metadata.
      */
     public List<JdbcColumnMeta> meta() {
         return meta;
@@ -90,4 +91,9 @@ public class JdbcQueryMetadataResult extends JdbcResult {
             }
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcQueryMetadataResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
index 0e144cc..4ef75f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java
@@ -24,24 +24,43 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
 
 /**
- * SQL listener command request.
+ * JDBC request.
  */
 public class JdbcRequest extends SqlListenerRequest implements JdbcRawBinarylizable {
-    /** Execute sql query. */
-    public static final byte QRY_EXEC = 2;
+    /** Execute sql query request. */
+    static final byte QRY_EXEC = 2;
 
-    /** Fetch query results. */
-    public static final byte QRY_FETCH = 3;
+    /** Fetch query results request. */
+    static final byte QRY_FETCH = 3;
 
-    /** Close query. */
-    public static final byte QRY_CLOSE = 4;
+    /** Close query request. */
+    static final byte QRY_CLOSE = 4;
 
-    /** Get columns meta query. */
-    public static final byte QRY_META = 5;
+    /** Get query columns metadata request. */
+    static final byte QRY_META = 5;
 
     /** Batch queries. */
     public static final byte BATCH_EXEC = 6;
 
+    /** Get tables metadata request. */
+    static final byte META_TABLES = 7;
+
+    /** Get columns metadata request. */
+    static final byte META_COLUMNS = 8;
+
+    /** Get indexes metadata request. */
+    static final byte META_INDEXES = 9;
+
+    /** Get SQL query parameters metadata request. */
+    static final byte META_PARAMS = 10;
+
+    /** Get primary keys metadata request. */
+    static final byte META_PRIMARY_KEYS = 11;
+
+    /** Get schemas metadata request. */
+    static final byte META_SCHEMAS = 12;
+
+
     /** Request type. */
     private byte type;
 
@@ -105,6 +124,36 @@ public class JdbcRequest extends SqlListenerRequest implements JdbcRawBinaryliza
 
                 break;
 
+            case META_TABLES:
+                req = new JdbcMetaTablesRequest();
+
+                break;
+
+            case META_COLUMNS:
+                req = new JdbcMetaColumnsRequest();
+
+                break;
+
+            case META_INDEXES:
+                req = new JdbcMetaIndexesRequest();
+
+                break;
+
+            case META_PARAMS:
+                req = new JdbcMetaParamsRequest();
+
+                break;
+
+            case META_PRIMARY_KEYS:
+                req = new JdbcMetaPrimaryKeysRequest();
+
+                break;
+
+            case META_SCHEMAS:
+                req = new JdbcMetaSchemasRequest();
+
+                break;
+
             default:
                 throw new IgniteException("Unknown SQL listener request ID: [request ID=" + reqType + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index 60c08f9..7e58f99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -17,8 +17,15 @@
 
 package org.apache.ignite.internal.processors.odbc.jdbc;
 
+import java.sql.ParameterMetaData;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
@@ -26,10 +33,15 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteVersionUtils;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
 import org.apache.ignite.internal.processors.odbc.SqlListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.SqlListenerResponse;
+import org.apache.ignite.internal.processors.odbc.odbc.OdbcQueryGetColumnsMetaRequest;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.typedef.F;
@@ -37,13 +49,19 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BATCH_EXEC;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_COLUMNS;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_INDEXES;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_PARAMS;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_PRIMARY_KEYS;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_SCHEMAS;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_TABLES;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_CLOSE;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_EXEC;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_FETCH;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_META;
 
 /**
- * SQL query handler.
+ * JDBC request handler.
  */
 public class JdbcRequestHandler implements SqlListenerRequestHandler {
     /** Query ID sequence. */
@@ -92,7 +110,7 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
      * @param autoCloseCursors Flag to automatically close server cursors.
      */
     public JdbcRequestHandler(GridKernalContext ctx, GridSpinBusyLock busyLock, int maxCursors,
-        boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly, 
+        boolean distributedJoins, boolean enforceJoinOrder, boolean collocated, boolean replicatedOnly,
         boolean autoCloseCursors) {
         this.ctx = ctx;
         this.busyLock = busyLock;
@@ -134,6 +152,24 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
 
                 case BATCH_EXEC:
                     return executeBatch((JdbcBatchExecuteRequest)req);
+
+                case META_TABLES:
+                    return getTablesMeta((JdbcMetaTablesRequest)req);
+
+                case META_COLUMNS:
+                    return getColumnsMeta((JdbcMetaColumnsRequest)req);
+
+                case META_INDEXES:
+                    return getIndexesMeta((JdbcMetaIndexesRequest)req);
+
+                case META_PARAMS:
+                    return getParametersMeta((JdbcMetaParamsRequest)req);
+
+                case META_PRIMARY_KEYS:
+                    return getPrimaryKeys((JdbcMetaPrimaryKeysRequest)req);
+
+                case META_SCHEMAS:
+                    return getSchemas((JdbcMetaSchemasRequest)req);
             }
 
             return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, "Unsupported JDBC request [req=" + req + ']');
@@ -148,6 +184,20 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
         return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeHandshake(BinaryWriterExImpl writer) {
+        // Handshake OK.
+        writer.writeBoolean(true);
+
+        // Write server version.
+        writer.writeByte(IgniteVersionUtils.VER.major());
+        writer.writeByte(IgniteVersionUtils.VER.minor());
+        writer.writeByte(IgniteVersionUtils.VER.maintenance());
+        writer.writeString(IgniteVersionUtils.VER.stage());
+        writer.writeLong(IgniteVersionUtils.VER.revisionTimestamp());
+        writer.writeByteArray(IgniteVersionUtils.VER.revisionHash());
+    }
+
     /**
      * {@link JdbcQueryExecuteRequest} command handler.
      *
@@ -318,7 +368,7 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
      * @return Response.
      */
     private SqlListenerResponse executeBatch(JdbcBatchExecuteRequest req) {
-        String schemaName = req.schema();
+        String schemaName = req.schemaName();
 
         if (F.isEmpty(schemaName))
             schemaName = QueryUtils.DFLT_SCHEMA;
@@ -365,4 +415,221 @@ public class JdbcRequestHandler implements SqlListenerRequestHandler {
                 SqlListenerResponse.STATUS_FAILED, e.toString()));
         }
     }
+
+    /**
+     * @param req Get tables metadata request.
+     * @return Response.
+     */
+    private JdbcResponse getTablesMeta(JdbcMetaTablesRequest req) {
+        try {
+            List<JdbcTableMeta> meta = new ArrayList<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (!matches(table.schemaName(), req.schemaName()))
+                        continue;
+
+                    if (!matches(table.tableName(), req.tableName()))
+                        continue;
+
+                    JdbcTableMeta tableMeta = new JdbcTableMeta(table.schemaName(), table.tableName(), "TABLE");
+
+                    if (!meta.contains(tableMeta))
+                        meta.add(tableMeta);
+                }
+            }
+
+            JdbcMetaTablesResult res = new JdbcMetaTablesResult(meta);
+
+            return new JdbcResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get tables metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * {@link OdbcQueryGetColumnsMetaRequest} command handler.
+     *
+     * @param req Get columns metadata request.
+     * @return Response.
+     */
+    private JdbcResponse getColumnsMeta(JdbcMetaColumnsRequest req) {
+        try {
+            Collection<JdbcColumnMeta> meta = new HashSet<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (!matches(table.schemaName(), req.schemaName()))
+                        continue;
+
+                    if (!matches(table.tableName(), req.tableName()))
+                        continue;
+
+                    for (Map.Entry<String, Class<?>> field : table.fields().entrySet()) {
+                        if (!matches(field.getKey(), req.columnName()))
+                            continue;
+
+                        JdbcColumnMeta columnMeta = new JdbcColumnMeta(table.schemaName(), table.tableName(),
+                            field.getKey(), field.getValue());
+
+                        if (!meta.contains(columnMeta))
+                            meta.add(columnMeta);
+                    }
+                }
+            }
+
+            JdbcMetaColumnsResult res = new JdbcMetaColumnsResult(meta);
+
+            return new JdbcResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get columns metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @return Response.
+     */
+    private SqlListenerResponse getIndexesMeta(JdbcMetaIndexesRequest req) {
+        try {
+            Collection<JdbcIndexMeta> meta = new HashSet<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (!matches(table.schemaName(), req.schemaName()))
+                        continue;
+
+                    if (!matches(table.tableName(), req.tableName()))
+                        continue;
+
+                    for (GridQueryIndexDescriptor idxDesc : table.indexes().values())
+                        meta.add(new JdbcIndexMeta(table.schemaName(), table.tableName(), idxDesc));
+                }
+            }
+
+            return new JdbcResponse(new JdbcMetaIndexesResult(meta));
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get parameters metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @return Response.
+     */
+    private SqlListenerResponse getParametersMeta(JdbcMetaParamsRequest req) {
+        try {
+            ParameterMetaData paramMeta = ctx.query().prepareNativeStatement(req.schemaName(), req.sql())
+                .getParameterMetaData();
+
+            int size = paramMeta.getParameterCount();
+
+            List<JdbcParameterMeta> meta = new ArrayList<>(size);
+
+            for (int i = 0; i < size; i++)
+                meta.add(new JdbcParameterMeta(paramMeta, i + 1));
+
+            JdbcMetaParamsResult res = new JdbcMetaParamsResult(meta);
+
+            return new JdbcResponse(res);
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get parameters metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @return Response.
+     */
+    private SqlListenerResponse getPrimaryKeys(JdbcMetaPrimaryKeysRequest req) {
+        try {
+            Collection<JdbcPrimaryKeyMeta> meta = new HashSet<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (!matches(table.schemaName(), req.schemaName()))
+                        continue;
+
+                    if (!matches(table.tableName(), req.tableName()))
+                        continue;
+
+                    List<String> fields = new ArrayList<>();
+
+                    for (String field : table.fields().keySet()) {
+                        if (table.property(field).key())
+                            fields.add(field);
+                    }
+
+
+                    final String keyName = table.keyFieldName() == null ?
+                        "PK_" + table.schemaName() + "_" + table.tableName() :
+                        table.keyFieldName();
+
+                    if (fields.isEmpty()) {
+                        meta.add(new JdbcPrimaryKeyMeta(table.schemaName(), table.tableName(), keyName,
+                            Collections.singletonList("_KEY")));
+                    }
+                    else
+                        meta.add(new JdbcPrimaryKeyMeta(table.schemaName(), table.tableName(), keyName, fields));
+                }
+            }
+
+            return new JdbcResponse(new JdbcMetaPrimaryKeysResult(meta));
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get parameters metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * @param req Request.
+     * @return Response.
+     */
+    private SqlListenerResponse getSchemas(JdbcMetaSchemasRequest req) {
+        try {
+            String schemaPtrn = req.schemaName();
+
+            Set<String> schemas = new HashSet<>();
+
+            for (String cacheName : ctx.cache().publicCacheNames()) {
+                for (GridQueryTypeDescriptor table : ctx.query().types(cacheName)) {
+                    if (matches(table.schemaName(), schemaPtrn))
+                        schemas.add(table.schemaName());
+                }
+            }
+
+            return new JdbcResponse(new JdbcMetaSchemasResult(schemas));
+        }
+        catch (Exception e) {
+            U.error(log, "Failed to get schemas metadata [reqId=" + req.requestId() + ", req=" + req + ']', e);
+
+            return new JdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
+        }
+    }
+
+    /**
+     * Checks whether string matches SQL pattern.
+     *
+     * @param str String.
+     * @param ptrn Pattern.
+     * @return Whether string matches pattern.
+     */
+    private static boolean matches(String str, String ptrn) {
+        return str != null && (F.isEmpty(ptrn) ||
+            str.matches(ptrn.replace("%", ".*").replace("_", ".")));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
index 48affe9..202905b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
@@ -23,21 +23,39 @@ import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 
 /**
- * SQL listener response.
+ * JDBC response result.
  */
 public class JdbcResult implements JdbcRawBinarylizable {
     /** Execute sql result. */
-    public static final byte QRY_EXEC = 2;
+    static final byte QRY_EXEC = 2;
 
     /** Fetch query results. */
-    public static final byte QRY_FETCH = 3;
+    static final byte QRY_FETCH = 3;
 
-    /** Get columns meta query result. */
-    public static final byte QRY_META = 4;
+    /** Query result's columns metadata result. */
+    static final byte QRY_META = 5;
 
     /** Batch queries. */
     public static final byte BATCH_EXEC = 6;
 
+    /** Tables metadata result. */
+    static final byte META_TABLES = 7;
+
+    /** Columns metadata result. */
+    static final byte META_COLUMNS = 8;
+
+    /** Indexes metadata result. */
+    static final byte META_INDEXES = 9;
+
+    /** SQL query parameters metadata result. */
+    static final byte META_PARAMS = 10;
+
+    /** Primary keys metadata result. */
+    static final byte META_PRIMARY_KEYS = 11;
+
+    /** Database schemas metadata result. */
+    static final byte META_SCHEMAS = 12;
+
     /** Success status. */
     private byte type;
 
@@ -91,6 +109,36 @@ public class JdbcResult implements JdbcRawBinarylizable {
 
                 break;
 
+            case META_TABLES:
+                res = new JdbcMetaTablesResult();
+
+                break;
+
+            case META_COLUMNS:
+                res = new JdbcMetaColumnsResult();
+
+                break;
+
+            case META_INDEXES:
+                res = new JdbcMetaIndexesResult();
+
+                break;
+
+            case META_PARAMS:
+                res = new JdbcMetaParamsResult();
+
+                break;
+
+            case META_PRIMARY_KEYS:
+                res = new JdbcMetaPrimaryKeysResult();
+
+                break;
+
+            case META_SCHEMAS:
+                res = new JdbcMetaSchemasResult();
+
+                break;
+
             default:
                 throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java
new file mode 100644
index 0000000..b954e97
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTableMeta.java
@@ -0,0 +1,82 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC table metadata.
+ */
+public class JdbcTableMeta implements JdbcRawBinarylizable {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcTableMeta() {
+        // No-op.
+    }
+
+    /**
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param tblType Table type.
+     */
+    JdbcTableMeta(String schemaName, String tblName, String tblType) {
+        this.schemaName = schemaName;
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) throws BinaryObjectException {
+        writer.writeString(schemaName);
+        writer.writeString(tblName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) throws BinaryObjectException {
+        schemaName = reader.readString();
+        tblName = reader.readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcTableMeta.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java
index 65efbf5..d556419 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcUtils.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.odbc.jdbc;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
@@ -29,7 +30,7 @@ import org.apache.ignite.internal.processors.odbc.SqlListenerUtils;
  */
 public class JdbcUtils {
     /**
-     * @param writer Binari writer.
+     * @param writer Binary writer.
      * @param items Query results items.
      */
     public static void writeItems(BinaryWriterExImpl writer, List<List<Object>> items) {
@@ -70,4 +71,38 @@ public class JdbcUtils {
         } else
             return Collections.emptyList();
     }
+
+    /**
+     * @param writer Binary writer.
+     * @param lst List to write.
+     */
+    public static void writeStringCollection(BinaryWriterExImpl writer, Collection<String> lst) {
+        if (lst == null)
+            writer.writeInt(0);
+        else {
+            writer.writeInt(lst.size());
+
+            for (String s : lst)
+                writer.writeString(s);
+        }
+    }
+
+    /**
+     * @param reader Binary reader.
+     * @return List of string.
+     */
+    public static List<String> readStringList(BinaryReaderExImpl reader) {
+        int size = reader.readInt();
+
+        if (size > 0) {
+            List<String> lst = new ArrayList<>(size);
+
+            for (int i = 0; i < size; ++i)
+                lst.add(reader.readString());
+
+            return lst;
+        }
+        else
+            return Collections.emptyList();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
index bb54b59..692043c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java
@@ -33,6 +33,7 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.GridBinaryMarshaller;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.odbc.SqlListenerRequest;
@@ -156,6 +157,11 @@ public class OdbcRequestHandler implements SqlListenerRequestHandler {
         return new OdbcResponse(SqlListenerResponse.STATUS_FAILED, e.toString());
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeHandshake(BinaryWriterExImpl writer) {
+        writer.writeBoolean(true);
+    }
+
     /**
      * {@link OdbcQueryExecuteRequest} command handler.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index baafb1e..1d154d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -688,7 +688,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                 if (!F.isEmpty(qryEntities)) {
                     for (QueryEntity qryEntity : qryEntities) {
-                        QueryTypeCandidate cand = QueryUtils.typeForQueryEntity(cacheName, cctx, qryEntity,
+                        QueryTypeCandidate cand = QueryUtils.typeForQueryEntity(cacheName, schemaName, cctx, qryEntity,
                             mustDeserializeClss, escape);
 
                         cands.add(cand);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
index c1a9e1e..c149335 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
@@ -33,6 +33,13 @@ public interface GridQueryTypeDescriptor {
     public String name();
 
     /**
+     * Gets schema name for type (database schema means here).
+     *
+     * @return Schema name.
+     */
+    public String schemaName();
+
+    /**
      * Gets table name for type.
      *
      * @return Table name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index c0da83f..79b90e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -42,6 +42,9 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
     /** */
     private String name;
 
+    /** Schema name. */
+    private String schemaName;
+
     /** */
     private String tblName;
 
@@ -120,6 +123,11 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
         return name;
     }
 
+    /** {@inheritDoc} */
+    @Override public String schemaName() {
+        return schemaName;
+    }
+
     /**
      * Sets type name.
      *
@@ -363,6 +371,13 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
         fields.put(name, prop.type());
     }
 
+    /**
+     * @param schemaName Schema name.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean valueTextIndex() {
         return valTextIdx;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 320b25a..26fc776 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -343,6 +343,7 @@ public class QueryUtils {
      * Create type candidate for query entity.
      *
      * @param cacheName Cache name.
+     * @param schemaName Schema name.
      * @param cctx Cache context.
      * @param qryEntity Query entity.
      * @param mustDeserializeClss Classes which must be deserialized.
@@ -350,8 +351,8 @@ public class QueryUtils {
      * @return Type candidate.
      * @throws IgniteCheckedException If failed.
      */
-    public static QueryTypeCandidate typeForQueryEntity(String cacheName, GridCacheContext cctx, QueryEntity qryEntity,
-        List<Class<?>> mustDeserializeClss, boolean escape) throws IgniteCheckedException {
+    public static QueryTypeCandidate typeForQueryEntity(String cacheName, String schemaName, GridCacheContext cctx,
+        QueryEntity qryEntity, List<Class<?>> mustDeserializeClss, boolean escape) throws IgniteCheckedException {
         GridKernalContext ctx = cctx.kernalContext();
         CacheConfiguration<?,?> ccfg = cctx.config();
 
@@ -361,6 +362,8 @@ public class QueryUtils {
 
         QueryTypeDescriptorImpl desc = new QueryTypeDescriptorImpl(cacheName);
 
+        desc.schemaName(schemaName);
+
         desc.aliases(qryEntity.getAliases());
 
         // Key and value classes still can be available if they are primitive or JDK part.

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e803144/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 99246eb..e1ab8e6 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -104,13 +104,13 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
     }
 
     /** */
-    private static TypeDesc typeAA = new TypeDesc("A", "A", Collections.<String, Class<?>>emptyMap(), null);
+    private static TypeDesc typeAA = new TypeDesc("A", "A", "A", Collections.<String, Class<?>>emptyMap(), null);
 
     /** */
-    private static TypeDesc typeAB = new TypeDesc("A", "B", Collections.<String, Class<?>>emptyMap(), textIdx);
+    private static TypeDesc typeAB = new TypeDesc("A", "A", "B", Collections.<String, Class<?>>emptyMap(), textIdx);
 
     /** */
-    private static TypeDesc typeBA = new TypeDesc("B", "A", Collections.<String, Class<?>>emptyMap(), null);
+    private static TypeDesc typeBA = new TypeDesc("B", "B", "A", Collections.<String, Class<?>>emptyMap(), null);
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
@@ -467,6 +467,9 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         private final String cacheName;
 
         /** */
+        private final String schemaName;
+
+        /** */
         private final Map<String, Class<?>> valFields;
 
         /** */
@@ -474,13 +477,15 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
 
         /**
          * @param cacheName Cache name.
+         * @param schemaName Schema name.
          * @param name Type name.
          * @param valFields Fields.
          * @param textIdx Fulltext index.
          */
-        private TypeDesc(String cacheName, String name, Map<String, Class<?>> valFields, GridQueryIndexDescriptor textIdx) {
+        private TypeDesc(String cacheName, String schemaName, String name, Map<String, Class<?>> valFields, GridQueryIndexDescriptor textIdx) {
             this.name = name;
             this.cacheName = cacheName;
+            this.schemaName = schemaName;
             this.valFields = Collections.unmodifiableMap(valFields);
             this.textIdx = textIdx;
         }
@@ -496,6 +501,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
+        @Override public String schemaName() {
+            return schemaName;
+        }
+
+        /** {@inheritDoc} */
         @Override public String tableName() {
             return null;
         }


[02/19] ignite git commit: IGNITE-6080: DML batches are now grouped by affinity. This closes #2454.

Posted by ag...@apache.org.
IGNITE-6080: DML batches are now grouped by affinity. This closes #2454.


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

Branch: refs/heads/ignite-5901
Commit: 37e58badefcf36536644344023dc62828f41d0f9
Parents: 5283e19
Author: devozerov <vo...@gridgain.com>
Authored: Thu Aug 17 11:24:06 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Aug 17 11:24:06 2017 +0300

----------------------------------------------------------------------
 .../query/h2/DmlStatementsProcessor.java        | 274 +++++++++++--------
 1 file changed, 163 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/37e58bad/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
index 4f7c288..0ff9cfe 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/DmlStatementsProcessor.java
@@ -32,6 +32,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 import javax.cache.processor.EntryProcessor;
@@ -46,7 +47,9 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheOperationContext;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -83,7 +86,6 @@ import org.h2.command.dml.Update;
 import org.h2.table.Column;
 import org.h2.util.DateTimeUtils;
 import org.h2.util.LocalDateTimeUtils;
-import org.h2.value.DataType;
 import org.h2.value.Value;
 import org.h2.value.ValueDate;
 import org.h2.value.ValueTime;
@@ -365,7 +367,7 @@ public class DmlStatementsProcessor {
         QueryCursorImpl<List<?>> cur;
 
         // Do a two-step query only if locality flag is not set AND if plan's SELECT corresponds to an actual
-        // subquery and not some dummy stuff like "select 1, 2, 3;"
+        // sub-query and not some dummy stuff like "select 1, 2, 3;"
         if (!loc && !plan.isLocSubqry) {
             SqlFieldsQuery newFieldsQry = new SqlFieldsQuery(plan.selectQry, fieldsQry.isCollocated())
                 .setArgs(fieldsQry.getArgs())
@@ -493,52 +495,28 @@ public class DmlStatementsProcessor {
     @SuppressWarnings({"unchecked", "ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
     private UpdateResult doDelete(GridCacheContext cctx, Iterable<List<?>> cursor, int pageSize)
         throws IgniteCheckedException {
-        // With DELETE, we have only two columns - key and value.
-        long res = 0;
+        BatchSender sender = new BatchSender(cctx, pageSize);
 
-        // Keys that failed to DELETE due to concurrent updates.
-        List<Object> failedKeys = new ArrayList<>();
+        for (List<?> row : cursor) {
+            if (row.size() != 2) {
+                U.warn(log, "Invalid row size on DELETE - expected 2, got " + row.size());
 
-        SQLException resEx = null;
-
-
-        Iterator<List<?>> it = cursor.iterator();
-        Map<Object, EntryProcessor<Object, Object, Boolean>> rows = new LinkedHashMap<>();
-
-        while (it.hasNext()) {
-            List<?> e = it.next();
-            if (e.size() != 2) {
-                U.warn(log, "Invalid row size on DELETE - expected 2, got " + e.size());
                 continue;
             }
 
-            rows.put(e.get(0), new ModifyingEntryProcessor(e.get(1), RMV));
-
-            if ((pageSize > 0 && rows.size() == pageSize) || (!it.hasNext())) {
-                PageProcessingResult pageRes = processPage(cctx, rows);
-
-                res += pageRes.cnt;
-
-                failedKeys.addAll(F.asList(pageRes.errKeys));
+            sender.add(row.get(0), new ModifyingEntryProcessor(row.get(1), RMV));
+        }
 
-                if (pageRes.ex != null) {
-                    if (resEx == null)
-                        resEx = pageRes.ex;
-                    else
-                        resEx.setNextException(pageRes.ex);
-                }
+        sender.flush();
 
-                if (it.hasNext())
-                    rows.clear(); // No need to clear after the last batch.
-            }
-        }
+        SQLException resEx = sender.error();
 
         if (resEx != null) {
-            if (!F.isEmpty(failedKeys)) {
+            if (!F.isEmpty(sender.failedKeys())) {
                 // Don't go for a re-run if processing of some keys yielded exceptions and report keys that
                 // had been modified concurrently right away.
                 String msg = "Failed to DELETE some keys because they had been modified concurrently " +
-                    "[keys=" + failedKeys + ']';
+                    "[keys=" + sender.failedKeys() + ']';
 
                 SQLException conEx = createJdbcSqlException(msg, IgniteQueryErrorCode.CONCURRENT_UPDATE);
 
@@ -550,7 +528,7 @@ public class DmlStatementsProcessor {
             throw new IgniteSQLException(resEx);
         }
 
-        return new UpdateResult(res, failedKeys.toArray());
+        return new UpdateResult(sender.updateCount(), sender.failedKeys().toArray());
     }
 
     /**
@@ -579,20 +557,10 @@ public class DmlStatementsProcessor {
         // or if its list of updated columns includes only _val, i.e. is single element.
         boolean hasProps = !hasNewVal || updatedColNames.length > 1;
 
-        long res = 0;
-
-        Map<Object, EntryProcessor<Object, Object, Boolean>> rows = new LinkedHashMap<>();
-
-        // Keys that failed to UPDATE due to concurrent updates.
-        List<Object> failedKeys = new ArrayList<>();
-
-        SQLException resEx = null;
-
-        Iterator<List<?>> it = cursor.iterator();
+        BatchSender sender = new BatchSender(cctx, pageSize);
 
-        while (it.hasNext()) {
-            List<?> e = it.next();
-            Object key = e.get(0);
+        for (List<?> row : cursor) {
+            Object key = row.get(0);
 
             Object newVal;
 
@@ -606,10 +574,10 @@ public class DmlStatementsProcessor {
 
                 assert prop != null;
 
-                newColVals.put(plan.colNames[i], convert(e.get(i + 2), desc, prop.type(), plan.colTypes[i]));
+                newColVals.put(plan.colNames[i], convert(row.get(i + 2), desc, prop.type(), plan.colTypes[i]));
             }
 
-            newVal = plan.valSupplier.apply(e);
+            newVal = plan.valSupplier.apply(row);
 
             if (newVal == null)
                 throw new IgniteSQLException("New value for UPDATE must not be null", IgniteQueryErrorCode.NULL_VALUE);
@@ -643,38 +611,24 @@ public class DmlStatementsProcessor {
                 newVal = ((BinaryObjectBuilder) newVal).build();
             }
 
-            Object srcVal = e.get(1);
+            Object srcVal = row.get(1);
 
             if (bin && !(srcVal instanceof BinaryObject))
                 srcVal = cctx.grid().binary().toBinary(srcVal);
 
-            rows.put(key, new ModifyingEntryProcessor(srcVal, new EntryValueUpdater(newVal)));
-
-            if ((pageSize > 0 && rows.size() == pageSize) || (!it.hasNext())) {
-                PageProcessingResult pageRes = processPage(cctx, rows);
-
-                res += pageRes.cnt;
-
-                failedKeys.addAll(F.asList(pageRes.errKeys));
+            sender.add(key, new ModifyingEntryProcessor(srcVal, new EntryValueUpdater(newVal)));
+        }
 
-                if (pageRes.ex != null) {
-                    if (resEx == null)
-                        resEx = pageRes.ex;
-                    else
-                        resEx.setNextException(pageRes.ex);
-                }
+        sender.flush();
 
-                if (it.hasNext())
-                    rows.clear(); // No need to clear after the last batch.
-            }
-        }
+        SQLException resEx = sender.error();
 
         if (resEx != null) {
-            if (!F.isEmpty(failedKeys)) {
+            if (!F.isEmpty(sender.failedKeys())) {
                 // Don't go for a re-run if processing of some keys yielded exceptions and report keys that
                 // had been modified concurrently right away.
                 String msg = "Failed to UPDATE some keys because they had been modified concurrently " +
-                    "[keys=" + failedKeys + ']';
+                    "[keys=" + sender.failedKeys() + ']';
 
                 SQLException dupEx = createJdbcSqlException(msg, IgniteQueryErrorCode.CONCURRENT_UPDATE);
 
@@ -686,7 +640,7 @@ public class DmlStatementsProcessor {
             throw new IgniteSQLException(resEx);
         }
 
-        return new UpdateResult(res, failedKeys.toArray());
+        return new UpdateResult(sender.updateCount(), sender.failedKeys().toArray());
     }
 
     /**
@@ -864,47 +818,22 @@ public class DmlStatementsProcessor {
                     IgniteQueryErrorCode.DUPLICATE_KEY);
         }
         else {
-            Map<Object, EntryProcessor<Object, Object, Boolean>> rows = plan.isLocSubqry ?
-                new LinkedHashMap<Object, EntryProcessor<Object, Object, Boolean>>(plan.rowsNum) :
-                new LinkedHashMap<Object, EntryProcessor<Object, Object, Boolean>>();
-
             // Keys that failed to INSERT due to duplication.
-            List<Object> duplicateKeys = new ArrayList<>();
-
-            int resCnt = 0;
-
-            SQLException resEx = null;
-
-            Iterator<List<?>> it = cursor.iterator();
-
-            while (it.hasNext()) {
-                List<?> row = it.next();
-
-                final IgniteBiTuple t = rowToKeyValue(cctx, row, plan);
-
-                rows.put(t.getKey(), new InsertEntryProcessor(t.getValue()));
-
-                if (!it.hasNext() || (pageSize > 0 && rows.size() == pageSize)) {
-                    PageProcessingResult pageRes = processPage(cctx, rows);
+            BatchSender sender = new BatchSender(cctx, pageSize);
 
-                    resCnt += pageRes.cnt;
+            for (List<?> row : cursor) {
+                final IgniteBiTuple keyValPair = rowToKeyValue(cctx, row, plan);
 
-                    duplicateKeys.addAll(F.asList(pageRes.errKeys));
+                sender.add(keyValPair.getKey(), new InsertEntryProcessor(keyValPair.getValue()));
+            }
 
-                    if (pageRes.ex != null) {
-                        if (resEx == null)
-                            resEx = pageRes.ex;
-                        else
-                            resEx.setNextException(pageRes.ex);
-                    }
+            sender.flush();
 
-                    rows.clear();
-                }
-            }
+            SQLException resEx = sender.error();
 
-            if (!F.isEmpty(duplicateKeys)) {
+            if (!F.isEmpty(sender.failedKeys())) {
                 String msg = "Failed to INSERT some keys because they are already in cache " +
-                    "[keys=" + duplicateKeys + ']';
+                    "[keys=" + sender.failedKeys() + ']';
 
                 SQLException dupEx = new SQLException(msg, null, IgniteQueryErrorCode.DUPLICATE_KEY);
 
@@ -917,7 +846,7 @@ public class DmlStatementsProcessor {
             if (resEx != null)
                 throw new IgniteSQLException(resEx);
 
-            return resCnt;
+            return sender.updateCount();
         }
     }
 
@@ -1133,7 +1062,7 @@ public class DmlStatementsProcessor {
         /** Number of processed items. */
         final long cnt;
 
-        /** Keys that failed to be UPDATEd or DELETEd due to concurrent modification of values. */
+        /** Keys that failed to be updated or deleted due to concurrent modification of values. */
         @NotNull
         final Object[] errKeys;
 
@@ -1150,7 +1079,7 @@ public class DmlStatementsProcessor {
         /** Number of successfully processed items. */
         final long cnt;
 
-        /** Keys that failed to be UPDATEd or DELETEd due to concurrent modification of values. */
+        /** Keys that failed to be updated or deleted due to concurrent modification of values. */
         @NotNull
         final Object[] errKeys;
 
@@ -1193,4 +1122,127 @@ public class DmlStatementsProcessor {
             this.ex = ex;
         }
     }
+
+    /**
+     * Batch sender class.
+     */
+    private static class BatchSender {
+        /** Cache context. */
+        private final GridCacheContext cctx;
+
+        /** Batch size. */
+        private final int size;
+
+        /** Batches. */
+        private final Map<UUID, Map<Object, EntryProcessor<Object, Object, Boolean>>> batches = new HashMap<>();
+
+        /** Result count. */
+        private long updateCnt;
+
+        /** Failed keys. */
+        private List<Object> failedKeys;
+
+        /** Exception. */
+        private SQLException err;
+
+        /**
+         * Constructor.
+         *
+         * @param cctx Cache context.
+         * @param size Batch.
+         */
+        public BatchSender(GridCacheContext cctx, int size) {
+            this.cctx = cctx;
+            this.size = size;
+        }
+
+        /**
+         * Add entry to batch.
+         *
+         * @param key Key.
+         * @param proc Processor.
+         */
+        public void add(Object key, EntryProcessor<Object, Object, Boolean> proc) throws IgniteCheckedException {
+            ClusterNode node = cctx.affinity().primaryByKey(key, AffinityTopologyVersion.NONE);
+
+            if (node == null)
+                throw new IgniteCheckedException("Failed to map key to node.");
+
+            UUID nodeId = node.id();
+
+            Map<Object, EntryProcessor<Object, Object, Boolean>> batch = batches.get(nodeId);
+
+            if (batch == null) {
+                batch = new HashMap<>();
+
+                batches.put(nodeId, batch);
+            }
+
+            batch.put(key, proc);
+
+            if (batch.size() >= size) {
+                sendBatch(batch);
+
+                batch.clear();
+            }
+        }
+
+        /**
+         * Flush any remaining entries.
+         *
+         * @throws IgniteCheckedException If failed.
+         */
+        public void flush() throws IgniteCheckedException {
+            for (Map<Object, EntryProcessor<Object, Object, Boolean>> batch : batches.values()) {
+                if (!batch.isEmpty())
+                    sendBatch(batch);
+            }
+        }
+
+        /**
+         * @return Update count.
+         */
+        public long updateCount() {
+            return updateCnt;
+        }
+
+        /**
+         * @return Failed keys.
+         */
+        public List<Object> failedKeys() {
+            return failedKeys != null ? failedKeys : Collections.emptyList();
+        }
+
+        /**
+         * @return Error.
+         */
+        public SQLException error() {
+            return err;
+        }
+
+        /**
+         * Send the batch.
+         *
+         * @param batch Batch.
+         * @throws IgniteCheckedException If failed.
+         */
+        private void sendBatch(Map<Object, EntryProcessor<Object, Object, Boolean>> batch)
+            throws IgniteCheckedException {
+            PageProcessingResult pageRes = processPage(cctx, batch);
+
+            updateCnt += pageRes.cnt;
+
+            if (failedKeys == null)
+                failedKeys = new ArrayList<>();
+
+            failedKeys.addAll(F.asList(pageRes.errKeys));
+
+            if (pageRes.ex != null) {
+                if (err == null)
+                    err = pageRes.ex;
+                else
+                    err.setNextException(pageRes.ex);
+            }
+        }
+    }
 }


[06/19] ignite git commit: IGNITE-6091 Fixed flaky test CacheLateAffinityAssignmentTest.testRandomOperations - Fixes #2462.

Posted by ag...@apache.org.
IGNITE-6091 Fixed flaky test CacheLateAffinityAssignmentTest.testRandomOperations - Fixes #2462.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-5901
Commit: 9cfb05011ab5085c8166fc0ba8b1047906845694
Parents: 0e80314
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Thu Aug 17 13:46:03 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Aug 17 13:46:03 2017 +0300

----------------------------------------------------------------------
 .../cache/transactions/IgniteTxImplicitSingleStateImpl.java     | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9cfb0501/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
index 886d0d6..10b06d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxImplicitSingleStateImpl.java
@@ -115,8 +115,9 @@ public class IgniteTxImplicitSingleStateImpl extends IgniteTxLocalStateAdapter {
         Throwable err = topFut.validateCache(cacheCtx, recovery, read, null, entry);
 
         if (err != null) {
-            return new IgniteCheckedException("Failed to perform cache operation (cache topology is not valid): " +
-                U.maskName(cacheCtx.name()));
+            return new IgniteCheckedException(
+                "Failed to perform cache operation (cache topology is not valid): "
+                    + U.maskName(cacheCtx.name()), err);
         }
 
         if (CU.affinityNodes(cacheCtx, topFut.topologyVersion()).isEmpty()) {