You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/08/29 18:04:45 UTC

[01/24] ignite git commit: IGNITE-3716: ODBC: Added SQL escape sequence parsing.

Repository: ignite
Updated Branches:
  refs/heads/master 76f26c1e6 -> d98cd3093


IGNITE-3716: ODBC: Added SQL escape sequence parsing.


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

Branch: refs/heads/master
Commit: 4e9e7b8ee1c990bacdc2d081b706ca315927fdce
Parents: f925873
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 24 12:12:00 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 24 12:12:00 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcNioListener.java        |   2 +-
 .../processors/odbc/OdbcRequestHandler.java     |  36 ++-
 .../odbc/escape/OdbcEscapeParseResult.java      |  73 +++++
 .../processors/odbc/escape/OdbcEscapeType.java  |  26 ++
 .../processors/odbc/escape/OdbcEscapeUtils.java | 263 +++++++++++++++++++
 .../odbc/OdbcEscapeSequenceSelfTest.java        | 184 +++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 7 files changed, 575 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4e9e7b8e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
index f720096..e7baaff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcNioListener.java
@@ -121,7 +121,7 @@ public class OdbcNioListener extends GridNioServerListenerAdapter<byte[]> {
 
             OdbcRequestHandler handler = connData.getHandler();
 
-            OdbcResponse resp = handler.handle(req);
+            OdbcResponse resp = handler.handle(reqId, req);
 
             if (log.isDebugEnabled()) {
                 long dur = (System.nanoTime() - startTime) / 1000;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e9e7b8e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index 43a1fa4..ce98720 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@ -18,10 +18,12 @@
 package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.IgniteCache;
+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.processors.cache.QueryCursorImpl;
+import org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeUtils;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
@@ -45,6 +47,9 @@ public class OdbcRequestHandler {
     /** Kernel context. */
     private final GridKernalContext ctx;
 
+    /** Logger. */
+    private final IgniteLogger log;
+
     /** Busy lock. */
     private final GridSpinBusyLock busyLock;
 
@@ -65,15 +70,18 @@ public class OdbcRequestHandler {
         this.ctx = ctx;
         this.busyLock = busyLock;
         this.maxCursors = maxCursors;
+
+        log = ctx.log(OdbcRequestHandler.class);
     }
 
     /**
      * Handle request.
      *
+     * @param reqId Request ID.
      * @param req Request.
      * @return Response.
      */
-    public OdbcResponse handle(OdbcRequest req) {
+    public OdbcResponse handle(long reqId, OdbcRequest req) {
         assert req != null;
 
         if (!busyLock.enterBusy())
@@ -83,22 +91,22 @@ public class OdbcRequestHandler {
         try {
             switch (req.command()) {
                 case HANDSHAKE:
-                    return performHandshake((OdbcHandshakeRequest) req);
+                    return performHandshake(reqId, (OdbcHandshakeRequest)req);
 
                 case EXECUTE_SQL_QUERY:
-                    return executeQuery((OdbcQueryExecuteRequest) req);
+                    return executeQuery(reqId, (OdbcQueryExecuteRequest)req);
 
                 case FETCH_SQL_QUERY:
-                    return fetchQuery((OdbcQueryFetchRequest) req);
+                    return fetchQuery((OdbcQueryFetchRequest)req);
 
                 case CLOSE_SQL_QUERY:
-                    return closeQuery((OdbcQueryCloseRequest) req);
+                    return closeQuery((OdbcQueryCloseRequest)req);
 
                 case GET_COLUMNS_META:
-                    return getColumnsMeta((OdbcQueryGetColumnsMetaRequest) req);
+                    return getColumnsMeta((OdbcQueryGetColumnsMetaRequest)req);
 
                 case GET_TABLES_META:
-                    return getTablesMeta((OdbcQueryGetTablesMetaRequest) req);
+                    return getTablesMeta((OdbcQueryGetTablesMetaRequest)req);
             }
 
             return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Unsupported ODBC request: " + req);
@@ -111,10 +119,11 @@ public class OdbcRequestHandler {
     /**
      * {@link OdbcHandshakeRequest} command handler.
      *
+     * @param reqId Request ID.
      * @param req Handshake request.
      * @return Response.
      */
-    private OdbcResponse performHandshake(OdbcHandshakeRequest req) {
+    private OdbcResponse performHandshake(long reqId, OdbcHandshakeRequest req) {
         OdbcHandshakeResult res;
 
         if (req.version() == OdbcMessageParser.PROTO_VER)
@@ -133,10 +142,11 @@ public class OdbcRequestHandler {
     /**
      * {@link OdbcQueryExecuteRequest} command handler.
      *
+     * @param reqId Request ID.
      * @param req Execute query request.
      * @return Response.
      */
-    private OdbcResponse executeQuery(OdbcQueryExecuteRequest req) {
+    private OdbcResponse executeQuery(long reqId, OdbcQueryExecuteRequest req) {
         int cursorCnt = qryCursors.size();
 
         if (maxCursors > 0 && cursorCnt >= maxCursors)
@@ -147,7 +157,13 @@ public class OdbcRequestHandler {
         long qryId = QRY_ID_GEN.getAndIncrement();
 
         try {
-            SqlFieldsQuery qry = new SqlFieldsQuery(req.sqlQuery());
+            String sql = OdbcEscapeUtils.parse(req.sqlQuery());
+
+            if (log.isDebugEnabled())
+                log.debug("ODBC query parsed [reqId=" + reqId + ", original=" + req.sqlQuery() +
+                    ", parsed=" + sql + ']');
+
+            SqlFieldsQuery qry = new SqlFieldsQuery(sql);
 
             qry.setArgs(req.arguments());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e9e7b8e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.java
new file mode 100644
index 0000000..cf05651
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeParseResult.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.escape;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * ODBC escape sequence parse result.
+ */
+public class OdbcEscapeParseResult {
+    /** Original start position. */
+    private final int originalStart;
+
+    /** Original length. */
+    private final int originalLen;
+
+    /** Resulting text. */
+    private final String res;
+
+    /**
+     * Constructor.
+     *
+     * @param originalStart Original start position.
+     * @param originalLen Original length.
+     * @param res Resulting text.
+     */
+    public OdbcEscapeParseResult(int originalStart, int originalLen, String res) {
+        this.originalStart = originalStart;
+        this.originalLen = originalLen;
+        this.res = res;
+    }
+
+    /**
+     * @return Original start position.
+     */
+    public int originalStart() {
+        return originalStart;
+    }
+
+    /**
+     * @return Original length.
+     */
+    public int originalLength() {
+        return originalLen;
+    }
+
+    /**
+     * @return Resulting text.
+     */
+    public String result() {
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(OdbcEscapeParseResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e9e7b8e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
new file mode 100644
index 0000000..2df413f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
@@ -0,0 +1,26 @@
+/*
+ * 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.escape;
+
+/**
+ * ODBC escape sequence type.
+ */
+public enum OdbcEscapeType {
+    /** Scalar function. */
+    FN
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e9e7b8e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
new file mode 100644
index 0000000..4d8ca69
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
@@ -0,0 +1,263 @@
+/*
+ * 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.escape;
+
+import org.apache.ignite.IgniteException;
+
+import java.util.LinkedList;
+
+/**
+ * ODBC escape sequence parse.
+ */
+public class OdbcEscapeUtils {
+    /**
+     * Parse escape sequence.
+     *
+     * @param text Original text.
+     * @return Result.
+     */
+    public static String parse(String text) {
+        if (text == null)
+            throw new IgniteException("Text cannot be null.");
+
+        return parse0(text.trim(), 0, false).result();
+    }
+
+    /**
+     * Internal parse routine.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param earlyExit When set to {@code true} we must return as soon as single expression is parsed.
+     * @return Parse result.
+     */
+    private static OdbcEscapeParseResult parse0(String text, int startPos, boolean earlyExit) {
+        StringBuilder res = new StringBuilder();
+
+        int curPos = startPos;
+
+        int plainPos = startPos;
+        int openPos = -1;
+
+        LinkedList<OdbcEscapeParseResult> nested = null;
+
+        while (curPos < text.length()) {
+            char curChar = text.charAt(curPos);
+
+            if (curChar == '{') {
+                if (openPos == -1) {
+                    // Top-level opening brace. Append previous portion and remember current position.
+                    res.append(text, plainPos, curPos);
+
+                    openPos = curPos;
+                }
+                else {
+                    // Nested opening brace -> perform recursion.
+                    OdbcEscapeParseResult nestedRes = parse0(text, curPos, true);
+
+                    if (nested == null)
+                        nested = new LinkedList<>();
+
+                    nested.add(nestedRes);
+
+                    curPos += nestedRes.originalLength() - 1;
+
+                    plainPos = curPos + 1;
+                }
+            }
+            else if (curChar == '}') {
+                if (openPos == -1)
+                    // Close without open -> exception.
+                    throw new IgniteException("Malformed escape sequence " +
+                        "(closing curly brace without opening curly brace): " + text);
+                else {
+                    String parseRes;
+
+                    if (nested == null)
+                        // Found sequence without nesting, process it.
+                        parseRes = parseExpression(text, openPos, curPos - openPos);
+                    else {
+                        // Special case to process nesting.
+                        String res0 = appendNested(text, openPos, curPos + 1, nested);
+
+                        nested = null;
+
+                        parseRes = parseExpression(res0, 0, res0.length()-1);
+                    }
+
+                    if (earlyExit)
+                        return new OdbcEscapeParseResult(startPos, curPos - startPos + 1, parseRes);
+                    else
+                        res.append(parseRes);
+
+                    openPos = -1;
+
+                    plainPos = curPos + 1;
+                }
+            }
+
+            curPos++;
+        }
+
+        if (openPos != -1)
+            throw new IgniteException("Malformed escape sequence (closing curly brace missing): " + text);
+
+        if (curPos > plainPos)
+            res.append(text, plainPos, curPos);
+
+        return new OdbcEscapeParseResult(startPos, curPos - startPos + 1, res.toString());
+    }
+
+    /**
+     * Parse concrete expression.
+     *
+     * @param text Text.
+     * @param startPos Start position within text.
+     * @param len Length.
+     * @return Result.
+     */
+    private static String parseExpression(String text, int startPos, int len) {
+        assert validSubstring(text, startPos, len);
+
+        char firstChar = text.charAt(startPos);
+
+        if (firstChar == '{') {
+            char lastChar = text.charAt(startPos + len);
+
+            if (lastChar != '}')
+                throw new IgniteException("Failed to parse escape sequence because it is not enclosed: " +
+                    substring(text, startPos, len));
+
+            OdbcEscapeType typ = sequenceType(text, startPos, len);
+
+            switch (typ) {
+                case FN:
+                    return parseScalarExpression(text, startPos, len);
+
+                default: {
+                    assert false : "Unknown expression type: " + typ;
+
+                    return null;
+                }
+            }
+        }
+        else {
+            // Nothing to escape, return original string.
+            if (startPos == 0 || text.length() == len)
+                return text;
+            else
+                return text.substring(startPos, startPos + len);
+        }
+    }
+
+    /**
+     * Parse concrete expression.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Parsed expression.
+     */
+    private static String parseScalarExpression(String text, int startPos, int len) {
+        assert validSubstring(text, startPos, len);
+
+        return substring(text, startPos + 3, len - 3).trim();
+    }
+
+    /**
+     * Append nested results.
+     *
+     * @param text Original text.
+     * @param startPos Start position.
+     * @param endPos End position.
+     * @param nestedRess Nested results.
+     * @return Result.
+     */
+    private static String appendNested(String text, int startPos, int endPos,
+        LinkedList<OdbcEscapeParseResult> nestedRess) {
+        StringBuilder res = new StringBuilder();
+
+        int curPos = startPos;
+
+        for (OdbcEscapeParseResult nestedRes : nestedRess) {
+            // Append text between current position and replace.
+            res.append(text, curPos, nestedRes.originalStart());
+
+            // Append replaced text.
+            res.append(nestedRes.result());
+
+            // Advance position.
+            curPos = nestedRes.originalStart() + nestedRes.originalLength();
+        }
+
+        // Append remainder.
+        res.append(text, curPos, endPos);
+
+        return res.toString();
+    }
+
+    /**
+     * Get escape sequence type.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @return Escape sequence type.
+     */
+    private static OdbcEscapeType sequenceType(String text, int startPos, int len) {
+        assert validSubstring(text, startPos, len);
+        assert text.charAt(startPos) == '{';
+
+        if (text.startsWith("fn", startPos + 1))
+            return OdbcEscapeType.FN;
+
+        throw new IgniteException("Unsupported escape sequence: " + text.substring(startPos, startPos + len));
+    }
+
+    /**
+     * Perform "substring" using start position and length.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Substring.
+     */
+    private static String substring(String text, int startPos, int len) {
+        assert validSubstring(text, startPos, len);
+
+        return text.substring(startPos, startPos + len);
+    }
+
+    /**
+     * Check whether substring is valid.
+     *
+     * @param text Substring.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return {@code True} if valid.
+     */
+    private static boolean validSubstring(String text, int startPos, int len) {
+        return text != null && startPos + len <= text.length();
+    }
+
+    /**
+     * Private constructor.
+     */
+    private OdbcEscapeUtils() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e9e7b8e/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
new file mode 100644
index 0000000..73fa0f4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeUtils;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import java.util.concurrent.Callable;
+
+/**
+ * Scalar function escape sequence parser tests.
+ */
+public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
+    /**
+     * Test simple cases.
+     */
+    public void testSimple() {
+        check(
+            "select * from table;",
+            "select * from table;"
+        );
+
+        check(
+            "test()",
+            "{fn test()}"
+        );
+
+        check(
+            "select test()",
+            "select {fn test()}"
+        );
+
+        check(
+            "select test() from table;",
+            "select {fn test()} from table;"
+        );
+    }
+
+    /**
+     * Test escape sequence series.
+     */
+    public void testSimpleFunction() throws Exception {
+        check(
+            "func(field1) func(field2)",
+            "{fn func(field1)} {fn func(field2)}"
+        );
+
+        check(
+            "select func(field1), func(field2)",
+            "select {fn func(field1)}, {fn func(field2)}"
+        );
+
+        check(
+            "select func(field1), func(field2) from table;",
+            "select {fn func(field1)}, {fn func(field2)} from table;"
+        );
+    }
+
+    /**
+     * Test simple nested escape sequences. Depth = 2.
+     */
+    public void testNestedFunction() throws Exception {
+        check(
+            "func1(field1, func2(field2))",
+            "{fn func1(field1, {fn func2(field2)})}"
+        );
+
+        check(
+            "select func1(field1, func2(field2))",
+            "select {fn func1(field1, {fn func2(field2)})}"
+        );
+
+        check(
+            "select func1(field1, func2(field2), field3) from SomeTable;",
+            "select {fn func1(field1, {fn func2(field2)}, field3)} from SomeTable;"
+        );
+    }
+
+    /**
+     * Test nested escape sequences. Depth > 2.
+     */
+    public void testDeepNestedFunction() {
+        check(
+            "func1(func2(func3(field1)))",
+            "{fn func1({fn func2({fn func3(field1)})})}"
+        );
+
+        check(
+            "func1(func2(func3(func4(field1))))",
+            "{fn func1({fn func2({fn func3({fn func4(field1)})})})}"
+        );
+
+        check(
+            "select func1(field1, func2(func3(field2), field3))",
+            "select {fn func1(field1, {fn func2({fn func3(field2)}, field3)})}"
+        );
+
+        check(
+            "select func1(field1, func2(func3(field2), field3)) from SomeTable;",
+            "select {fn func1(field1, {fn func2({fn func3(field2)}, field3)})} from SomeTable;"
+        );
+    }
+
+    /**
+     * Test series of nested escape sequences.
+     */
+    public void testNestedFunctionMixed() {
+        check(
+            "func1(func2(field1), func3(field2))",
+            "{fn func1({fn func2(field1)}, {fn func3(field2)})}"
+        );
+
+        check(
+            "select func1(func2(field1), func3(field2)) from table;",
+            "select {fn func1({fn func2(field1)}, {fn func3(field2)})} from table;"
+        );
+
+        check(
+            "func1(func2(func3(field1))) func1(func2(field2))",
+            "{fn func1({fn func2({fn func3(field1)})})} {fn func1({fn func2(field2)})}"
+        );
+    }
+
+    /**
+     * Test non-closed escape sequence.
+     */
+    public void testFailedOnInvalidSequence1() {
+        checkFail("select {fn func1(field1, {fn func2(field2), field3)} from SomeTable;");
+    }
+
+    /**
+     * Test closing undeclared escape sequence.
+     */
+    public void testFailedOnClosingNotOpenedSequence() {
+        checkFail("select {fn func1(field1, func2(field2)}, field3)} from SomeTable;");
+    }
+
+    /**
+     * Check parsing logic.
+     *
+     * @param exp Expected result.
+     * @param qry SQL query text.
+     */
+    private void check(String exp, String qry) {
+        String actualRes = OdbcEscapeUtils.parse(qry);
+
+        assertEquals(exp, actualRes);
+    }
+
+    /**
+     * Check that query parsing fails.
+     *
+     * @param qry Query.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    private void checkFail(final String qry) {
+        GridTestUtils.assertThrows(null, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                OdbcEscapeUtils.parse(qry);
+
+                fail("Parsing should fail: " + qry);
+
+                return null;
+            }
+        }, IgniteException.class, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e9e7b8e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index d56c29d..6bb2c11 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.closure.GridClosureProcessorSelfTes
 import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest;
 import org.apache.ignite.internal.processors.continuous.GridMessageListenSelfTest;
 import org.apache.ignite.internal.processors.odbc.OdbcProcessorValidationSelfTest;
+import org.apache.ignite.internal.processors.odbc.OdbcEscapeSequenceSelfTest;
 import org.apache.ignite.internal.processors.service.ClosureServiceClientsNodesTest;
 import org.apache.ignite.internal.product.GridProductVersionSelfTest;
 import org.apache.ignite.internal.util.nio.IgniteExceptionInNioWorkerSelfTest;
@@ -128,6 +129,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);
 
         suite.addTestSuite(OdbcProcessorValidationSelfTest.class);
+        suite.addTestSuite(OdbcEscapeSequenceSelfTest.class);
 
         GridTestUtils.addTestIfNeeded(suite, DynamicProxySerializationMultiJvmSelfTest.class, ignoredTests);
 


[22/24] ignite git commit: Merged ignite-1.6.7 to ignite-1.7.2

Posted by sb...@apache.org.
Merged ignite-1.6.7 to ignite-1.7.2


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

Branch: refs/heads/master
Commit: fea3eeaac839381ec3662275a82f4d12af6f12dd
Parents: 60afa37 0465874
Author: agura <ag...@gridgain.com>
Authored: Mon Aug 29 18:38:35 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Mon Aug 29 18:38:35 2016 +0300

----------------------------------------------------------------------
 .../local/LocalIgfsSecondaryFileSystem.java     |   38 +-
 .../processors/cache/GridCacheMapEntry.java     |   19 +-
 .../GridCachePartitionExchangeManager.java      |    7 +
 .../GridDistributedTxPrepareRequest.java        |    4 +-
 .../distributed/dht/GridDhtLockFuture.java      |   53 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |    4 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   26 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  109 +-
 .../dht/GridDhtTxPrepareRequest.java            |    4 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |   69 ++
 .../GridNearAtomicSingleUpdateFuture.java       |   70 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   68 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   37 +-
 .../distributed/near/GridNearLockFuture.java    |   90 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   13 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  263 +++--
 ...ridNearOptimisticTxPrepareFutureAdapter.java |    5 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    8 +-
 .../near/GridNearTxFinishFuture.java            |    5 +-
 .../cache/distributed/near/GridNearTxLocal.java |   16 +-
 .../near/GridNearTxPrepareRequest.java          |    4 +-
 .../cache/transactions/IgniteInternalTx.java    |    3 +-
 .../cache/transactions/IgniteTxAdapter.java     |   37 +-
 .../cache/transactions/IgniteTxHandler.java     |    9 +-
 .../transactions/IgniteTxLocalAdapter.java      |   19 +-
 .../cache/transactions/IgniteTxManager.java     |   86 +-
 .../cache/transactions/IgniteTxStateImpl.java   |   11 +-
 .../cache/transactions/TxDeadlockDetection.java |   51 +-
 .../processors/odbc/OdbcNioListener.java        |    2 +-
 .../processors/odbc/OdbcRequestHandler.java     |   36 +-
 .../odbc/escape/OdbcEscapeParseResult.java      |   73 ++
 .../processors/odbc/escape/OdbcEscapeToken.java |   61 +
 .../processors/odbc/escape/OdbcEscapeType.java  |  105 ++
 .../processors/odbc/escape/OdbcEscapeUtils.java |  361 ++++++
 .../service/GridServiceProcessor.java           |   18 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    7 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |   91 +-
 .../IgniteTxTimeoutAbstractTest.java            |    8 +-
 ...tionedMultiNodeLongTxTimeoutFullApiTest.java |   34 +
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |   41 +
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |    5 +-
 .../transactions/DepthFirstSearchTest.java      |  100 +-
 .../TxDeadlockDetectionNoHangsTest.java         |  246 ++++
 .../transactions/TxDeadlockDetectionTest.java   |   13 +-
 ...timisticDeadlockDetectionCrossCacheTest.java |  257 +++++
 .../TxOptimisticDeadlockDetectionTest.java      |  574 ++++++++++
 ...simisticDeadlockDetectionCrossCacheTest.java |  165 ++-
 .../TxPessimisticDeadlockDetectionTest.java     |   50 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          | 1067 ++++++++++++++++++
 .../processors/igfs/IgfsAbstractSelfTest.java   | 1012 +----------------
 ...SecondaryFileSystemDualAbstractSelfTest.java |  143 +++
 .../odbc/OdbcEscapeSequenceSelfTest.java        |  420 +++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +
 .../IgniteCacheFullApiSelfTestSuite.java        |    4 +
 .../TxDeadlockDetectionTestSuite.java           |    6 +
 .../cpp/common/src/common/big_integer.cpp       |    1 -
 modules/platforms/cpp/odbc-test/Makefile.am     |    9 +
 .../platforms/cpp/odbc-test/include/Makefile.am |    4 +-
 .../odbc-test/include/sql_test_suite_fixture.h  |  191 ++++
 .../cpp/odbc-test/include/test_utils.h          |   45 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   11 +
 .../project/vs/odbc-test.vcxproj.filters        |   33 +
 .../cpp/odbc-test/src/queries_test.cpp          |   78 +-
 .../src/sql_aggregate_functions_test.cpp        |  249 ++++
 .../src/sql_numeric_functions_test.cpp          |  309 +++++
 .../cpp/odbc-test/src/sql_operators_test.cpp    |  214 ++++
 .../odbc-test/src/sql_string_functions_test.cpp |  291 +++++
 .../odbc-test/src/sql_system_functions_test.cpp |   47 +
 .../odbc-test/src/sql_test_suite_fixture.cpp    |  271 +++++
 .../cpp/odbc-test/src/sql_types_test.cpp        |   60 +
 .../src/sql_value_expressions_test.cpp          |   94 ++
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |   36 +
 .../cpp/odbc/src/config/connection_info.cpp     |   57 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |    6 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |   14 +-
 .../commands/cache/VisorCacheStopCommand.scala  |    5 +-
 76 files changed, 6455 insertions(+), 1599 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index f0af551,e67e60f..620d9ae
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@@ -696,9 -666,10 +695,13 @@@ public class IgniteTxHandler 
          assert nodeId != null;
          assert req != null;
  
 +        if (locTx != null)
 +            req.txState(locTx.txState());
 +
+         // 'baseVersion' message field is re-used for version to be added in completed versions.
+         if (!req.commit() && req.baseVersion() != null)
+             ctx.tm().addRolledbackTx(null, req.baseVersion());
+ 
          // Transaction on local cache only.
          if (locTx != null && !locTx.nearLocallyMapped() && !locTx.colocatedLocallyMapped())
              return new GridFinishedFuture<IgniteInternalTx>(locTx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index 8f2d092,ce98720..3805d32
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@@ -120,10 -123,12 +129,10 @@@ public class OdbcRequestHandler 
       * @param req Handshake request.
       * @return Response.
       */
-     private OdbcResponse performHandshake(OdbcHandshakeRequest req) {
+     private OdbcResponse performHandshake(long reqId, OdbcHandshakeRequest req) {
 -        OdbcHandshakeResult res;
 +        OdbcProtocolVersion version = req.version();
  
 -        if (req.version() == OdbcMessageParser.PROTO_VER)
 -            res = new OdbcHandshakeResult(true, null, null);
 -        else {
 +        if (version.isUnknown()) {
              IgniteProductVersion ver = ctx.grid().version();
  
              String verStr = Byte.toString(ver.major()) + '.' + ver.minor() + '.' + ver.maintenance();

http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index aead2af,84f5a29..0a72640
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@@ -76,9 -76,33 +76,36 @@@
      <ClCompile Include="..\..\src\queries_test.cpp">
        <Filter>Code</Filter>
      </ClCompile>
 +    <ClCompile Include="..\..\..\odbc\src\protocol_version.cpp">
 +      <Filter>Externals</Filter>
 +    </ClCompile>
+     <ClCompile Include="..\..\src\sql_string_functions_test.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
+     <ClCompile Include="..\..\src\test_utils.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
+     <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
+     <ClCompile Include="..\..\src\sql_aggregate_functions_test.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
+     <ClCompile Include="..\..\src\sql_system_functions_test.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
+     <ClCompile Include="..\..\src\sql_operators_test.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
+     <ClCompile Include="..\..\src\sql_types_test.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
+     <ClCompile Include="..\..\src\sql_test_suite_fixture.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
+     <ClCompile Include="..\..\src\sql_value_expressions_test.cpp">
+       <Filter>Code</Filter>
+     </ClCompile>
    </ItemGroup>
    <ItemGroup>
      <ClInclude Include="..\..\include\test_type.h">

http://git-wip-us.apache.org/repos/asf/ignite/blob/fea3eeaa/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------
diff --cc modules/platforms/cpp/odbc-test/src/queries_test.cpp
index 58b5f64,2d9bd58..ab59952
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@@ -77,12 -53,41 +54,41 @@@ using ignite::impl::binary::BinaryUtils
  struct QueriesTestSuiteFixture 
  {
      /**
 -     * Constructor.
 +     * Establish connection to node.
 +     *
 +     * @param connectStr Connection string.
       */
 -    QueriesTestSuiteFixture() : testCache(0), env(NULL), dbc(NULL), stmt(NULL)
 +    void Connect(const std::string& connectStr)
      {
+         IgniteConfiguration cfg;
+ 
+         cfg.jvmOpts.push_back("-Xdebug");
+         cfg.jvmOpts.push_back("-Xnoagent");
+         cfg.jvmOpts.push_back("-Djava.compiler=NONE");
+         cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
+         cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
+ 
+ #ifdef IGNITE_TESTS_32
+         cfg.jvmInitMem = 256;
+         cfg.jvmMaxMem = 768;
+ #else
+         cfg.jvmInitMem = 1024;
+         cfg.jvmMaxMem = 4096;
+ #endif
+ 
+         char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
+ 
 -        BOOST_REQUIRE(cfgPath != 0);
 -
 -        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
++        cfg.springCfgPath = std::string(cfgPath).append("/").append("queries-test.xml");
+ 
+         IgniteError err;
+ 
+         grid = Ignition::Start(cfg, &err);
+ 
+         if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+             BOOST_FAIL(err.GetText());
+ 
+         testCache = grid.GetCache<int64_t, TestType>("cache");
+ 
          // Allocate an environment handle
          SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
  
@@@ -133,71 -138,8 +139,79 @@@
          // Releasing allocated handles.
          SQLFreeHandle(SQL_HANDLE_DBC, dbc);
          SQLFreeHandle(SQL_HANDLE_ENV, env);
 +    }
  
 -        Ignition::Stop(grid.GetName(), true);
 +    static Ignite StartAdditionalNode(const char* name)
 +    {
 +        IgniteConfiguration cfg;
 +
 +        cfg.jvmOpts.push_back("-Xdebug");
 +        cfg.jvmOpts.push_back("-Xnoagent");
 +        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
 +        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
 +        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
 +
 +#ifdef IGNITE_TESTS_32
 +        cfg.jvmInitMem = 256;
 +        cfg.jvmMaxMem = 768;
 +#else
 +        cfg.jvmInitMem = 1024;
 +        cfg.jvmMaxMem = 4096;
 +#endif
 +
-         cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH")).append("/queries-test-noodbc.xml");
++        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
++
++        BOOST_REQUIRE(cfgPath != 0);
++
++        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
 +
 +        IgniteError err;
 +
 +        return Ignition::Start(cfg, name);
 +    }
 +
 +    /**
 +     * Constructor.
 +     */
 +    QueriesTestSuiteFixture() : testCache(0), env(NULL), dbc(NULL), stmt(NULL)
 +    {
 +        IgniteConfiguration cfg;
 +
 +        cfg.jvmOpts.push_back("-Xdebug");
 +        cfg.jvmOpts.push_back("-Xnoagent");
 +        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
 +        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
 +        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
 +
 +#ifdef IGNITE_TESTS_32
 +        cfg.jvmInitMem = 256;
 +        cfg.jvmMaxMem = 768;
 +#else
 +        cfg.jvmInitMem = 1024;
 +        cfg.jvmMaxMem = 4096;
 +#endif
 +
-         cfg.springCfgPath.assign(getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH")).append("/queries-test.xml");
++        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
++
++        BOOST_REQUIRE(cfgPath != 0);
++
++        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
 +
 +        IgniteError err;
 +
 +        grid = Ignition::Start(cfg, "NodeMain");
 +
 +        testCache = grid.GetCache<int64_t, TestType>("cache");
 +    }
 +
 +    /**
 +     * Destructor.
 +     */
 +    ~QueriesTestSuiteFixture()
 +    {
 +        Disconnect();
 +
 +        Ignition::StopAll(true);
      }
  
      template<typename T>


[23/24] ignite git commit: Merge ignite-1.7.2 to master

Posted by sb...@apache.org.
Merge ignite-1.7.2 to master


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

Branch: refs/heads/master
Commit: f89375bdd6159134fae5d0b7b28b876b6e335a32
Parents: 2d5d5bc fea3eea
Author: agura <ag...@gridgain.com>
Authored: Mon Aug 29 20:35:08 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Mon Aug 29 20:35:08 2016 +0300

----------------------------------------------------------------------
 .../local/LocalIgfsSecondaryFileSystem.java     |   38 +-
 .../processors/cache/GridCacheMapEntry.java     |   19 +-
 .../GridCachePartitionExchangeManager.java      |    7 +
 .../GridDistributedTxPrepareRequest.java        |    4 +-
 .../distributed/dht/GridDhtLockFuture.java      |   53 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |    4 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   26 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  101 +-
 .../dht/GridDhtTxPrepareRequest.java            |    4 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |   69 ++
 .../GridNearAtomicSingleUpdateFuture.java       |   70 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   68 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   37 +-
 .../distributed/near/GridNearLockFuture.java    |   90 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   13 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  263 +++--
 ...ridNearOptimisticTxPrepareFutureAdapter.java |    5 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    8 +-
 .../near/GridNearTxFinishFuture.java            |    5 +-
 .../cache/distributed/near/GridNearTxLocal.java |   16 +-
 .../near/GridNearTxPrepareRequest.java          |    4 +-
 .../cache/transactions/IgniteInternalTx.java    |    3 +-
 .../cache/transactions/IgniteTxAdapter.java     |   37 +-
 .../cache/transactions/IgniteTxHandler.java     |   11 +-
 .../transactions/IgniteTxLocalAdapter.java      |   19 +-
 .../cache/transactions/IgniteTxManager.java     |   86 +-
 .../cache/transactions/IgniteTxStateImpl.java   |   11 +-
 .../cache/transactions/TxDeadlockDetection.java |   51 +-
 .../processors/odbc/OdbcNioListener.java        |    2 +-
 .../processors/odbc/OdbcRequestHandler.java     |   36 +-
 .../odbc/escape/OdbcEscapeParseResult.java      |   73 ++
 .../processors/odbc/escape/OdbcEscapeToken.java |   61 +
 .../processors/odbc/escape/OdbcEscapeType.java  |  105 ++
 .../processors/odbc/escape/OdbcEscapeUtils.java |  361 ++++++
 .../service/GridServiceProcessor.java           |   18 +-
 .../util/future/GridCompoundFuture.java         |   10 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    7 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |   91 +-
 .../IgniteTxTimeoutAbstractTest.java            |    8 +-
 ...tionedMultiNodeLongTxTimeoutFullApiTest.java |   34 +
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |   41 +
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |    5 +-
 .../transactions/DepthFirstSearchTest.java      |  100 +-
 .../TxDeadlockDetectionNoHangsTest.java         |  246 ++++
 .../transactions/TxDeadlockDetectionTest.java   |   13 +-
 ...timisticDeadlockDetectionCrossCacheTest.java |  257 +++++
 .../TxOptimisticDeadlockDetectionTest.java      |  574 ++++++++++
 ...simisticDeadlockDetectionCrossCacheTest.java |  165 ++-
 .../TxPessimisticDeadlockDetectionTest.java     |   50 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          | 1067 ++++++++++++++++++
 .../processors/igfs/IgfsAbstractSelfTest.java   | 1012 +----------------
 ...SecondaryFileSystemDualAbstractSelfTest.java |  143 +++
 .../odbc/OdbcEscapeSequenceSelfTest.java        |  420 +++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +
 .../IgniteCacheFullApiSelfTestSuite.java        |    4 +
 .../TxDeadlockDetectionTestSuite.java           |    6 +
 .../cpp/common/include/ignite/common/utils.h    |    7 +
 .../cpp/common/os/win/src/common/utils.cpp      |   20 +
 .../cpp/common/src/common/big_integer.cpp       |    1 -
 .../cpp/core/include/ignite/ignition.h          |    2 +-
 modules/platforms/cpp/core/src/ignition.cpp     |    8 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |    9 +
 .../platforms/cpp/odbc-test/include/Makefile.am |    4 +-
 .../odbc-test/include/sql_test_suite_fixture.h  |  191 ++++
 .../cpp/odbc-test/include/test_utils.h          |   45 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   11 +
 .../project/vs/odbc-test.vcxproj.filters        |   33 +
 .../cpp/odbc-test/src/configuration_test.cpp    |    6 +-
 .../cpp/odbc-test/src/queries_test.cpp          |   78 +-
 .../src/sql_aggregate_functions_test.cpp        |  249 ++++
 .../src/sql_numeric_functions_test.cpp          |  309 +++++
 .../cpp/odbc-test/src/sql_operators_test.cpp    |  214 ++++
 .../odbc-test/src/sql_string_functions_test.cpp |  291 +++++
 .../odbc-test/src/sql_system_functions_test.cpp |   47 +
 .../odbc-test/src/sql_test_suite_fixture.cpp    |  271 +++++
 .../cpp/odbc-test/src/sql_types_test.cpp        |   60 +
 .../src/sql_value_expressions_test.cpp          |   94 ++
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |   36 +
 modules/platforms/cpp/odbc/Makefile.am          |    1 +
 modules/platforms/cpp/odbc/include/Makefile.am  |    1 +
 .../platforms/cpp/odbc/include/ignite/odbc.h    |    1 -
 .../include/ignite/odbc/config/configuration.h  |   93 +-
 .../ignite/odbc/diagnostic/diagnostic_record.h  |    2 +-
 .../cpp/odbc/include/ignite/odbc/dsn_config.h   |   61 +
 .../odbc/include/ignite/odbc/protocol_version.h |   20 +-
 .../include/ignite/odbc/system/odbc_constants.h |    4 -
 .../odbc/system/ui/dsn_configuration_window.h   |  152 +++
 .../ignite/odbc/system/ui/custom_window.h       |  189 ++++
 .../win/include/ignite/odbc/system/ui/window.h  |  201 ++++
 .../odbc/os/win/src/system/ui/custom_window.cpp |  184 +++
 .../src/system/ui/dsn_configuration_window.cpp  |  305 +++++
 .../cpp/odbc/os/win/src/system/ui/window.cpp    |  192 ++++
 .../cpp/odbc/os/win/src/system_dsn.cpp          |  218 ++++
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   25 +-
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   30 +
 .../cpp/odbc/src/config/configuration.cpp       |   17 +-
 .../cpp/odbc/src/config/connection_info.cpp     |   57 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |    2 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |    2 +-
 .../diagnostic/diagnostic_record_storage.cpp    |    2 +-
 modules/platforms/cpp/odbc/src/dsn_config.cpp   |  115 ++
 modules/platforms/cpp/odbc/src/entry_points.cpp |    8 -
 modules/platforms/cpp/odbc/src/odbc.cpp         |   88 +-
 .../platforms/cpp/odbc/src/protocol_version.cpp |   23 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |    6 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |   14 +-
 .../commands/cache/VisorCacheStopCommand.scala  |    5 +-
 107 files changed, 8330 insertions(+), 1707 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 187c8a4,b005b29..f2b5f49
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@@ -298,8 -292,10 +292,10 @@@ public final class GridDhtLockFuture ex
      /**
       * @return Entries.
       */
-     public synchronized Collection<GridDhtCacheEntry> entriesCopy() {
-         return new ArrayList<>(entries());
+     public Collection<GridDhtCacheEntry> entriesCopy() {
 -        synchronized (futs) {
++        synchronized (sync) {
+             return new ArrayList<>(entries());
+         }
      }
  
      /**
@@@ -412,7 -408,7 +408,7 @@@
              return null;
          }
  
-         synchronized (this) {
 -        synchronized (futs) {
++        synchronized (sync) {
              entries.add(c == null || c.reentry() ? null : entry);
  
              if (c != null && !c.reentry())
@@@ -614,7 -610,7 +610,7 @@@
       * @param t Error.
       */
      public void onError(Throwable t) {
-         synchronized (this) {
 -        synchronized (futs) {
++        synchronized (sync) {
              if (err != null)
                  return;
  
@@@ -661,8 -657,9 +657,9 @@@
              log.debug("Received onOwnerChanged() callback [entry=" + entry + ", owner=" + owner + "]");
  
          if (owner != null && owner.version().equals(lockVer)) {
-             synchronized (this) {
-                 pendingLocks.remove(entry.key());
 -            synchronized (futs) {
++            synchronized (sync) {
+                 if (!pendingLocks.remove(entry.key()))
+                     return false;
              }
  
              if (checkLocks())
@@@ -677,8 -674,10 +674,10 @@@
      /**
       * @return {@code True} if locks have been acquired.
       */
-     private synchronized boolean checkLocks() {
-         return pendingLocks.isEmpty();
+     private boolean checkLocks() {
 -        synchronized (futs) {
++        synchronized (sync) {
+             return pendingLocks.isEmpty();
+         }
      }
  
      /** {@inheritDoc} */
@@@ -709,7 -708,7 +708,7 @@@
          if (isDone() || (err == null && success && !checkLocks()))
              return false;
  
-         synchronized (this) {
 -        synchronized (futs) {
++        synchronized (sync) {
              if (this.err == null)
                  this.err = err;
          }
@@@ -782,7 -787,7 +787,7 @@@
       * @param entries Entries.
       */
      private void map(Iterable<GridDhtCacheEntry> entries) {
-         synchronized (this) {
 -        synchronized (futs) {
++        synchronized (sync) {
              if (mapped)
                  return;
  
@@@ -1109,7 -1119,14 +1119,14 @@@
              if (log.isDebugEnabled())
                  log.debug("Timed out waiting for lock response: " + this);
  
-             timedOut = true;
 -            synchronized (futs) {
++            synchronized (sync) {
+                 timedOut = true;
+ 
+                 // Stop locks and responses processing.
+                 pendingLocks.clear();
+ 
 -                futs.clear();
++                clear();
+             }
  
              boolean releaseLocks = !(inTx() && cctx.tm().deadlockDetectionEnabled());
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 2800897,1bdd9b8..3cb0fbc
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@@ -269,7 -278,7 +278,7 @@@ public final class GridDhtTxPrepareFutu
  
          boolean rmv;
  
-         synchronized (lockKeys) {
 -        synchronized (futs) {
++        synchronized (sync) {
              rmv = lockKeys.remove(entry.txKey());
          }
  
@@@ -300,7 -309,7 +309,7 @@@
          if (!locksReady)
              return false;
  
-         synchronized (lockKeys) {
 -        synchronized (futs) {
++        synchronized (sync) {
              return lockKeys.isEmpty();
          }
      }
@@@ -583,7 -588,7 +588,7 @@@
              }
  
              if (tx.optimistic() && txEntry.explicitVersion() == null) {
-                 synchronized (lockKeys) {
 -                synchronized (futs) {
++                synchronized (sync) {
                      lockKeys.add(txEntry.txKey());
                  }
              }
@@@ -1284,9 -1303,12 +1303,12 @@@
  
                  for (GridDistributedTxMapping nearMapping : tx.nearMap().values()) {
                      if (!tx.dhtMap().containsKey(nearMapping.node().id())) {
+                         if (tx.remainingTime() == -1)
+                             return;
+ 
                          MiniFuture fut = new MiniFuture(nearMapping.node().id(), null, nearMapping);
  
 -                        add(fut);
 +                        add(fut); // Append new future.
  
                          GridDhtTxPrepareRequest req = new GridDhtTxPrepareRequest(
                              futId,
@@@ -1719,4 -1742,38 +1742,38 @@@
              return S.toString(MiniFuture.class, this, "done", isDone(), "cancelled", isCancelled(), "err", error());
          }
      }
+ 
+     /**
+      *
+      */
+     private class PrepareTimeoutObject extends GridTimeoutObjectAdapter {
+         /** */
+         private final long timeout;
+ 
+         /**
+          * @param timeout Timeout.
+          */
+         PrepareTimeoutObject(long timeout) {
+             super(timeout);
+ 
+             this.timeout = timeout;
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public void onTimeout() {
 -            synchronized (futs) {
 -                futs.clear();
++            synchronized (sync) {
++                clear();
+ 
+                 lockKeys.clear();
+             }
+ 
+             onError(new IgniteTxTimeoutCheckedException("Failed to acquire lock within " +
+                 "provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']'));
+         }
+ 
+         /** {@inheritDoc} */
+         @Override public String toString() {
+             return S.toString(PrepareTimeoutObject.class, this);
+         }
+     }
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 05b4a2b,b0eea01..ddb6500
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@@ -443,17 -443,27 +443,27 @@@ public final class GridDhtColocatedLock
      /**
       * @return Keys for which locks requested from remote nodes but response isn't received.
       */
-     public Set<KeyCacheObject> requestedKeys() {
-         Set<KeyCacheObject> requestedKeys = null;
+     public Set<IgniteTxKey> requestedKeys() {
 -        synchronized (futs) {
++        synchronized (sync) {
+             if (timeoutObj != null && timeoutObj.requestedKeys != null)
+                 return timeoutObj.requestedKeys;
+ 
+             return requestedKeys0();
+         }
+     }
  
+     /**
+      * @return Keys for which locks requested from remote nodes but response isn't received.
+      */
+     private Set<IgniteTxKey> requestedKeys0() {
          for (IgniteInternalFuture<Boolean> miniFut : futures()) {
              if (isMini(miniFut) && !miniFut.isDone()) {
-                 if (requestedKeys == null)
-                     requestedKeys = new HashSet<>();
- 
                  MiniFuture mini = (MiniFuture)miniFut;
  
-                 requestedKeys.addAll(mini.keys);
+                 Set<IgniteTxKey> requestedKeys = U.newHashSet(mini.keys.size());
+ 
+                 for (KeyCacheObject key : mini.keys)
+                     requestedKeys.add(new IgniteTxKey(key, cctx.cacheId()));
  
                  return requestedKeys;
              }
@@@ -1318,6 -1331,12 +1331,12 @@@
                  log.debug("Timed out waiting for lock response: " + this);
  
              if (inTx() && cctx.tm().deadlockDetectionEnabled()) {
 -                synchronized (futs) {
++                synchronized (sync) {
+                     requestedKeys = requestedKeys0();
+ 
 -                    futs.clear(); // Stop response processing.
++                    clear(); // Stop response processing.
+                 }
+ 
                  Set<IgniteTxKey> keys = new HashSet<>();
  
                  for (IgniteTxEntry txEntry : tx.allEntries()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 3b53c5e,3d9b6ab..02f6cce
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@@ -481,6 -487,38 +487,38 @@@ public final class GridNearLockFuture e
      }
  
      /**
+      * @return Keys for which locks requested from remote nodes but response isn't received.
+      */
+     public Set<IgniteTxKey> requestedKeys() {
 -        synchronized (futs) {
++        synchronized (sync) {
+             if (timeoutObj != null && timeoutObj.requestedKeys != null)
+                 return timeoutObj.requestedKeys;
+ 
+             return requestedKeys0();
+         }
+     }
+ 
+     /**
+      * @return Keys for which locks requested from remote nodes but response isn't received.
+      */
+     private Set<IgniteTxKey> requestedKeys0() {
+         for (IgniteInternalFuture<Boolean> miniFut : futures()) {
+             if (isMini(miniFut) && !miniFut.isDone()) {
+                 MiniFuture mini = (MiniFuture)miniFut;
+ 
+                 Set<IgniteTxKey> requestedKeys = U.newHashSet(mini.keys.size());
+ 
+                 for (KeyCacheObject key : mini.keys)
+                     requestedKeys.add(new IgniteTxKey(key, cctx.cacheId()));
+ 
+                 return requestedKeys;
+             }
+         }
+ 
+         return null;
+     }
+ 
+     /**
       * Finds pending mini future by the given mini ID.
       *
       * @param miniId Mini ID to find.
@@@ -1381,7 -1426,42 +1426,42 @@@
  
              timedOut = true;
  
-             onComplete(false, true);
+             if (inTx() && cctx.tm().deadlockDetectionEnabled()) {
 -                synchronized (futs) {
++                synchronized (sync) {
+                     requestedKeys = requestedKeys0();
+ 
 -                    futs.clear(); // Stop response processing.
++                    clear(); // Stop response processing.
+                 }
+ 
+                 Set<IgniteTxKey> keys = new HashSet<>();
+ 
+                 for (IgniteTxEntry txEntry : tx.allEntries()) {
+                     if (!txEntry.locked())
+                         keys.add(txEntry.txKey());
+                 }
+ 
+                 IgniteInternalFuture<TxDeadlock> fut = cctx.tm().detectDeadlock(tx, keys);
+ 
+                 fut.listen(new IgniteInClosure<IgniteInternalFuture<TxDeadlock>>() {
+                     @Override public void apply(IgniteInternalFuture<TxDeadlock> fut) {
+                         try {
+                             TxDeadlock deadlock = fut.get();
+ 
+                             if (deadlock != null)
+                                 err = new TransactionDeadlockException(deadlock.toString(cctx.shared()));
+                         }
+                         catch (IgniteCheckedException e) {
+                             err = e;
+ 
+                             U.warn(log, "Failed to detect deadlock.", e);
+                         }
+ 
+                         onComplete(false, true);
+                     }
+                 });
+             }
+             else
+                 onComplete(false, true);
          }
  
          /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 7a49422,5a300ff..0382b15
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@@ -182,6 -198,33 +198,33 @@@ public class GridNearOptimisticTxPrepar
      }
  
      /**
+      * @return Keys for which {@link MiniFuture} isn't completed.
+      */
+     @SuppressWarnings("ForLoopReplaceableByForEach")
+     public Set<IgniteTxKey> requestedKeys() {
 -        synchronized (futs) {
 -            for (int i = 0; i < futs.size(); i++) {
 -                IgniteInternalFuture<GridNearTxPrepareResponse> fut = futs.get(i);
++        synchronized (sync) {
++            for (int i = 0; i < futuresCount(); i++) {
++                IgniteInternalFuture<GridNearTxPrepareResponse> fut = future(i);
+ 
+                 if (isMini(fut) && !fut.isDone()) {
+                     MiniFuture miniFut = (MiniFuture)fut;
+ 
+                     Collection<IgniteTxEntry> entries = miniFut.mapping().entries();
+ 
+                     Set<IgniteTxKey> keys = U.newHashSet(entries.size());
+ 
+                     for (IgniteTxEntry entry : entries)
+                         keys.add(entry.txKey());
+ 
+                     return keys;
+                 }
+             }
+         }
+ 
+         return null;
+     }
+ 
+     /**
       * Finds pending mini future by the given mini ID.
       *
       * @param miniId Mini ID to find.
@@@ -623,6 -674,61 +674,61 @@@
          return cur;
      }
  
+     /**
+      *
+      */
+     @SuppressWarnings("ForLoopReplaceableByForEach")
+     private void onTimeout() {
+         if (cctx.tm().deadlockDetectionEnabled()) {
+             Set<IgniteTxKey> keys = null;
+ 
+             if (keyLockFut != null)
+                 keys = new HashSet<>(keyLockFut.lockKeys);
+             else {
 -                if (futs != null && !futs.isEmpty()) {
 -                    for (int i = 0; i < futs.size(); i++) {
 -                        IgniteInternalFuture<GridNearTxPrepareResponse> fut = futs.get(i);
++                synchronized (sync) {
++                    for (int i = 0; i < futuresCount(); i++) {
++                        IgniteInternalFuture<GridNearTxPrepareResponse> fut = future(i);
+ 
+                         if (isMini(fut) && !fut.isDone()) {
+                             MiniFuture miniFut = (MiniFuture)fut;
+ 
+                             Collection<IgniteTxEntry> entries = miniFut.mapping().entries();
+ 
+                             keys = U.newHashSet(entries.size());
+ 
+                             for (IgniteTxEntry entry : entries)
+                                 keys.add(entry.txKey());
+ 
+                             break;
+                         }
+                     }
+                 }
+             }
+ 
+             add(new GridEmbeddedFuture<>(new IgniteBiClosure<TxDeadlock, Exception, GridNearTxPrepareResponse>() {
+                 @Override public GridNearTxPrepareResponse apply(TxDeadlock deadlock, Exception e) {
+                     if (e != null)
+                         U.warn(log, "Failed to detect deadlock.", e);
+                     else {
+                         e = new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout for " +
+                             "transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']',
+                             deadlock != null ? new TransactionDeadlockException(deadlock.toString(cctx)) : null);
+                     }
+ 
+                     onDone(null, e);
+ 
+                     return null;
+                 }
+             }, cctx.tm().detectDeadlock(tx, keys)));
+         }
+         else {
+             ERR_UPD.compareAndSet(this, null, new IgniteTxTimeoutCheckedException("Failed to acquire lock " +
+                 "within provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']'));
+ 
+             onComplete(false);
+         }
+     }
+ 
      /** {@inheritDoc} */
      @Override public String toString() {
          Collection<String> futs = F.viewReadOnly(futures(), new C1<IgniteInternalFuture<?>, String>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index f0af551,620d9ae..68b884c
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@@ -679,7 -678,7 +678,7 @@@ public class IgniteTxHandler 
  
          IgniteInternalFuture<IgniteInternalTx> fut = finish(nodeId, null, req);
  
--        assert req.txState() != null || fut.error() != null ||
++        assert req.txState() != null || (fut != null && fut.error() != null) ||
              (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null);
  
          return fut;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
index b83133a,3409341..0f7e020
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
@@@ -159,15 -154,9 +159,16 @@@ public class GridCompoundFuture<T, R> e
       *
       * @return Collection of futures.
       */
++    @SuppressWarnings("unchecked")
      public Collection<IgniteInternalFuture<T>> futures() {
 -        synchronized (futs) {
 -            return new ArrayList<>(futs);
 +        synchronized (sync) {
 +            if(futs == null)
 +                return Collections.emptyList();
 +
 +            if (futs instanceof IgniteInternalFuture)
 +                return Collections.singletonList((IgniteInternalFuture<T>)futs);
 +
 +            return new ArrayList<>((Collection<IgniteInternalFuture<T>>)futs);
          }
      }
  
@@@ -240,8 -217,8 +241,17 @@@
      }
  
      /**
 -     * @return {@code True} if this future was initialized. Initialization happens when
 -     *      {@link #markInitialized()} method is called on future.
++     * Clear futures.
++     */
++    protected void clear() {
++        synchronized (sync) {
++            futs = null;
++        }
++    }
++
++    /**
 +     * @return {@code True} if this future was initialized. Initialization happens when {@link #markInitialized()}
 +     * method is called on future.
       */
      public boolean initialized() {
          return initFlag == INIT_FLAG;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f89375bd/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 939e226,6bb2c11..846f6ea
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@@ -127,8 -127,9 +128,9 @@@ public class IgniteBasicTestSuite exten
          suite.addTestSuite(GridNodeMetricsLogSelfTest.class);
  
          suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);
 -
 +        suite.addTestSuite(IgniteLocalNodeMapBeforeStartTest.class);
          suite.addTestSuite(OdbcProcessorValidationSelfTest.class);
+         suite.addTestSuite(OdbcEscapeSequenceSelfTest.class);
  
          GridTestUtils.addTestIfNeeded(suite, DynamicProxySerializationMultiJvmSelfTest.class, ignoredTests);
  


[16/24] ignite git commit: IGNITE-3745: ODBC: Implemented date/time/timestamp escape sequence parsing. This closes #991.

Posted by sb...@apache.org.
IGNITE-3745: ODBC: Implemented date/time/timestamp escape sequence parsing. This closes #991.


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

Branch: refs/heads/master
Commit: 92f18bf353cc8c3821c6500ce9f1cd397a7cf17c
Parents: 87a1928
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Aug 26 15:31:30 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 26 15:31:30 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/escape/OdbcEscapeUtils.java |  30 ++-
 .../odbc/OdbcEscapeSequenceSelfTest.java        | 191 +++++++++++++++++--
 2 files changed, 203 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/92f18bf3/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
index 83ec9d8..4c1deb6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
@@ -26,10 +26,17 @@ import java.util.regex.Pattern;
  * ODBC escape sequence parse.
  */
 public class OdbcEscapeUtils {
+    /** Odbc date regexp pattern: '2016-08-23' */
+    private static final Pattern DATE_PATTERN = Pattern.compile("^'\\d{4}-\\d{2}-\\d{2}'$");
 
-    /**
-     * GUID regexp pattern: '12345678-9abc-def0-1234-123456789abc'
-     */
+    /** Odbc time regexp pattern: '14:33:44' */
+    private static final Pattern TIME_PATTERN = Pattern.compile("^'\\d{2}:\\d{2}:\\d{2}'$");
+
+    /** Odbc timestamp regexp pattern: '2016-08-23 14:33:44.12345' */
+    private static final Pattern TIMESTAMP_PATTERN =
+        Pattern.compile("^'\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2}(\\.\\d+)?'$");
+
+    /** GUID regexp pattern: '12345678-9abc-def0-1234-123456789abc' */
     private static final Pattern GUID_PATTERN =
         Pattern.compile("^'\\p{XDigit}{8}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{12}'$");
 
@@ -241,7 +248,16 @@ public class OdbcEscapeUtils {
                 return parseScalarExpression(text, startPos0, len0);
 
             case GUID:
-                return parseGuidExpression(text, startPos0, len0);
+                return parseExpression(text, startPos0, len0, token.type(), GUID_PATTERN);
+
+            case DATE:
+                return parseExpression(text, startPos0, len0, token.type(), DATE_PATTERN);
+
+            case TIME:
+                return parseExpression(text, startPos0, len0, token.type(), TIME_PATTERN);
+
+            case TIMESTAMP:
+                return parseExpression(text, startPos0, len0, token.type(), TIMESTAMP_PATTERN);
 
             default:
                 throw new IgniteException("Unsupported escape sequence token [text=" +
@@ -269,11 +285,11 @@ public class OdbcEscapeUtils {
      * @param len Length.
      * @return Parsed expression.
      */
-    private static String parseGuidExpression(String text, int startPos, int len) {
+    private static String parseExpression(String text, int startPos, int len, OdbcEscapeType type, Pattern pattern) {
         String val = substring(text, startPos, len).trim();
 
-        if (!GUID_PATTERN.matcher(val).matches())
-            throw new IgniteException("Invalid GUID escape sequence: " + substring(text, startPos, len));
+        if (!pattern.matcher(val).matches())
+            throw new IgniteException("Invalid " + type + " escape sequence: " + substring(text, startPos, len));
 
         return val;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/92f18bf3/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
index 7225c1a..1aa90fd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
@@ -142,12 +142,14 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
      * Test invalid escape sequence.
      */
     public void testFailedOnInvalidFunctionSequence() {
+        checkFail("{fnfunc1()}");
+
         checkFail("select {fn func1(field1, {fn func2(field2), field3)} from SomeTable;");
 
         checkFail("select {fn func1(field1, fn func2(field2)}, field3)} from SomeTable;");
     }
 
-     /**
+    /**
      * Test escape sequences with additional whitespace characters
      */
     public void testFunctionEscapeSequenceWithWhitespaces() throws Exception {
@@ -155,7 +157,7 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
 
         check("func1()", "{    fn  func1()}");
 
-        check("func1()", "{ \n fn  func1()}");
+        check("func1()", "{ \n fn\nfunc1()}");
 
         checkFail("{ \n func1()}");
     }
@@ -178,6 +180,21 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
             "select '12345678-9abc-def0-1234-123456789abc'",
             "select {guid '12345678-9abc-def0-1234-123456789abc'}"
         );
+    }
+
+    /**
+     * Test invalid escape sequence.
+     */
+    public void testFailedOnInvalidGuidSequence() {
+        checkFail("select {guid'12345678-9abc-def0-1234-123456789abc'}");
+
+        checkFail("select {guid 12345678-9abc-def0-1234-123456789abc'}");
+
+        checkFail("select {guid '12345678-9abc-def0-1234-123456789abc}");
+
+        checkFail("select {guid '12345678-9abc-def0-1234-123456789abc' from SomeTable;");
+
+        checkFail("select guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;");
 
         checkFail("select {guid '1234567-1234-1234-1234-123456789abc'}");
 
@@ -191,15 +208,6 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Test invalid escape sequence.
-     */
-    public void testFailedOnInvalidGuidSequence() {
-        checkFail("select {guid '12345678-9abc-def0-1234-123456789abc' from SomeTable;");
-
-        checkFail("select guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;");
-    }
-
-    /**
      * Test escape sequences with additional whitespace characters
      */
     public void testGuidEscapeSequenceWithWhitespaces() throws Exception {
@@ -220,6 +228,167 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test date escape sequences
+     */
+    public void testDateEscapeSequence() throws Exception {
+        check(
+            "'2016-08-26'",
+            "{d '2016-08-26'}"
+        );
+
+        check(
+            "select '2016-08-26'",
+            "select {d '2016-08-26'}"
+        );
+
+        check(
+            "select '2016-08-26' from table;",
+            "select {d '2016-08-26'} from table;"
+        );
+    }
+
+    /**
+     * Test date escape sequences with additional whitespace characters
+     */
+    public void testDateEscapeSequenceWithWhitespaces() throws Exception {
+        check("'2016-08-26'", "{ d '2016-08-26'}");
+
+        check("'2016-08-26'", "{   d  '2016-08-26'}");
+
+        check("'2016-08-26'", "{ \n d\n'2016-08-26'}");
+    }
+
+    /**
+     * Test invalid escape sequence.
+     */
+    public void testFailedOnInvalidDateSequence() {
+        checkFail("{d'2016-08-26'}");
+
+        checkFail("{d 2016-08-26'}");
+
+        checkFail("{d '2016-08-26}");
+
+        checkFail("{d '16-08-26'}");
+
+        checkFail("{d '2016/08/02'}");
+
+        checkFail("select {d '2016-08-26' from table;");
+
+        checkFail("select {}d '2016-08-26'} from table;");
+    }
+
+    /**
+     * Test date escape sequences
+     */
+    public void testTimeEscapeSequence() throws Exception {
+        check("'13:15:08'", "{t '13:15:08'}");
+
+        check("select '13:15:08'", "select {t '13:15:08'}");
+
+        check("select '13:15:08' from table;", "select {t '13:15:08'} from table;"
+        );
+    }
+
+    /**
+     * Test date escape sequences with additional whitespace characters
+     */
+    public void testTimeEscapeSequenceWithWhitespaces() throws Exception {
+        check("'13:15:08'", "{ t '13:15:08'}");
+
+        check("'13:15:08'", "{   t  '13:15:08'}");
+
+        check("'13:15:08'", "{ \n t\n'13:15:08'}");
+    }
+
+    /**
+     * Test invalid escape sequence.
+     */
+    public void testFailedOnInvalidTimeSequence() {
+        checkFail("{t'13:15:08'}");
+
+        checkFail("{t 13:15:08'}");
+
+        checkFail("{t '13:15:08}");
+
+        checkFail("{t '13 15:08'}");
+
+        checkFail("{t '3:15:08'}");
+
+        checkFail("select {t '13:15:08' from table;");
+
+        checkFail("select {}t '13:15:08'} from table;");
+    }
+
+    /**
+     * Test timestamp escape sequences
+     */
+    public void testTimestampEscapeSequence() throws Exception {
+        check(
+            "'2016-08-26 13:15:08'",
+            "{ts '2016-08-26 13:15:08'}"
+        );
+
+        check(
+            "'2016-08-26 13:15:08.123456'",
+            "{ts '2016-08-26 13:15:08.123456'}"
+        );
+
+        check(
+            "select '2016-08-26 13:15:08'",
+            "select {ts '2016-08-26 13:15:08'}"
+        );
+
+        check(
+            "select '2016-08-26 13:15:08' from table;",
+            "select {ts '2016-08-26 13:15:08'} from table;"
+        );
+    }
+
+    /**
+     * Test timestamp escape sequences with additional whitespace characters
+     */
+    public void testTimestampEscapeSequenceWithWhitespaces() throws Exception {
+        check("'2016-08-26 13:15:08'",
+            "{ ts '2016-08-26 13:15:08'}"
+        );
+
+        check("'2016-08-26 13:15:08'",
+            "{   ts  '2016-08-26 13:15:08'}"
+        );
+
+        check("'2016-08-26 13:15:08'",
+            "{ \n ts\n'2016-08-26 13:15:08'}"
+        );
+    }
+
+    /**
+     * Test invalid escape sequence.
+     */
+    public void testFailedOnInvalidTimestampSequence() {
+        checkFail("{ts '2016-08-26 13:15:08,12345'}");
+
+        checkFail("{ts'2016-08-26 13:15:08'}");
+
+        checkFail("{ts 2016-08-26 13:15:08'}");
+
+        checkFail("{ts '2016-08-26 13:15:08}");
+
+        checkFail("{ts '16-08-26 13:15:08'}");
+
+        checkFail("{ts '2016-08-26 3:25:08'}");
+
+        checkFail("{ts '2016-08 26 03:25:08'}");
+
+        checkFail("{ts '2016-08-26 03 25:08'}");
+
+        checkFail("{t s '2016-08-26 13:15:08''}");
+
+        checkFail("select {ts '2016-08-26 13:15:08' from table;");
+
+        checkFail("select {}ts '2016-08-26 13:15:08'} from table;");
+    }
+
+    /**
      * Check parsing logic.
      *
      * @param exp Expected result.


[21/24] ignite git commit: ignite-2968 Deadlock detection for optimistic tx and near caches

Posted by sb...@apache.org.
ignite-2968 Deadlock detection for optimistic tx and near caches


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

Branch: refs/heads/master
Commit: 0465874d9dddcf962a82a2ef38589121201f0b75
Parents: 2891703
Author: agura <ag...@gridgain.com>
Authored: Wed Aug 24 21:13:29 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Mon Aug 29 16:01:16 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  19 +-
 .../GridCachePartitionExchangeManager.java      |   7 +
 .../GridDistributedTxPrepareRequest.java        |   4 +-
 .../distributed/dht/GridDhtLockFuture.java      |  53 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   4 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  26 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java | 109 +++-
 .../dht/GridDhtTxPrepareRequest.java            |   4 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  37 +-
 .../distributed/near/GridNearLockFuture.java    |  90 ++-
 ...arOptimisticSerializableTxPrepareFuture.java |  13 +-
 .../near/GridNearOptimisticTxPrepareFuture.java | 263 ++++++---
 ...ridNearOptimisticTxPrepareFutureAdapter.java |   5 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   8 +-
 .../near/GridNearTxFinishFuture.java            |   5 +-
 .../cache/distributed/near/GridNearTxLocal.java |  16 +-
 .../near/GridNearTxPrepareRequest.java          |   4 +-
 .../cache/transactions/IgniteInternalTx.java    |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     |  37 +-
 .../cache/transactions/IgniteTxHandler.java     |   9 +-
 .../transactions/IgniteTxLocalAdapter.java      |  19 +-
 .../cache/transactions/IgniteTxManager.java     |  86 ++-
 .../cache/transactions/IgniteTxStateImpl.java   |  11 +-
 .../cache/transactions/TxDeadlockDetection.java |  51 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |  91 ++-
 .../IgniteTxTimeoutAbstractTest.java            |   8 +-
 ...tionedMultiNodeLongTxTimeoutFullApiTest.java |  34 ++
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |  41 ++
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |   5 +-
 .../transactions/DepthFirstSearchTest.java      | 100 +++-
 .../TxDeadlockDetectionNoHangsTest.java         | 246 ++++++++
 .../transactions/TxDeadlockDetectionTest.java   |  13 +-
 ...timisticDeadlockDetectionCrossCacheTest.java | 257 +++++++++
 .../TxOptimisticDeadlockDetectionTest.java      | 574 +++++++++++++++++++
 ...simisticDeadlockDetectionCrossCacheTest.java | 165 ++++--
 .../TxPessimisticDeadlockDetectionTest.java     |  50 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |   4 +
 .../TxDeadlockDetectionTestSuite.java           |   6 +
 .../commands/cache/VisorCacheStopCommand.scala  |   5 +-
 39 files changed, 2127 insertions(+), 355 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 57fa68e..f692bf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -4493,17 +4493,30 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /**
-     * @return All MVCC local candidates.
+     * @return All MVCC local and non near candidates.
      */
+    @SuppressWarnings("ForLoopReplaceableByForEach")
     @Nullable public synchronized List<GridCacheMvccCandidate> mvccAllLocal() {
         GridCacheMvcc mvcc = extras != null ? extras.mvcc() : null;
 
         if (mvcc == null)
             return null;
 
-        List<GridCacheMvccCandidate> locs = mvcc.allLocal();
+        List<GridCacheMvccCandidate> allLocs = mvcc.allLocal();
 
-        return (locs == null || locs.isEmpty()) ? null : new ArrayList<>(locs);
+        if (allLocs == null || allLocs.isEmpty())
+            return null;
+
+        List<GridCacheMvccCandidate> locs = new ArrayList<>(allLocs.size());
+
+        for (int i = 0; i < allLocs.size(); i++) {
+            GridCacheMvccCandidate loc = allLocs.get(i);
+
+            if (!loc.nearLocal())
+                locs.add(loc);
+        }
+
+        return locs.isEmpty() ? null : locs;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index e6ab046..4eb61e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -1302,6 +1302,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
             for (GridCacheFuture<?> fut : mvcc.atomicFutures())
                 U.warn(log, ">>> " + fut);
+
+            if (tm != null) {
+                U.warn(log, "Pending transaction deadlock detection futures:");
+
+                for (IgniteInternalFuture<?> fut : tm.deadlockDetectionFutures())
+                    U.warn(log, ">>> " + fut);
+            }
         }
 
         for (GridCacheContext ctx : cctx.cacheContexts()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
index 72e68db..c691374 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java
@@ -154,6 +154,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
 
     /**
      * @param tx Cache transaction.
+     * @param timeout Transactions timeout.
      * @param reads Read entries.
      * @param writes Write entries.
      * @param txNodes Transaction nodes mapping.
@@ -162,6 +163,7 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
      */
     public GridDistributedTxPrepareRequest(
         IgniteInternalTx tx,
+        long timeout,
         @Nullable Collection<IgniteTxEntry> reads,
         Collection<IgniteTxEntry> writes,
         Map<UUID, Collection<UUID>> txNodes,
@@ -174,12 +176,12 @@ public class GridDistributedTxPrepareRequest extends GridDistributedBaseMessage
         threadId = tx.threadId();
         concurrency = tx.concurrency();
         isolation = tx.isolation();
-        timeout = tx.timeout();
         invalidate = tx.isInvalidate();
         txSize = tx.size();
         sys = tx.system();
         plc = tx.ioPolicy();
 
+        this.timeout = timeout;
         this.reads = reads;
         this.writes = writes;
         this.txNodes = txNodes;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 64b8745..b005b29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -242,12 +242,6 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             msgLog = cctx.shared().txLockMessageLogger();
             log = U.logger(cctx.kernalContext(), logRef, GridDhtLockFuture.class);
         }
-
-        if (timeout > 0) {
-            timeoutObj = new LockTimeoutObject();
-
-            cctx.time().addTimeoutObject(timeoutObj);
-        }
     }
 
     /** {@inheritDoc} */
@@ -298,8 +292,10 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     /**
      * @return Entries.
      */
-    public synchronized Collection<GridDhtCacheEntry> entriesCopy() {
-        return new ArrayList<>(entries());
+    public Collection<GridDhtCacheEntry> entriesCopy() {
+        synchronized (futs) {
+            return new ArrayList<>(entries());
+        }
     }
 
     /**
@@ -412,7 +408,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             return null;
         }
 
-        synchronized (this) {
+        synchronized (futs) {
             entries.add(c == null || c.reentry() ? null : entry);
 
             if (c != null && !c.reentry())
@@ -614,7 +610,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
      * @param t Error.
      */
     public void onError(Throwable t) {
-        synchronized (this) {
+        synchronized (futs) {
             if (err != null)
                 return;
 
@@ -654,15 +650,16 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
      * @param entry Entry whose lock ownership changed.
      */
     @Override public boolean onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate owner) {
-        if (isDone())
+        if (isDone() || (inTx() && tx.remainingTime() == -1))
             return false; // Check other futures.
 
         if (log.isDebugEnabled())
             log.debug("Received onOwnerChanged() callback [entry=" + entry + ", owner=" + owner + "]");
 
         if (owner != null && owner.version().equals(lockVer)) {
-            synchronized (this) {
-                pendingLocks.remove(entry.key());
+            synchronized (futs) {
+                if (!pendingLocks.remove(entry.key()))
+                    return false;
             }
 
             if (checkLocks())
@@ -677,8 +674,10 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     /**
      * @return {@code True} if locks have been acquired.
      */
-    private synchronized boolean checkLocks() {
-        return pendingLocks.isEmpty();
+    private boolean checkLocks() {
+        synchronized (futs) {
+            return pendingLocks.isEmpty();
+        }
     }
 
     /** {@inheritDoc} */
@@ -709,7 +708,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
         if (isDone() || (err == null && success && !checkLocks()))
             return false;
 
-        synchronized (this) {
+        synchronized (futs) {
             if (this.err == null)
                 this.err = err;
         }
@@ -776,13 +775,19 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
         }
 
         readyLocks();
+
+        if (timeout > 0) {
+            timeoutObj = new LockTimeoutObject();
+
+            cctx.time().addTimeoutObject(timeoutObj);
+        }
     }
 
     /**
      * @param entries Entries.
      */
     private void map(Iterable<GridDhtCacheEntry> entries) {
-        synchronized (this) {
+        synchronized (futs) {
             if (mapped)
                 return;
 
@@ -842,6 +847,8 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             if (log.isDebugEnabled())
                 log.debug("Mapped DHT lock future [dhtMap=" + F.nodeIds(dhtMap.keySet()) + ", dhtLockFut=" + this + ']');
 
+            long timeout = inTx() ? tx.remainingTime() : this.timeout;
+
             // Create mini futures.
             for (Map.Entry<ClusterNode, List<GridDhtCacheEntry>> mapped : dhtMap.entrySet()) {
                 ClusterNode n = mapped.getKey();
@@ -853,6 +860,9 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                 if (cnt > 0) {
                     assert !n.id().equals(cctx.localNodeId());
 
+                    if (inTx() && tx.remainingTime() == -1)
+                        return;
+
                     MiniFuture fut = new MiniFuture(n, dhtMapping);
 
                     GridDhtLockRequest req = new GridDhtLockRequest(
@@ -1109,7 +1119,14 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             if (log.isDebugEnabled())
                 log.debug("Timed out waiting for lock response: " + this);
 
-            timedOut = true;
+            synchronized (futs) {
+                timedOut = true;
+
+                // Stop locks and responses processing.
+                pendingLocks.clear();
+
+                futs.clear();
+            }
 
             boolean releaseLocks = !(inTx() && cctx.tm().deadlockDetectionEnabled());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index 4ece775..d2e26b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -133,6 +133,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public boolean onNodeLeft(UUID nodeId) {
         for (IgniteInternalFuture<?> fut : futures())
             if (isMini(fut)) {
@@ -391,8 +392,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      * @param nearMap Near map.
      * @return {@code True} in case there is at least one synchronous {@code MiniFuture} to wait for.
      */
-    private boolean finish(Map<UUID, GridDistributedTxMapping> dhtMap,
-        Map<UUID, GridDistributedTxMapping> nearMap) {
+    private boolean finish(Map<UUID, GridDistributedTxMapping> dhtMap, Map<UUID, GridDistributedTxMapping> nearMap) {
         if (tx.onePhaseCommit())
             return false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index c9d4345..b659abb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@ -314,6 +314,8 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
                 true);
         }
 
+        long timeout = remainingTime();
+
         // For pessimistic mode we don't distribute prepare request.
         GridDhtTxPrepareFuture fut = prepFut;
 
@@ -322,11 +324,16 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
             if (!PREP_FUT_UPD.compareAndSet(this, null, fut = new GridDhtTxPrepareFuture(
                 cctx,
                 this,
+                timeout,
                 nearMiniId,
                 Collections.<IgniteTxKey, GridCacheVersion>emptyMap(),
                 true,
-                needReturnValue())))
+                needReturnValue()))) {
+                if (timeout == -1)
+                    prepFut.onError(timeoutException());
+
                 return prepFut;
+            }
         }
         else
             // Prepare was called explicitly.
@@ -334,15 +341,16 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
 
         if (!state(PREPARING)) {
             if (setRollbackOnly()) {
-                if (timedOut())
-                    fut.onError(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + this));
+                if (timeout == -1)
+                    fut.onError(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " +
+                        this));
                 else
                     fut.onError(new IgniteCheckedException("Invalid transaction state for prepare [state=" + state() +
                         ", tx=" + this + ']'));
             }
             else
-                fut.onError(new IgniteTxRollbackCheckedException("Invalid transaction state for prepare [state=" + state()
-                    + ", tx=" + this + ']'));
+                fut.onError(new IgniteTxRollbackCheckedException("Invalid transaction state for prepare [state=" +
+                    state() + ", tx=" + this + ']'));
 
             return fut;
         }
@@ -394,6 +402,8 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
         // In optimistic mode prepare still can be called explicitly from salvageTx.
         GridDhtTxPrepareFuture fut = prepFut;
 
+        long timeout = remainingTime();
+
         if (fut == null) {
             init();
 
@@ -401,6 +411,7 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
             if (!PREP_FUT_UPD.compareAndSet(this, null, fut = new GridDhtTxPrepareFuture(
                 cctx,
                 this,
+                timeout,
                 nearMiniId,
                 verMap,
                 last,
@@ -410,6 +421,9 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
                 assert f.nearMiniId().equals(nearMiniId) : "Wrong near mini id on existing future " +
                     "[futMiniId=" + f.nearMiniId() + ", miniId=" + nearMiniId + ", fut=" + f + ']';
 
+                if (timeout == -1)
+                    f.onError(timeoutException());
+
                 return chainOnePhasePrepare(f);
             }
         }
@@ -427,7 +441,7 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
                     fut.complete();
 
                 if (setRollbackOnly()) {
-                    if (timedOut())
+                    if (timeout == -1)
                         fut.onError(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " +
                             this));
                     else

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index e9805aa..1bdd9b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -59,8 +59,10 @@ import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.dr.GridDrType;
+import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
+import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
@@ -204,9 +206,13 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
     /** */
     private boolean invoke;
 
+    /** Timeout object. */
+    private final PrepareTimeoutObject timeoutObj;
+
     /**
      * @param cctx Context.
      * @param tx Transaction.
+     * @param timeout Timeout.
      * @param nearMiniId Near mini future id.
      * @param dhtVerMap DHT versions map.
      * @param last {@code True} if this is last prepare operation for node.
@@ -215,6 +221,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
     public GridDhtTxPrepareFuture(
         GridCacheSharedContext cctx,
         final GridDhtTxLocalAdapter tx,
+        long timeout,
         IgniteUuid nearMiniId,
         Map<IgniteTxKey, GridCacheVersion> dhtVerMap,
         boolean last,
@@ -243,6 +250,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
         assert dhtMap != null;
         assert nearMap != null;
+
+        timeoutObj = timeout > 0 ? new PrepareTimeoutObject(timeout) : null;
     }
 
     /** {@inheritDoc} */
@@ -269,7 +278,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
         boolean rmv;
 
-        synchronized (lockKeys) {
+        synchronized (futs) {
             rmv = lockKeys.remove(entry.txKey());
         }
 
@@ -300,7 +309,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         if (!locksReady)
             return false;
 
-        synchronized (lockKeys) {
+        synchronized (futs) {
             return lockKeys.isEmpty();
         }
     }
@@ -483,32 +492,28 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      * @param res Result.
      */
     public void onResult(UUID nodeId, GridDhtTxPrepareResponse res) {
-        if (!isDone()) {
-            boolean found = false;
-
-            MiniFuture mini = miniFuture(res.miniId());
+        if (isDone()) {
+            if (msgLog.isDebugEnabled()) {
+                msgLog.debug("DHT prepare fut, response for finished future [txId=" + tx.nearXidVersion() +
+                    ", dhtTxId=" + tx.xidVersion() +
+                    ", node=" + nodeId +
+                    ", res=" + res +
+                    ", fut=" + this + ']');
+            }
 
-            if (mini != null) {
-                found = true;
+            return;
+        }
 
-                assert mini.node().id().equals(nodeId);
+        MiniFuture mini = miniFuture(res.miniId());
 
-                mini.onResult(res);
-            }
+        if (mini != null) {
+            assert mini.node().id().equals(nodeId);
 
-            if (!found) {
-                if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("DHT prepare fut, failed to find mini future [txId=" + tx.nearXidVersion() +
-                        ", dhtTxId=" + tx.xidVersion() +
-                        ", node=" + nodeId +
-                        ", res=" + res +
-                        ", fut=" + this + ']');
-                }
-            }
+            mini.onResult(res);
         }
         else {
             if (msgLog.isDebugEnabled()) {
-                msgLog.debug("DHT prepare fut, response for finished future [txId=" + tx.nearXidVersion() +
+                msgLog.debug("DHT prepare fut, failed to find mini future [txId=" + tx.nearXidVersion() +
                     ", dhtTxId=" + tx.xidVersion() +
                     ", node=" + nodeId +
                     ", res=" + res +
@@ -525,8 +530,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
     private MiniFuture miniFuture(IgniteUuid miniId) {
-        // We iterate directly over the futs collection here to avoid copy.
         synchronized (futs) {
+            // We iterate directly over the futs collection here to avoid copy.
             // Avoid iterator creation.
             for (int i = 0; i < futs.size(); i++) {
                 IgniteInternalFuture<IgniteInternalTx> fut = futs.get(i);
@@ -543,9 +548,9 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         return null;
                 }
             }
-        }
 
-        return null;
+            return null;
+        }
     }
 
     /**
@@ -583,7 +588,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
             }
 
             if (tx.optimistic() && txEntry.explicitVersion() == null) {
-                synchronized (lockKeys) {
+                synchronized (futs) {
                     lockKeys.add(txEntry.txKey());
                 }
             }
@@ -934,6 +939,9 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
             // Don't forget to clean up.
             cctx.mvcc().removeMvccFuture(this);
 
+            if (timeoutObj != null)
+                cctx.time().removeTimeoutObject(timeoutObj);
+
             return true;
         }
 
@@ -989,6 +997,11 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
         readyLocks();
 
+        if (timeoutObj != null) {
+            // Start timeout tracking after 'readyLocks' to avoid race with timeout processing.
+            cctx.time().addTimeoutObject(timeoutObj);
+        }
+
         mapIfLocked();
     }
 
@@ -1158,6 +1171,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
             if (last) {
                 assert tx.transactionNodes() != null;
 
+                final long timeout = timeoutObj != null ? timeoutObj.timeout : 0;
+
                 // Create mini futures.
                 for (GridDistributedTxMapping dhtMapping : tx.dhtMap().values()) {
                     assert !dhtMapping.empty();
@@ -1175,6 +1190,9 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                     if (F.isEmpty(dhtWrites) && F.isEmpty(nearWrites))
                         continue;
 
+                    if (tx.remainingTime() == -1)
+                        return;
+
                     MiniFuture fut = new MiniFuture(n.id(), dhtMapping, nearMapping);
 
                     add(fut); // Append new future.
@@ -1186,6 +1204,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         fut.futureId(),
                         tx.topologyVersion(),
                         tx,
+                        timeout,
                         dhtWrites,
                         nearWrites,
                         txNodes,
@@ -1284,15 +1303,19 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                 for (GridDistributedTxMapping nearMapping : tx.nearMap().values()) {
                     if (!tx.dhtMap().containsKey(nearMapping.node().id())) {
+                        if (tx.remainingTime() == -1)
+                            return;
+
                         MiniFuture fut = new MiniFuture(nearMapping.node().id(), null, nearMapping);
 
-                        add(fut); // Append new future.
+                        add(fut);
 
                         GridDhtTxPrepareRequest req = new GridDhtTxPrepareRequest(
                             futId,
                             fut.futureId(),
                             tx.topologyVersion(),
                             tx,
+                            timeout,
                             null,
                             nearMapping.writes(),
                             tx.transactionNodes(),
@@ -1719,4 +1742,38 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
             return S.toString(MiniFuture.class, this, "done", isDone(), "cancelled", isCancelled(), "err", error());
         }
     }
+
+    /**
+     *
+     */
+    private class PrepareTimeoutObject extends GridTimeoutObjectAdapter {
+        /** */
+        private final long timeout;
+
+        /**
+         * @param timeout Timeout.
+         */
+        PrepareTimeoutObject(long timeout) {
+            super(timeout);
+
+            this.timeout = timeout;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onTimeout() {
+            synchronized (futs) {
+                futs.clear();
+
+                lockKeys.clear();
+            }
+
+            onError(new IgniteTxTimeoutCheckedException("Failed to acquire lock within " +
+                "provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']'));
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(PrepareTimeoutObject.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
index d31ecba..1cdc96f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
@@ -112,6 +112,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
      * @param miniId Mini future ID.
      * @param topVer Topology version.
      * @param tx Transaction.
+     * @param timeout Transaction timeout.
      * @param dhtWrites DHT writes.
      * @param nearWrites Near writes.
      * @param txNodes Transaction nodes mapping.
@@ -124,6 +125,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
         IgniteUuid miniId,
         AffinityTopologyVersion topVer,
         GridDhtTxLocalAdapter tx,
+        long timeout,
         Collection<IgniteTxEntry> dhtWrites,
         Collection<IgniteTxEntry> nearWrites,
         Map<UUID, Collection<UUID>> txNodes,
@@ -133,7 +135,7 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
         UUID subjId,
         int taskNameHash,
         boolean addDepInfo) {
-        super(tx, null, dhtWrites, txNodes, onePhaseCommit, addDepInfo);
+        super(tx, timeout, null, dhtWrites, txNodes, onePhaseCommit, addDepInfo);
 
         assert futId != null;
         assert miniId != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index f77efee..b0eea01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -443,23 +443,33 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
     /**
      * @return Keys for which locks requested from remote nodes but response isn't received.
      */
-    public Set<KeyCacheObject> requestedKeys() {
-        Set<KeyCacheObject> requestedKeys = null;
+    public Set<IgniteTxKey> requestedKeys() {
+        synchronized (futs) {
+            if (timeoutObj != null && timeoutObj.requestedKeys != null)
+                return timeoutObj.requestedKeys;
+
+            return requestedKeys0();
+        }
+    }
 
+    /**
+     * @return Keys for which locks requested from remote nodes but response isn't received.
+     */
+    private Set<IgniteTxKey> requestedKeys0() {
         for (IgniteInternalFuture<Boolean> miniFut : futures()) {
             if (isMini(miniFut) && !miniFut.isDone()) {
-                if (requestedKeys == null)
-                    requestedKeys = new HashSet<>();
-
                 MiniFuture mini = (MiniFuture)miniFut;
 
-                requestedKeys.addAll(mini.keys);
+                Set<IgniteTxKey> requestedKeys = U.newHashSet(mini.keys.size());
+
+                for (KeyCacheObject key : mini.keys)
+                    requestedKeys.add(new IgniteTxKey(key, cctx.cacheId()));
 
                 return requestedKeys;
             }
         }
 
-        return requestedKeys;
+        return null;
     }
 
     /**
@@ -1312,12 +1322,21 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
             super(timeout);
         }
 
+        /** Requested keys. */
+        private Set<IgniteTxKey> requestedKeys;
+
         /** {@inheritDoc} */
         @Override public void onTimeout() {
             if (log.isDebugEnabled())
                 log.debug("Timed out waiting for lock response: " + this);
 
             if (inTx() && cctx.tm().deadlockDetectionEnabled()) {
+                synchronized (futs) {
+                    requestedKeys = requestedKeys0();
+
+                    futs.clear(); // Stop response processing.
+                }
+
                 Set<IgniteTxKey> keys = new HashSet<>();
 
                 for (IgniteTxEntry txEntry : tx.allEntries()) {
@@ -1434,7 +1453,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                 tx.removeMapping(node.id());
 
             // Primary node left the grid, so fail the future.
-            GridDhtColocatedLockFuture.this.onDone(newTopologyException(e, node.id()));
+            GridDhtColocatedLockFuture.this.onDone(false, newTopologyException(e, node.id()));
 
             onDone(true);
         }
@@ -1494,7 +1513,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                 else
                     remap();
             }
-            else  {
+            else {
                 int i = 0;
 
                 for (KeyCacheObject k : keys) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 4b6448b..3d9b6ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -20,10 +20,12 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
@@ -48,8 +50,10 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopolo
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
+import org.apache.ignite.internal.processors.cache.transactions.TxDeadlock;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
+import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -63,7 +67,9 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.transactions.TransactionDeadlockException;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
@@ -481,6 +487,38 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
     }
 
     /**
+     * @return Keys for which locks requested from remote nodes but response isn't received.
+     */
+    public Set<IgniteTxKey> requestedKeys() {
+        synchronized (futs) {
+            if (timeoutObj != null && timeoutObj.requestedKeys != null)
+                return timeoutObj.requestedKeys;
+
+            return requestedKeys0();
+        }
+    }
+
+    /**
+     * @return Keys for which locks requested from remote nodes but response isn't received.
+     */
+    private Set<IgniteTxKey> requestedKeys0() {
+        for (IgniteInternalFuture<Boolean> miniFut : futures()) {
+            if (isMini(miniFut) && !miniFut.isDone()) {
+                MiniFuture mini = (MiniFuture)miniFut;
+
+                Set<IgniteTxKey> requestedKeys = U.newHashSet(mini.keys.size());
+
+                for (KeyCacheObject key : mini.keys)
+                    requestedKeys.add(new IgniteTxKey(key, cctx.cacheId()));
+
+                return requestedKeys;
+            }
+        }
+
+        return null;
+    }
+
+    /**
      * Finds pending mini future by the given mini ID.
      *
      * @param miniId Mini ID to find.
@@ -621,6 +659,10 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
         if (log.isDebugEnabled())
             log.debug("Received onDone(..) callback [success=" + success + ", err=" + err + ", fut=" + this + ']');
 
+        if (inTx() && cctx.tm().deadlockDetectionEnabled() &&
+            (this.err instanceof IgniteTxTimeoutCheckedException || timedOut))
+            return false;
+
         // If locks were not acquired yet, delay completion.
         if (isDone() || (err == null && success && !checkLocks()))
             return false;
@@ -727,7 +769,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             topVer = tx.topologyVersionSnapshot();
 
         if (topVer != null) {
-            for (GridDhtTopologyFuture fut : cctx.shared().exchange().exchangeFutures()){
+            for (GridDhtTopologyFuture fut : cctx.shared().exchange().exchangeFutures()) {
                 if (fut.topologyVersion().equals(topVer)){
                     Throwable err = fut.validateCache(cctx);
 
@@ -1373,6 +1415,9 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             super(timeout);
         }
 
+        /** Requested keys. */
+        private Set<IgniteTxKey> requestedKeys;
+
         /** {@inheritDoc} */
         @SuppressWarnings({"ThrowableInstanceNeverThrown"})
         @Override public void onTimeout() {
@@ -1381,7 +1426,42 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
 
             timedOut = true;
 
-            onComplete(false, true);
+            if (inTx() && cctx.tm().deadlockDetectionEnabled()) {
+                synchronized (futs) {
+                    requestedKeys = requestedKeys0();
+
+                    futs.clear(); // Stop response processing.
+                }
+
+                Set<IgniteTxKey> keys = new HashSet<>();
+
+                for (IgniteTxEntry txEntry : tx.allEntries()) {
+                    if (!txEntry.locked())
+                        keys.add(txEntry.txKey());
+                }
+
+                IgniteInternalFuture<TxDeadlock> fut = cctx.tm().detectDeadlock(tx, keys);
+
+                fut.listen(new IgniteInClosure<IgniteInternalFuture<TxDeadlock>>() {
+                    @Override public void apply(IgniteInternalFuture<TxDeadlock> fut) {
+                        try {
+                            TxDeadlock deadlock = fut.get();
+
+                            if (deadlock != null)
+                                err = new TransactionDeadlockException(deadlock.toString(cctx.shared()));
+                        }
+                        catch (IgniteCheckedException e) {
+                            err = e;
+
+                            U.warn(log, "Failed to detect deadlock.", e);
+                        }
+
+                        onComplete(false, true);
+                    }
+                });
+            }
+            else
+                onComplete(false, true);
         }
 
         /** {@inheritDoc} */
@@ -1466,7 +1546,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
                 tx.removeMapping(node.id());
 
             // Primary node left the grid, so fail the future.
-            GridNearLockFuture.this.onDone(newTopologyException(e, node.id()));
+            GridNearLockFuture.this.onDone(false, newTopologyException(e, node.id()));
 
             onDone(true);
         }
@@ -1483,6 +1563,10 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             }
 
             if (res.error() != null) {
+                if (inTx() && cctx.tm().deadlockDetectionEnabled() &&
+                    (res.error() instanceof IgniteTxTimeoutCheckedException || tx.remainingTime() == -1))
+                    return;
+
                 if (log.isDebugEnabled())
                     log.debug("Finishing mini future with an error due to error in response [miniFut=" + this +
                         ", res=" + res + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
index 6515140..d251528 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticSerializableTxPrepareFuture.java
@@ -184,7 +184,7 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
             }
         }
 
-        if (e instanceof IgniteTxOptimisticCheckedException) {
+        if (e instanceof IgniteTxOptimisticCheckedException || e instanceof IgniteTxTimeoutCheckedException) {
             if (m != null)
                 tx.removeMapping(m.node().id());
         }
@@ -424,10 +424,21 @@ public class GridNearOptimisticSerializableTxPrepareFuture extends GridNearOptim
 
         final ClusterNode n = m.node();
 
+        long timeout = tx.remainingTime();
+
+        if (timeout == -1) {
+            IgniteCheckedException err = tx.timeoutException();
+
+            fut.onResult(err);
+
+            return err;
+        }
+
         GridNearTxPrepareRequest req = new GridNearTxPrepareRequest(
             futId,
             tx.topologyVersion(),
             tx,
+            timeout,
             m.reads(),
             m.writes(),
             m.near(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 1ea99c4..5a300ff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -20,9 +20,11 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import java.util.ArrayDeque;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Queue;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
@@ -41,8 +43,11 @@ import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTx
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxMapping;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
+import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
+import org.apache.ignite.internal.processors.cache.transactions.TxDeadlock;
 import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
+import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -53,7 +58,9 @@ import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiClosure;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.transactions.TransactionDeadlockException;
 import org.apache.ignite.transactions.TransactionTimeoutException;
 import org.jetbrains.annotations.Nullable;
 
@@ -73,8 +80,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
      * @param cctx Context.
      * @param tx Transaction.
      */
-    public GridNearOptimisticTxPrepareFuture(GridCacheSharedContext cctx,
-        GridNearTxLocal tx) {
+    public GridNearOptimisticTxPrepareFuture(GridCacheSharedContext cctx, GridNearTxLocal tx) {
         super(cctx, tx);
 
         assert tx.optimistic() && !tx.serializable() : tx;
@@ -85,7 +91,11 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         if (log.isDebugEnabled())
             log.debug("Transaction future received owner changed callback: " + entry);
 
-        if ((entry.context().isNear() || entry.context().isLocal()) && owner != null && tx.hasWriteKey(entry.txKey())) {
+        if (tx.remainingTime() == -1)
+            return false;
+
+        if ((entry.context().isNear() || entry.context().isLocal()) &&
+            owner != null && tx.hasWriteKey(entry.txKey())) {
             if (keyLockFut != null)
                 keyLockFut.onKeyLocked(entry.txKey());
 
@@ -124,6 +134,12 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
      * @param discoThread {@code True} if executed from discovery thread.
      */
     void onError(Throwable e, boolean discoThread) {
+        if (e instanceof IgniteTxTimeoutCheckedException) {
+            onTimeout();
+
+            return;
+        }
+
         if (X.hasCause(e, ClusterTopologyCheckedException.class) || X.hasCause(e, ClusterTopologyException.class)) {
             if (tx.onePhaseCommit()) {
                 tx.markForBackupCheck();
@@ -160,7 +176,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
             if (mini != null) {
                 assert mini.node().id().equals(nodeId);
 
-                mini.onResult(nodeId, res);
+                mini.onResult(res);
             }
             else {
                 if (msgLog.isDebugEnabled()) {
@@ -182,6 +198,33 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
     }
 
     /**
+     * @return Keys for which {@link MiniFuture} isn't completed.
+     */
+    @SuppressWarnings("ForLoopReplaceableByForEach")
+    public Set<IgniteTxKey> requestedKeys() {
+        synchronized (futs) {
+            for (int i = 0; i < futs.size(); i++) {
+                IgniteInternalFuture<GridNearTxPrepareResponse> fut = futs.get(i);
+
+                if (isMini(fut) && !fut.isDone()) {
+                    MiniFuture miniFut = (MiniFuture)fut;
+
+                    Collection<IgniteTxEntry> entries = miniFut.mapping().entries();
+
+                    Set<IgniteTxKey> keys = U.newHashSet(entries.size());
+
+                    for (IgniteTxEntry entry : entries)
+                        keys.add(entry.txKey());
+
+                    return keys;
+                }
+            }
+        }
+
+        return null;
+    }
+
+    /**
      * Finds pending mini future by the given mini ID.
      *
      * @param miniId Mini ID to find.
@@ -264,7 +307,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
 
             if (!txStateCheck) {
                 if (tx.setRollbackOnly()) {
-                    if (tx.timedOut())
+                    if (tx.remainingTime() == -1)
                         onError(new IgniteTxTimeoutCheckedException("Transaction timed out and " +
                             "was rolled back: " + this), false);
                     else
@@ -437,89 +480,97 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
 
             final ClusterNode n = m.node();
 
-            GridNearTxPrepareRequest req = new GridNearTxPrepareRequest(
-                futId,
-                tx.topologyVersion(),
-                tx,
-                null,
-                m.writes(),
-                m.near(),
-                txMapping.transactionNodes(),
-                m.last(),
-                tx.onePhaseCommit(),
-                tx.needReturnValue() && tx.implicit(),
-                tx.implicitSingle(),
-                m.explicitLock(),
-                tx.subjectId(),
-                tx.taskNameHash(),
-                m.clientFirst(),
-                tx.activeCachesDeploymentEnabled());
-
-            for (IgniteTxEntry txEntry : m.entries()) {
-                if (txEntry.op() == TRANSFORM)
-                    req.addDhtVersion(txEntry.txKey(), null);
-            }
+            long timeout = tx.remainingTime();
+
+            if (timeout != -1) {
+                GridNearTxPrepareRequest req = new GridNearTxPrepareRequest(
+                    futId,
+                    tx.topologyVersion(),
+                    tx,
+                    timeout,
+                    null,
+                    m.writes(),
+                    m.near(),
+                    txMapping.transactionNodes(),
+                    m.last(),
+                    tx.onePhaseCommit(),
+                    tx.needReturnValue() && tx.implicit(),
+                    tx.implicitSingle(),
+                    m.explicitLock(),
+                    tx.subjectId(),
+                    tx.taskNameHash(),
+                    m.clientFirst(),
+                    tx.activeCachesDeploymentEnabled());
+
+                for (IgniteTxEntry txEntry : m.entries()) {
+                    if (txEntry.op() == TRANSFORM)
+                        req.addDhtVersion(txEntry.txKey(), null);
+                }
 
-            // Must lock near entries separately.
-            if (m.near()) {
-                try {
-                    tx.optimisticLockEntries(req.writes());
+                // Must lock near entries separately.
+                if (m.near()) {
+                    try {
+                        tx.optimisticLockEntries(req.writes());
 
-                    tx.userPrepare();
-                }
-                catch (IgniteCheckedException e) {
-                    onError(e, false);
+                        tx.userPrepare();
+                    }
+                    catch (IgniteCheckedException e) {
+                        onError(e, false);
+                    }
                 }
-            }
 
-            final MiniFuture fut = new MiniFuture(this, m, mappings);
+                final MiniFuture fut = new MiniFuture(this, m, mappings);
 
-            req.miniId(fut.futureId());
+                req.miniId(fut.futureId());
 
-            add(fut); // Append new future.
+                add(fut); // Append new future.
 
-            // If this is the primary node for the keys.
-            if (n.isLocal()) {
-                // At this point, if any new node joined, then it is
-                // waiting for this transaction to complete, so
-                // partition reassignments are not possible here.
-                IgniteInternalFuture<GridNearTxPrepareResponse> prepFut = cctx.tm().txHandler().prepareTx(n.id(), tx, req);
+                // If this is the primary node for the keys.
+                if (n.isLocal()) {
+                    // At this point, if any new node joined, then it is
+                    // waiting for this transaction to complete, so
+                    // partition reassignments are not possible here.
+                    IgniteInternalFuture<GridNearTxPrepareResponse> prepFut =
+                        cctx.tm().txHandler().prepareTx(n.id(), tx, req);
 
-                prepFut.listen(new CI1<IgniteInternalFuture<GridNearTxPrepareResponse>>() {
-                    @Override public void apply(IgniteInternalFuture<GridNearTxPrepareResponse> prepFut) {
-                        try {
-                            fut.onResult(n.id(), prepFut.get());
+                    prepFut.listen(new CI1<IgniteInternalFuture<GridNearTxPrepareResponse>>() {
+                        @Override public void apply(IgniteInternalFuture<GridNearTxPrepareResponse> prepFut) {
+                            try {
+                                fut.onResult(prepFut.get());
+                            }
+                            catch (IgniteCheckedException e) {
+                                fut.onResult(e);
+                            }
                         }
-                        catch (IgniteCheckedException e) {
-                            fut.onResult(e);
+                    });
+                }
+                else {
+                    try {
+                        cctx.io().send(n, req, tx.ioPolicy());
+
+                        if (msgLog.isDebugEnabled()) {
+                            msgLog.debug("Near optimistic prepare fut, sent request [txId=" + tx.nearXidVersion() +
+                                ", node=" + n.id() + ']');
                         }
                     }
-                });
-            }
-            else {
-                try {
-                    cctx.io().send(n, req, tx.ioPolicy());
+                    catch (ClusterTopologyCheckedException e) {
+                        e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion()));
 
-                    if (msgLog.isDebugEnabled()) {
-                        msgLog.debug("Near optimistic prepare fut, sent request [txId=" + tx.nearXidVersion() +
-                            ", node=" + n.id() + ']');
+                        fut.onNodeLeft(e, false);
                     }
-                }
-                catch (ClusterTopologyCheckedException e) {
-                    e.retryReadyFuture(cctx.nextAffinityReadyFuture(tx.topologyVersion()));
+                    catch (IgniteCheckedException e) {
+                        if (msgLog.isDebugEnabled()) {
+                            msgLog.debug("Near optimistic prepare fut, failed to sent request [txId=" + tx.nearXidVersion() +
+                                ", node=" + n.id() +
+                                ", err=" + e + ']');
+                        }
 
-                    fut.onNodeLeft(e, false);
-                }
-                catch (IgniteCheckedException e) {
-                    if (msgLog.isDebugEnabled()) {
-                        msgLog.debug("Near optimistic prepare fut, failed to sent request [txId=" + tx.nearXidVersion() +
-                            ", node=" + n.id() +
-                            ", err=" + e + ']');
+                        fut.onResult(e);
                     }
-
-                    fut.onResult(e);
                 }
             }
+            else
+                onTimeout();
         }
         finally {
             if (set)
@@ -623,6 +674,61 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         return cur;
     }
 
+    /**
+     *
+     */
+    @SuppressWarnings("ForLoopReplaceableByForEach")
+    private void onTimeout() {
+        if (cctx.tm().deadlockDetectionEnabled()) {
+            Set<IgniteTxKey> keys = null;
+
+            if (keyLockFut != null)
+                keys = new HashSet<>(keyLockFut.lockKeys);
+            else {
+                if (futs != null && !futs.isEmpty()) {
+                    for (int i = 0; i < futs.size(); i++) {
+                        IgniteInternalFuture<GridNearTxPrepareResponse> fut = futs.get(i);
+
+                        if (isMini(fut) && !fut.isDone()) {
+                            MiniFuture miniFut = (MiniFuture)fut;
+
+                            Collection<IgniteTxEntry> entries = miniFut.mapping().entries();
+
+                            keys = U.newHashSet(entries.size());
+
+                            for (IgniteTxEntry entry : entries)
+                                keys.add(entry.txKey());
+
+                            break;
+                        }
+                    }
+                }
+            }
+
+            add(new GridEmbeddedFuture<>(new IgniteBiClosure<TxDeadlock, Exception, GridNearTxPrepareResponse>() {
+                @Override public GridNearTxPrepareResponse apply(TxDeadlock deadlock, Exception e) {
+                    if (e != null)
+                        U.warn(log, "Failed to detect deadlock.", e);
+                    else {
+                        e = new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout for " +
+                            "transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']',
+                            deadlock != null ? new TransactionDeadlockException(deadlock.toString(cctx)) : null);
+                    }
+
+                    onDone(null, e);
+
+                    return null;
+                }
+            }, cctx.tm().detectDeadlock(tx, keys)));
+        }
+        else {
+            ERR_UPD.compareAndSet(this, null, new IgniteTxTimeoutCheckedException("Failed to acquire lock " +
+                "within provided timeout for transaction [timeout=" + tx.timeout() + ", tx=" + tx + ']'));
+
+            onComplete(false);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         Collection<String> futs = F.viewReadOnly(futures(), new C1<IgniteInternalFuture<?>, String>() {
@@ -652,7 +758,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         private static final long serialVersionUID = 0L;
 
         /** Receive result flag updater. */
-        private static AtomicIntegerFieldUpdater<MiniFuture> RCV_RES_UPD =
+        private static final AtomicIntegerFieldUpdater<MiniFuture> RCV_RES_UPD =
             AtomicIntegerFieldUpdater.newUpdater(MiniFuture.class, "rcvRes");
 
         /** Parent future. */
@@ -745,15 +851,21 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
         }
 
         /**
-         * @param nodeId Failed node ID.
          * @param res Result callback.
          */
         @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-        void onResult(UUID nodeId, final GridNearTxPrepareResponse res) {
+        void onResult(final GridNearTxPrepareResponse res) {
             if (isDone())
                 return;
 
             if (RCV_RES_UPD.compareAndSet(this, 0, 1)) {
+                if (parent.cctx.tm().deadlockDetectionEnabled() &&
+                    (parent.tx.remainingTime() == -1 || res.error() instanceof IgniteTxTimeoutCheckedException)) {
+                    parent.onTimeout();
+
+                    return;
+                }
+
                 if (res.error() != null) {
                     // Fail the whole compound future.
                     parent.onError(res.error(), false);
@@ -801,8 +913,7 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearOptimisticTxPrepa
          */
         private void remap() {
             parent.prepareOnTopology(true, new Runnable() {
-                @Override
-                public void run() {
+                @Override public void run() {
                     onDone((GridNearTxPrepareResponse) null);
                 }
             });

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
index 4d77a3c..a00cf3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
@@ -40,8 +40,7 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT
      * @param cctx Context.
      * @param tx Transaction.
      */
-    public GridNearOptimisticTxPrepareFutureAdapter(GridCacheSharedContext cctx,
-        GridNearTxLocal tx) {
+    public GridNearOptimisticTxPrepareFutureAdapter(GridCacheSharedContext cctx, GridNearTxLocal tx) {
         super(cctx, tx);
 
         assert tx.optimistic() : tx;
@@ -172,7 +171,7 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT
     protected static class KeyLockFuture extends GridFutureAdapter<GridNearTxPrepareResponse> {
         /** */
         @GridToStringInclude
-        private Collection<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
+        protected Collection<IgniteTxKey> lockKeys = new GridConcurrentHashSet<>();
 
         /** */
         private volatile boolean allKeysAdded;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index ef2edc9..34b8281 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -153,7 +153,7 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
     @Override public void prepare() {
         if (!tx.state(PREPARING)) {
             if (tx.setRollbackOnly()) {
-                if (tx.timedOut())
+                if (tx.remainingTime() == -1)
                     onDone(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + tx));
                 else
                     onDone(new IgniteCheckedException("Invalid transaction state for prepare " +
@@ -222,6 +222,11 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
 
         checkOnePhase();
 
+        long timeout = tx.remainingTime();
+
+        if (timeout == -1)
+            onDone(new IgniteTxTimeoutCheckedException("Transaction timed out and was rolled back: " + tx));
+
         for (final GridDistributedTxMapping m : mappings.values()) {
             final ClusterNode node = m.node();
 
@@ -229,6 +234,7 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
                 futId,
                 tx.topologyVersion(),
                 tx,
+                timeout,
                 m.reads(),
                 m.writes(),
                 m.near(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index adde63c..bb5d482 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -633,6 +633,9 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
         if (m.explicitLock())
             syncMode = FULL_SYNC;
 
+        // Version to be added in completed versions on primary node.
+        GridCacheVersion completedVer = !commit && tx.timeout() > 0 ? tx.xidVersion() : null;
+
         GridNearTxFinishRequest req = new GridNearTxFinishRequest(
             futId,
             tx.xidVersion(),
@@ -645,7 +648,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             m.explicitLock(),
             tx.storeEnabled(),
             tx.topologyVersion(),
-            null,
+            completedVer, // Reuse 'baseVersion'  to do not add new fields in message.
             null,
             null,
             tx.size(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 28c60d4..410baf8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -796,6 +796,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         GridNearTxPrepareFutureAdapter fut = (GridNearTxPrepareFutureAdapter)prepFut;
 
         if (fut == null) {
+            long timeout = remainingTime();
+
             // Future must be created before any exception can be thrown.
             if (optimistic()) {
                 fut = serializable() ?
@@ -807,6 +809,12 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
 
             if (!PREP_FUT_UPD.compareAndSet(this, null, fut))
                 return prepFut;
+
+            if (timeout == -1) {
+                fut.onDone(this, timeoutException());
+
+                return fut;
+            }
         }
         else
             // Prepare was called explicitly.
@@ -964,8 +972,10 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         Map<UUID, Collection<UUID>> txNodes,
         boolean last
     ) {
+        long timeout = remainingTime();
+
         if (state() != PREPARING) {
-            if (timedOut())
+            if (timeout == -1)
                 return new GridFinishedFuture<>(
                     new IgniteTxTimeoutCheckedException("Transaction timed out: " + this));
 
@@ -975,11 +985,15 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
                 new IgniteCheckedException("Invalid transaction state for prepare [state=" + state() + ", tx=" + this + ']'));
         }
 
+        if (timeout == -1)
+            return new GridFinishedFuture<>(timeoutException());
+
         init();
 
         GridDhtTxPrepareFuture fut = new GridDhtTxPrepareFuture(
             cctx,
             this,
+            timeout,
             IgniteUuid.randomUuid(),
             Collections.<IgniteTxKey, GridCacheVersion>emptyMap(),
             last,

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
index 9dfdb43..e55566b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
@@ -94,6 +94,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
      * @param futId Future ID.
      * @param topVer Topology version.
      * @param tx Transaction.
+     * @param timeout Transaction timeout.
      * @param reads Read entries.
      * @param writes Write entries.
      * @param near {@code True} if mapping is for near caches.
@@ -112,6 +113,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
         IgniteUuid futId,
         AffinityTopologyVersion topVer,
         IgniteInternalTx tx,
+        long timeout,
         Collection<IgniteTxEntry> reads,
         Collection<IgniteTxEntry> writes,
         boolean near,
@@ -126,7 +128,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
         boolean firstClientReq,
         boolean addDepInfo
     ) {
-        super(tx, reads, writes, txNodes, onePhaseCommit, addDepInfo);
+        super(tx, timeout, reads, writes, txNodes, onePhaseCommit, addDepInfo);
 
         assert futId != null;
         assert !firstClientReq || tx.optimistic() : tx;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
index 8c0425d..dd900fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
@@ -33,7 +33,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheFilterFailedExceptio
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException;
 import org.apache.ignite.internal.util.lang.GridTuple;
 import org.apache.ignite.lang.IgniteAsyncSupported;
@@ -46,7 +45,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Transaction managed by cache ({@code 'Ex'} stands for external).
  */
-public interface IgniteInternalTx extends AutoCloseable, GridTimeoutObject {
+public interface IgniteInternalTx extends AutoCloseable {
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index f76f4bf..eb2989e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -712,7 +712,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
     /**
      * @return Transaction timeout exception.
      */
-    protected final IgniteCheckedException timeoutException() {
+    public final IgniteCheckedException timeoutException() {
         return new IgniteTxTimeoutCheckedException("Failed to acquire lock within provided timeout " +
             "for transaction [timeout=" + timeout() + ", tx=" + this + ']');
     }
@@ -1032,7 +1032,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
      * @return {@code True} if state changed.
      */
     @SuppressWarnings({"TooBroadScope"})
-    private boolean state(TransactionState state, boolean timedOut) {
+    protected boolean state(TransactionState state, boolean timedOut) {
         boolean valid = false;
 
         TransactionState prev;
@@ -1154,24 +1154,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteUuid timeoutId() {
-        return xidVer.asGridUuid();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long endTime() {
-        long endTime = timeout == 0 ? Long.MAX_VALUE : startTime + timeout;
-
-        return endTime > 0 ? endTime : endTime < 0 ? Long.MAX_VALUE : endTime;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onTimeout() {
-        if (local() && !dht())
-            state(MARKED_ROLLBACK, true);
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean timedOut() {
         return timedOut;
     }
@@ -2387,21 +2369,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteUuid timeoutId() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long endTime() {
-            return 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onTimeout() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
         @Override public boolean equals(Object o) {
             return this == o || o instanceof IgniteInternalTx && xid.equals(((IgniteInternalTx)o).xid());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 7c3c206..e67e60f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -112,8 +112,7 @@ public class IgniteTxHandler {
      * @param req Request.
      * @return Prepare future.
      */
-    public IgniteInternalFuture<?> processNearTxPrepareRequest(final UUID nearNodeId,
-        final GridNearTxPrepareRequest req) {
+    public IgniteInternalFuture<?> processNearTxPrepareRequest(final UUID nearNodeId, GridNearTxPrepareRequest req) {
         if (txPrepareMsgLog.isDebugEnabled()) {
             txPrepareMsgLog.debug("Received near prepare request [txId=" + req.version() +
                 ", node=" + nearNodeId + ']');
@@ -361,7 +360,7 @@ public class IgniteTxHandler {
                         req.deployInfo() != null);
 
                     try {
-                        ctx.io().send(nearNode, res, req.policy());
+                        ctx.io().send(nearNodeId, res, req.policy());
 
                         if (txPrepareMsgLog.isDebugEnabled()) {
                             txPrepareMsgLog.debug("Sent remap response for near prepare [txId=" + req.version() +
@@ -667,6 +666,10 @@ public class IgniteTxHandler {
         assert nodeId != null;
         assert req != null;
 
+        // 'baseVersion' message field is re-used for version to be added in completed versions.
+        if (!req.commit() && req.baseVersion() != null)
+            ctx.tm().addRolledbackTx(null, req.baseVersion());
+
         // Transaction on local cache only.
         if (locTx != null && !locTx.nearLocallyMapped() && !locTx.colocatedLocallyMapped())
             return new GridFinishedFuture<IgniteInternalTx>(locTx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index d9aca4a..9ad7fb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -107,6 +107,7 @@ import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE;
 import static org.apache.ignite.internal.processors.dr.GridDrType.DR_PRIMARY;
 import static org.apache.ignite.transactions.TransactionState.COMMITTED;
 import static org.apache.ignite.transactions.TransactionState.COMMITTING;
+import static org.apache.ignite.transactions.TransactionState.MARKED_ROLLBACK;
 import static org.apache.ignite.transactions.TransactionState.PREPARING;
 import static org.apache.ignite.transactions.TransactionState.ROLLED_BACK;
 import static org.apache.ignite.transactions.TransactionState.ROLLING_BACK;
@@ -547,14 +548,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
     @SuppressWarnings({"CatchGenericClass"})
     public void userPrepare() throws IgniteCheckedException {
         if (state() != PREPARING) {
-            if (timedOut())
+            if (remainingTime() == -1)
                 throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this);
 
             TransactionState state = state();
 
             setRollbackOnly();
 
-            throw new IgniteCheckedException("Invalid transaction state for prepare [state=" + state + ", tx=" + this + ']');
+            throw new IgniteCheckedException("Invalid transaction state for prepare [state=" +
+                state + ", tx=" + this + ']');
         }
 
         checkValid();
@@ -629,7 +631,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
         TransactionState state = state();
 
         if (state != COMMITTING) {
-            if (timedOut())
+            if (remainingTime() == -1)
                 throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this);
 
             setRollbackOnly();
@@ -3540,8 +3542,11 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
      * @throws IgniteCheckedException If transaction check failed.
      */
     protected void checkValid() throws IgniteCheckedException {
+        if (local() && !dht() && remainingTime() == -1)
+            state(MARKED_ROLLBACK, true);
+
         if (isRollbackOnly()) {
-            if (timedOut())
+            if (remainingTime() == -1)
                 throw new IgniteTxTimeoutCheckedException("Cache transaction timed out: " + this);
 
             TransactionState state = state();
@@ -3556,10 +3561,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
 
             throw new IgniteCheckedException("Cache transaction marked as rollback-only: " + this);
         }
-
-        if (remainingTime() == -1 && setRollbackOnly())
-            throw new IgniteTxTimeoutCheckedException("Cache transaction timed out " +
-                "(was rolled back automatically): " + this);
     }
 
     /** {@inheritDoc} */
@@ -3604,7 +3605,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
 
         TransactionState state = state();
 
-        assert state == TransactionState.ACTIVE || timedOut() :
+        assert state == TransactionState.ACTIVE || remainingTime() == -1 :
             "Invalid tx state for adding entry [op=" + op + ", val=" + val + ", entry=" + entry + ", filter=" +
                 Arrays.toString(filter) + ", txCtx=" + cctx.tm().txContextVersion() + ", tx=" + this + ']';
 


[11/24] ignite git commit: IGNITE-3390: ODBC: Added DSN configuration dialog for Windows. This closes #881.

Posted by sb...@apache.org.
IGNITE-3390: ODBC: Added DSN configuration dialog for Windows. This closes #881.


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

Branch: refs/heads/master
Commit: 60afa372906246d43456518e5f76a454ca2bcadf
Parents: 4343c73
Author: isapego <ig...@gmail.com>
Authored: Thu Aug 25 16:37:43 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 25 16:37:43 2016 +0300

----------------------------------------------------------------------
 .../cpp/common/include/ignite/common/utils.h    |   7 +
 .../cpp/common/os/win/src/common/utils.cpp      |  20 ++
 .../cpp/core/include/ignite/ignition.h          |   2 +-
 modules/platforms/cpp/core/src/ignition.cpp     |   8 +-
 .../cpp/odbc-test/src/configuration_test.cpp    |   6 +-
 modules/platforms/cpp/odbc/Makefile.am          |   1 +
 modules/platforms/cpp/odbc/include/Makefile.am  |   1 +
 .../platforms/cpp/odbc/include/ignite/odbc.h    |   1 -
 .../include/ignite/odbc/config/configuration.h  |  93 +++++-
 .../ignite/odbc/diagnostic/diagnostic_record.h  |   2 +-
 .../cpp/odbc/include/ignite/odbc/dsn_config.h   |  61 ++++
 .../odbc/include/ignite/odbc/protocol_version.h |  20 +-
 .../include/ignite/odbc/system/odbc_constants.h |   4 -
 .../odbc/system/ui/dsn_configuration_window.h   | 152 +++++++++
 .../ignite/odbc/system/ui/custom_window.h       | 189 ++++++++++++
 .../win/include/ignite/odbc/system/ui/window.h  | 201 ++++++++++++
 .../odbc/os/win/src/system/ui/custom_window.cpp | 184 +++++++++++
 .../src/system/ui/dsn_configuration_window.cpp  | 305 +++++++++++++++++++
 .../cpp/odbc/os/win/src/system/ui/window.cpp    | 192 ++++++++++++
 .../cpp/odbc/os/win/src/system_dsn.cpp          | 218 +++++++++++++
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |  25 +-
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |  30 ++
 .../cpp/odbc/src/config/configuration.cpp       |  17 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |   2 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |   2 +-
 .../diagnostic/diagnostic_record_storage.cpp    |   2 +-
 modules/platforms/cpp/odbc/src/dsn_config.cpp   | 115 +++++++
 modules/platforms/cpp/odbc/src/entry_points.cpp |   8 -
 modules/platforms/cpp/odbc/src/odbc.cpp         |  88 ++----
 .../platforms/cpp/odbc/src/protocol_version.cpp |  23 +-
 30 files changed, 1868 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/common/include/ignite/common/utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/include/ignite/common/utils.h b/modules/platforms/cpp/common/include/ignite/common/utils.h
index 331fcb2..f4d2a9f 100644
--- a/modules/platforms/cpp/common/include/ignite/common/utils.h
+++ b/modules/platforms/cpp/common/include/ignite/common/utils.h
@@ -60,6 +60,13 @@ namespace ignite
         }
 
         /**
+         * Strips leading and trailing whitespaces from string.
+         *
+         * @param str String to be transformed.
+         */
+        IGNITE_IMPORT_EXPORT void StripSurroundingWhitespaces(std::string& str);
+
+        /**
          * Get string representation of long in decimal form.
          *
          * @param val Long value to be converted to string.

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/common/os/win/src/common/utils.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/os/win/src/common/utils.cpp b/modules/platforms/cpp/common/os/win/src/common/utils.cpp
index 47d7f43..77c90b8 100644
--- a/modules/platforms/cpp/common/os/win/src/common/utils.cpp
+++ b/modules/platforms/cpp/common/os/win/src/common/utils.cpp
@@ -40,6 +40,26 @@ namespace ignite
             return false;
         }
 
+        void StripSurroundingWhitespaces(std::string& str)
+        {
+            std::string::size_type newBegin = 0;
+            while (newBegin < str.size() && ::isspace(str[newBegin]))
+                ++newBegin;
+
+            if (newBegin == str.size())
+            {
+                str.clear();
+
+                return;
+            }
+
+            std::string::size_type newEnd = str.size() - 1;
+            while (::isspace(str[newEnd]))
+                --newEnd;
+
+            str.assign(str, newBegin, (newEnd - newBegin) + 1);
+        }
+
         time_t IgniteTimeGm(const tm& time)
         {
             tm tmc = time;

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/core/include/ignite/ignition.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/ignition.h b/modules/platforms/cpp/core/include/ignite/ignition.h
index f88efe5..57fd51c 100644
--- a/modules/platforms/cpp/core/include/ignite/ignition.h
+++ b/modules/platforms/cpp/core/include/ignite/ignition.h
@@ -154,7 +154,7 @@ namespace ignite
          * @param err Error.
          */
         static void StopAll(bool cancel, IgniteError* err);
-    };    
+    };
 }
 
 #endif //_IGNITE_IGNITION
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/core/src/ignition.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/src/ignition.cpp b/modules/platforms/cpp/core/src/ignition.cpp
index 4a7d941..83adb4c 100644
--- a/modules/platforms/cpp/core/src/ignition.cpp
+++ b/modules/platforms/cpp/core/src/ignition.cpp
@@ -216,9 +216,9 @@ namespace ignite
 
                 int optsLen;
                 char** opts = CreateJvmOptions(cfg, homeFound ? &home : NULL, cp, &optsLen);
-                
+
                 envTarget = new SharedPointer<IgniteEnvironment>(env);
-                
+
                 SharedPointer<JniContext> ctx(
                     JniContext::Create(opts, optsLen, env.Get()->GetJniHandlers(envTarget), &jniErr));
 
@@ -258,7 +258,7 @@ namespace ignite
 
                     if (!javaRef) {
                         IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
-                        
+
                         failed = true;
                     }
                     else {
@@ -458,7 +458,7 @@ namespace ignite
             JniErrorInfo jniErr;
 
             SharedPointer<JniContext> ctx(JniContext::Create(NULL, 0, JniHandlers(), &jniErr));
-             
+
             IgniteError::SetError(jniErr.code, jniErr.errCls, jniErr.errMsg, err);
 
             if (err->GetCode() == IgniteError::IGNITE_SUCCESS)

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
index c0bb439..1851eae 100644
--- a/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/configuration_test.cpp
@@ -49,7 +49,7 @@ void CheckValidAddress(const char* connectStr, uint16_t port)
 
     BOOST_CHECK_NO_THROW(cfg.FillFromConnectString(connectStr));
 
-    BOOST_CHECK_EQUAL(cfg.GetPort(), port);
+    BOOST_CHECK_EQUAL(cfg.GetTcpPort(), port);
 }
 
 void CheckValidProtocolVersion(const char* connectStr, ignite::odbc::ProtocolVersion version)
@@ -92,7 +92,7 @@ void CheckConnectionConfig(const Configuration& cfg)
 {
     BOOST_CHECK_EQUAL(cfg.GetDriver(), testDriverName);
     BOOST_CHECK_EQUAL(cfg.GetHost(), testServerHost);
-    BOOST_CHECK_EQUAL(cfg.GetPort(), testServerPort);
+    BOOST_CHECK_EQUAL(cfg.GetTcpPort(), testServerPort);
     BOOST_CHECK_EQUAL(cfg.GetAddress(), testAddress);
     BOOST_CHECK_EQUAL(cfg.GetCache(), testCacheName);
     BOOST_CHECK_EQUAL(cfg.GetDsn(), std::string());
@@ -119,7 +119,7 @@ void CheckDsnConfig(const Configuration& cfg)
     BOOST_CHECK_EQUAL(cfg.GetCache(), Configuration::DefaultValue::cache);
     BOOST_CHECK_EQUAL(cfg.GetAddress(), Configuration::DefaultValue::address);
     BOOST_CHECK_EQUAL(cfg.GetHost(), std::string());
-    BOOST_CHECK_EQUAL(cfg.GetPort(), Configuration::DefaultValue::port);
+    BOOST_CHECK_EQUAL(cfg.GetTcpPort(), Configuration::DefaultValue::port);
     BOOST_CHECK_EQUAL(cfg.IsDistributedJoins(), false);
     BOOST_CHECK_EQUAL(cfg.IsEnforceJoinOrder(), false);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am
index 9faa999..1781bc0 100644
--- a/modules/platforms/cpp/odbc/Makefile.am
+++ b/modules/platforms/cpp/odbc/Makefile.am
@@ -62,6 +62,7 @@ libignite_odbc_la_SOURCES = \
     src/meta/table_meta.cpp \
     src/odbc.cpp \
     src/entry_points.cpp \
+    src/dsn_config.cpp \
     src/query/column_metadata_query.cpp \
     src/query/data_query.cpp \
     src/query/foreign_keys_query.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am
index 0776548..073dcaa 100644
--- a/modules/platforms/cpp/odbc/include/Makefile.am
+++ b/modules/platforms/cpp/odbc/include/Makefile.am
@@ -38,6 +38,7 @@ noinst_HEADERS = \
     ignite/odbc/row.h \
     ignite/odbc/utility.h \
     ignite/odbc/environment.h \
+    ignite/odbc/dsn_config.h \
     ignite/odbc/system/odbc_constants.h \
     ignite/odbc/system/socket_client.h \
     ignite/odbc/meta/primary_key_meta.h \

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/include/ignite/odbc.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc.h b/modules/platforms/cpp/odbc/include/ignite/odbc.h
index 40158dd..810d186 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc.h
@@ -251,7 +251,6 @@ namespace ignite
                                 SQLSMALLINT tableNameLen,
                                 SQLSMALLINT scope,
                                 SQLSMALLINT nullable);
-
 } // namespace ignite
 
 #endif //_IGNITE_ODBC
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
index 30f9ad6..8e19a6e 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/configuration.h
@@ -160,12 +160,19 @@ namespace ignite
                  *
                  * @return Server port.
                  */
-                uint16_t GetPort() const
+                uint16_t GetTcpPort() const
                 {
                     return endPoint.port;
                 }
 
                 /**
+                 * Set server port.
+                 *
+                 * @param port Server port.
+                 */
+                void SetTcpPort(uint16_t port);
+
+                /**
                  * Get DSN.
                  *
                  * @return Data Source Name.
@@ -176,6 +183,16 @@ namespace ignite
                 }
 
                 /**
+                 * Set DSN.
+                 *
+                 * @param dsn Data Source Name.
+                 */
+                void SetDsn(const std::string& dsn)
+                {
+                    arguments[Key::dsn] = dsn;
+                }
+
+                /**
                  * Get Driver.
                  *
                  * @return Driver name.
@@ -196,6 +213,16 @@ namespace ignite
                 }
 
                 /**
+                 * Set server host.
+                 *
+                 * @param server Server host.
+                 */
+                void SetHost(const std::string& server)
+                {
+                    arguments[Key::server] = server;
+                }
+
+                /**
                  * Get cache.
                  *
                  * @return Cache name.
@@ -206,6 +233,16 @@ namespace ignite
                 }
 
                 /**
+                 * Set cache.
+                 *
+                 * @param cache Cache name.
+                 */
+                void SetCache(const std::string& cache)
+                {
+                    arguments[Key::cache] = cache;
+                }
+
+                /**
                  * Get address.
                  *
                  * @return Address.
@@ -216,6 +253,16 @@ namespace ignite
                 }
 
                 /**
+                 * Set address.
+                 *
+                 * @param address Address.
+                 */
+                void SetAddress(const std::string& address)
+                {
+                    arguments[Key::address] = address;
+                }
+
+                /**
                  * Check distributed joins flag.
                  *
                  * @return True if distributed joins are enabled.
@@ -226,6 +273,16 @@ namespace ignite
                 }
 
                 /**
+                 * Set distributed joins.
+                 *
+                 * @param val Value to set.
+                 */
+                void SetDistributedJoins(bool val)
+                {
+                    SetBoolValue(Key::distributedJoins, val);
+                }
+
+                /**
                  * Check enforce join order flag.
                  *
                  * @return True if enforcing of join order is enabled.
@@ -236,6 +293,26 @@ namespace ignite
                 }
 
                 /**
+                 * Set enforce joins.
+                 *
+                 * @param val Value to set.
+                 */
+                void SetEnforceJoinOrder(bool val)
+                {
+                    SetBoolValue(Key::enforceJoinOrder, val);
+                }
+
+                /**
+                 * Get argument map.
+                 *
+                 * @return Argument map.
+                 */
+                const ArgumentMap& GetMap() const
+                {
+                    return arguments;
+                }
+
+                /**
                  * Get protocol version.
                  *
                  * @return Protocol version.
@@ -243,6 +320,13 @@ namespace ignite
                 ProtocolVersion GetProtocolVersion() const;
 
                 /**
+                 * Set protocol version.
+                 *
+                 * @param version Version to set.
+                 */
+                void SetProtocolVersion(const std::string& version);
+
+                /**
                  * Get string value from the config.
                  *
                  * @param key Configuration key.
@@ -269,6 +353,13 @@ namespace ignite
                  */
                 bool GetBoolValue(const std::string& key, bool dflt) const;
 
+                /**
+                 * Set bool value to the config.
+                 *
+                 * @param key Configuration key.
+                 * @param val Value to set.
+                 */
+                void SetBoolValue(const std::string& key, bool val);
             private:
                 /**
                  * Parse connect string into key-value storage.

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h
index bfb4f4c..670e0aa 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/diagnostic/diagnostic_record.h
@@ -84,7 +84,7 @@ namespace ignite
                  *
                  * @return An informational message on the error or warning.
                  */
-                const std::string& GetMessage() const;
+                const std::string& GetMessageText() const;
 
                 /**
                  * Get connection name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h b/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h
new file mode 100644
index 0000000..dbad9b5
--- /dev/null
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/dsn_config.h
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_DSN_CONFIG
+#define _IGNITE_ODBC_DSN_CONFIG
+
+#include "ignite/odbc/config/configuration.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        /**
+         * Extract last setup error and throw it like IgniteError.
+         */
+        void ThrowLastSetupError();
+
+        /**
+         * Add new string to the DSN file.
+         *
+         * @param dsn DSN name.
+         * @param key Key.
+         * @param value Value.
+         */
+        void WriteDsnString(const char* dsn, const char* key, const char* value);
+
+        /**
+         * Get string from the DSN file.
+         *
+         * @param dsn DSN name.
+         * @param key Key.
+         * @param dflt Default value.
+         * @return Value.
+         */
+        std::string ReadDsnString(const char* dsn, const char* key, const char* dflt);
+
+        /**
+         * Read DSN to fill the configuration.
+         *
+         * @param dsn DSN name.
+         * @param config Configuration.
+         */
+        void ReadDsnConfiguration(const char* dsn, config::Configuration& config);
+    }
+}
+
+#endif //_IGNITE_ODBC_DSN_CONFIG
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
index 747d78d..d3d288e 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/protocol_version.h
@@ -47,6 +47,13 @@ namespace ignite
             static const ProtocolVersion VERSION_UNKNOWN;
 
             /**
+             * Get string to version map.
+             *
+             * @return String to version map.
+             */
+            static const StringToVersionMap& GetMap();
+
+            /**
              * Get current version.
              *
              * @return Current version.
@@ -86,6 +93,13 @@ namespace ignite
             bool IsUnknown() const;
 
             /**
+             * Check if the distributed joins supported.
+             *
+             * @retuen True if the distributed joins supported.
+             */
+            bool IsDistributedJoinsSupported() const;
+
+            /**
              * Comparison operator.
              *
              * @param val1 First value.
@@ -152,10 +166,12 @@ namespace ignite
              *
              * @param major Major version.
              * @param minor Minor version.
-             * @param maintenance Maintenance version.
+             * @param revision Revision.
              * @return Int value for the version.
              */
-            static int64_t MakeVersion(uint16_t major, uint16_t minor, uint16_t maintenance);
+            static int64_t MakeVersion(uint16_t major, uint16_t minor, uint16_t revision);
+
+            ProtocolVersion();
 
             /** String to version map. */
             static const StringToVersionMap stringToVersionMap;

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h
index 60a6552..ecd1a55 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/odbc_constants.h
@@ -27,10 +27,6 @@
 #   undef min
 #endif // min
 
-#ifdef GetMessage
-#   undef GetMessage
-#endif // GetMessage
-
 #endif //_WIN32
 
 #define ODBCVER 0x0380

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
new file mode 100644
index 0000000..d2cb569
--- /dev/null
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/system/ui/dsn_configuration_window.h
@@ -0,0 +1,152 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW
+#define _IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW
+
+#include "ignite/odbc/config/configuration.h"
+#include "ignite/odbc/system/ui/custom_window.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        namespace system
+        {
+            namespace ui
+            {
+                /**
+                 * DSN configuration window class.
+                 */
+                class DsnConfigurationWindow : public CustomWindow
+                {
+                    /**
+                     * Children windows ids.
+                     */
+                    enum ChildId
+                    {
+                        ID_CONNECTION_SETTINGS_GROUP_BOX,
+                        ID_NAME_EDIT,
+                        ID_NAME_LABEL,
+                        ID_ADDRESS_EDIT,
+                        ID_ADDRESS_LABEL,
+                        ID_CACHE_EDIT,
+                        ID_CACHE_LABEL,
+                        ID_DISTRIBUTED_JOINS_CHECK_BOX,
+                        ID_ENFORCE_JOIN_ORDER_CHECK_BOX,
+                        ID_PROTOCOL_VERSION_LABEL,
+                        ID_PROTOCOL_VERSION_COMBO_BOX,
+                        ID_OK_BUTTON,
+                        ID_CANCEL_BUTTON
+                    };
+
+                public:
+                    /**
+                     * Constructor.
+                     *
+                     * @param parent Parent window handle.
+                     */
+                    explicit DsnConfigurationWindow(Window* parent, config::Configuration& config);
+
+                    /**
+                     * Destructor.
+                     */
+                    virtual ~DsnConfigurationWindow();
+
+                    /**
+                     * Create window in the center of the parent window.
+                     */
+                    void Create();
+
+                    /**
+                    * @copedoc ignite::odbc::system::ui::CustomWindow::OnCreate
+                    */
+                    virtual void OnCreate();
+
+                    /**
+                     * @copedoc ignite::odbc::system::ui::CustomWindow::OnMessage
+                     */
+                    virtual bool OnMessage(UINT msg, WPARAM wParam, LPARAM lParam);
+
+                private:
+                    IGNITE_NO_COPY_ASSIGNMENT(DsnConfigurationWindow)
+
+                    /**
+                     * Retrieves current values from the children and stores
+                     * them to the specified configuration.
+                     *
+                     * @param cfg Configuration.
+                     */
+                    void RetrieveParameters(config::Configuration& cfg) const;
+
+                    /** Window width. */
+                    int width;
+
+                    /** Window height. */
+                    int height;
+
+                    /** Connection settings group box. */
+                    std::auto_ptr<Window> connectionSettingsGroupBox;
+
+                    /** DSN name edit field label. */
+                    std::auto_ptr<Window> nameLabel;
+
+                    /** DSN name edit field. */
+                    std::auto_ptr<Window> nameEdit;
+
+                    /** DSN address edit field label. */
+                    std::auto_ptr<Window> addressLabel;
+
+                    /** DSN address edit field. */
+                    std::auto_ptr<Window> addressEdit;
+
+                    /** DSN cache edit field label. */
+                    std::auto_ptr<Window> cacheLabel;
+
+                    /** DSN cache edit field. */
+                    std::auto_ptr<Window> cacheEdit;
+
+                    /** Distributed joins CheckBox. */
+                    std::auto_ptr<Window> distributedJoinsCheckBox;
+
+                    /** Enforce join order CheckBox. */
+                    std::auto_ptr<Window> enforceJoinOrderCheckBox;
+
+                    /** Protocol version edit field. */
+                    std::auto_ptr<Window> protocolVersionLabel;
+
+                    /** Protocol verion ComboBox. */
+                    std::auto_ptr<Window> protocolVersionComboBox;
+
+                    /** Ok button. */
+                    std::auto_ptr<Window> okButton;
+
+                    /** Cancel button. */
+                    std::auto_ptr<Window> cancelButton;
+
+                    /** Configuration. */
+                    config::Configuration& config;
+
+                    /** Flag indicating whether OK option was selected. */
+                    bool accepted;
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_ODBC_SYSTEM_UI_DSN_CONFIGURATION_WINDOW
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h
new file mode 100644
index 0000000..1502c07
--- /dev/null
+++ b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/custom_window.h
@@ -0,0 +1,189 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW
+#define _IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW
+
+#include "ignite/odbc/system/ui/window.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        namespace system
+        {
+            namespace ui
+            {
+                /**
+                 * Application execution result.
+                 */
+                enum Result
+                {
+                    RESULT_OK,
+                    RESULT_CANCEL
+                };
+
+                /**
+                 * Process UI messages in current thread.
+                 * Blocks until quit message has been received.
+                 *
+                 * @param window Main window.
+                 * @return Application execution result.
+                 */
+                Result ProcessMessages(Window& window);
+
+                /**
+                 * Window class.
+                 */
+                class CustomWindow : public Window
+                {
+                public:
+                    /**
+                     * Constructor.
+                     *
+                     * @param parent Parent window.
+                     * @param className Window class name.
+                     * @param title Window title.
+                     * @param callback Event processing function.
+                     */
+                    CustomWindow(Window* parent, const char* className, const char* title);
+
+                    /**
+                     * Destructor.
+                     */
+                    virtual ~CustomWindow();
+
+                    /**
+                     * Callback which is called upon receiving new message.
+                     * Pure virtual. Should be defined by user.
+                     *
+                     * @param msg Message.
+                     * @param wParam Word-sized parameter.
+                     * @param lParam Long parameter.
+                     * @return Should return true if the message has been
+                     *     processed by the handler and false otherwise.
+                     */
+                    virtual bool OnMessage(UINT msg, WPARAM wParam, LPARAM lParam) = 0;
+
+                    /**
+                     * Callback that is called upon window creation.
+                     */
+                    virtual void OnCreate() = 0;
+
+                    /**
+                     * Create child group box window.
+                     *
+                     * @param posX Position by X coordinate.
+                     * @param posY Position by Y coordinate.
+                     * @param sizeX Size by X coordinate.
+                     * @param sizeY Size by Y coordinate.
+                     * @param title Title.
+                     * @param id ID to be assigned to the created window.
+                     * @return Auto pointer containing new window.
+                     */
+                    std::auto_ptr<Window> CreateGroupBox(int posX, int posY,
+                        int sizeX, int sizeY, const char* title, int id);
+
+                    /**
+                     * Create child label window.
+                     *
+                     * @param posX Position by X coordinate.
+                     * @param posY Position by Y coordinate.
+                     * @param sizeX Size by X coordinate.
+                     * @param sizeY Size by Y coordinate.
+                     * @param title Title.
+                     * @param id ID to be assigned to the created window.
+                     * @return Auto pointer containing new window.
+                     */
+                    std::auto_ptr<Window> CreateLabel(int posX, int posY,
+                        int sizeX, int sizeY, const char* title, int id);
+
+                    /**
+                     * Create child Edit window.
+                     *
+                     * @param posX Position by X coordinate.
+                     * @param posY Position by Y coordinate.
+                     * @param sizeX Size by X coordinate.
+                     * @param sizeY Size by Y coordinate.
+                     * @param title Title.
+                     * @param id ID to be assigned to the created window.
+                     * @return Auto pointer containing new window.
+                     */
+                    std::auto_ptr<Window> CreateEdit(int posX, int posY,
+                        int sizeX, int sizeY, const char* title, int id, int style = 0);
+
+                    /**
+                     * Create child button window.
+                     *
+                     * @param posX Position by X coordinate.
+                     * @param posY Position by Y coordinate.
+                     * @param sizeX Size by X coordinate.
+                     * @param sizeY Size by Y coordinate.
+                     * @param title Title.
+                     * @param id ID to be assigned to the created window.
+                     * @return Auto pointer containing new window.
+                     */
+                    std::auto_ptr<Window> CreateButton(int posX, int posY,
+                        int sizeX, int sizeY, const char* title, int id);
+
+                    /**
+                     * Create child CheckBox window.
+                     *
+                     * @param posX Position by X coordinate.
+                     * @param posY Position by Y coordinate.
+                     * @param sizeX Size by X coordinate.
+                     * @param sizeY Size by Y coordinate.
+                     * @param title Title.
+                     * @param id ID to be assigned to the created window.
+                     * @return Auto pointer containing new window.
+                     */
+                    std::auto_ptr<Window> CreateCheckBox(int posX, int posY,
+                        int sizeX, int sizeY, const char* title, int id, bool state);
+
+                    /**
+                     * Create child ComboBox window.
+                     *
+                     * @param posX Position by X coordinate.
+                     * @param posY Position by Y coordinate.
+                     * @param sizeX Size by X coordinate.
+                     * @param sizeY Size by Y coordinate.
+                     * @param title Title.
+                     * @param id ID to be assigned to the created window.
+                     * @return Auto pointer containing new window.
+                     */
+                    std::auto_ptr<Window> CreateComboBox(int posX, int posY,
+                        int sizeX, int sizeY, const char* title, int id);
+                private:
+                    IGNITE_NO_COPY_ASSIGNMENT(CustomWindow)
+
+                    /**
+                     * Static callback.
+                     *
+                     * @param hwnd Window handle.
+                     * @param msg Message.
+                     * @param wParam Word-sized parameter.
+                     * @param lParam Long parameter.
+                     * @return Operation result.
+                     */
+                    static LRESULT CALLBACK WndProc(HWND hwnd, UINT msg, WPARAM wParam, LPARAM lParam);
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_ODBC_SYSTEM_UI_CUSTOM_WINDOW
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h
new file mode 100644
index 0000000..32a54b2
--- /dev/null
+++ b/modules/platforms/cpp/odbc/os/win/include/ignite/odbc/system/ui/window.h
@@ -0,0 +1,201 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_SYSTEM_UI_WINDOW
+#define _IGNITE_ODBC_SYSTEM_UI_WINDOW
+
+#include "ignite/odbc/utility.h"
+#include "ignite/odbc/system/odbc_constants.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        namespace system
+        {
+            namespace ui
+            {
+                /**
+                 * Get handle for the current module.
+                 *
+                 * @return Handle for the current module.
+                 */
+                HINSTANCE GetHInstance();
+
+                /**
+                 * Window class.
+                 */
+                class Window
+                {
+                public:
+                    /**
+                     * Constructor for a new window that is going to be created.
+                     *
+                     * @param parent Parent window handle.
+                     * @param className Window class name.
+                     * @param title Window title.
+                     * @param callback Event processing function.
+                     */
+                    Window(Window* parent, const char* className, const char* title);
+
+                    /**
+                     * Constructor for the existing window.
+                     *
+                     * @param handle Window handle.
+                     */
+                    Window(HWND handle);
+
+                    /**
+                     * Destructor.
+                     */
+                    virtual ~Window();
+
+                    /**
+                     * Create window.
+                     *
+                     * @param style Window style.
+                     * @param posX Window x position.
+                     * @param posY Window y position.
+                     * @param width Window width.
+                     * @param height Window height.
+                     * @param id ID for child window.
+                     */
+                    void Create(DWORD style, int posX, int posY, int width, int height, int id);
+
+                    /**
+                     * Show window.
+                     */
+                    void Show();
+
+                    /**
+                     * Update window.
+                     */
+                    void Update();
+
+                    /**
+                     * Destroy window.
+                     */
+                    void Destroy();
+
+                    /**
+                     * Get window handle.
+                     *
+                     * @return Window handle.
+                     */
+                    HWND GetHandle() const
+                    {
+                        return handle;
+                    }
+
+                    /**
+                     * Get window text.
+                     *
+                     * @param text Text.
+                     */
+                    void GetText(std::string& text) const;
+
+                    /**
+                     * Set window text.
+                     *
+                     * @param text Text.
+                     */
+                    void SetText(const std::string& text) const;
+
+                    /**
+                     * Get CheckBox state.
+                     *
+                     * @param True if checked.
+                     */
+                    bool IsChecked() const;
+
+                    /**
+                     * Set CheckBox state.
+                     *
+                     * @param state True if checked.
+                     */
+                    void SetChecked(bool state);
+
+                    /**
+                     * Add string.
+                     *
+                     * @param str String.
+                     */
+                    void AddString(const std::string& str);
+
+                    /**
+                     * Set current ComboBox selection.
+                     *
+                     * @param idx List index.
+                     */
+                    void SetSelection(int idx);
+
+                    /**
+                     * Get current ComboBox selection.
+                     *
+                     * @return idx List index.
+                     */
+                    int GetSelection() const;
+
+                    /**
+                     * Set enabled.
+                     *
+                     * @param enabled Enable flag.
+                     */
+                    void SetEnabled(bool enabled);
+
+                    /**
+                     * Check if the window is enabled.
+                     *
+                     * @return True if enabled.
+                     */
+                    bool IsEnabled() const;
+
+                protected:
+                    /**
+                     * Set window handle.
+                     *
+                     * @param value Window handle.
+                     */
+                    void SetHandle(HWND value)
+                    {
+                        handle = value;
+                    }
+
+                    /** Window class name. */
+                    std::string className;
+
+                    /** Window title. */
+                    std::string title;
+
+                    /** Window handle. */
+                    HWND handle;
+
+                    /** Specifies whether window has been created by the thread and needs destruction. */
+                    bool created;
+
+                    /** Window parent. */
+                    Window* parent;
+
+                private:
+                    IGNITE_NO_COPY_ASSIGNMENT(Window)
+                };
+            }
+        }
+    }
+}
+
+#endif //_IGNITE_ODBC_SYSTEM_UI_WINDOW
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp
new file mode 100644
index 0000000..1e855a1
--- /dev/null
+++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/custom_window.cpp
@@ -0,0 +1,184 @@
+/*
+ * 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.
+ */
+
+#include <Windowsx.h>
+
+#include "ignite/odbc/system/ui/custom_window.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        namespace system
+        {
+            namespace ui
+            {
+                Result ProcessMessages(Window& window)
+                {
+                    MSG msg;
+
+                    while (GetMessage(&msg, NULL, 0, 0) > 0)
+                    {
+                        if (!IsDialogMessage(window.GetHandle(), &msg))
+                        {
+                            TranslateMessage(&msg);
+
+                            DispatchMessage(&msg);
+                        }
+                    }
+
+                    return static_cast<Result>(msg.wParam);
+                }
+
+                LRESULT CALLBACK CustomWindow::WndProc(HWND hwnd, UINT msg, WPARAM wParam, LPARAM lParam)
+                {
+                    CustomWindow* window = reinterpret_cast<CustomWindow*>(GetWindowLongPtr(hwnd, GWLP_USERDATA));
+
+                    switch (msg)
+                    {
+                        case WM_NCCREATE:
+                        {
+                            _ASSERT(lParam != NULL);
+
+                            CREATESTRUCT* createStruct = reinterpret_cast<CREATESTRUCT*>(lParam);
+
+                            LONG_PTR longSelfPtr = reinterpret_cast<LONG_PTR>(createStruct->lpCreateParams);
+
+                            SetWindowLongPtr(hwnd, GWLP_USERDATA, longSelfPtr);
+
+                            return DefWindowProc(hwnd, msg, wParam, lParam);
+                        }
+
+                        case WM_CREATE:
+                        {
+                            _ASSERT(window != NULL);
+
+                            window->SetHandle(hwnd);
+
+                            window->OnCreate();
+
+                            return 0;
+                        }
+
+                        default:
+                            break;
+                    }
+
+                    if (window && window->OnMessage(msg, wParam, lParam))
+                        return 0;
+
+                    return DefWindowProc(hwnd, msg, wParam, lParam);
+                }
+
+                CustomWindow::CustomWindow(Window* parent, const char* className, const char* title) :
+                    Window(parent, className, title)
+                {
+                    WNDCLASS wcx;
+
+                    wcx.style = CS_HREDRAW | CS_VREDRAW;
+                    wcx.lpfnWndProc = WndProc;
+                    wcx.cbClsExtra = 0;
+                    wcx.cbWndExtra = 0;
+                    wcx.hInstance = GetHInstance();
+                    wcx.hIcon = NULL;
+                    wcx.hCursor = LoadCursor(NULL, IDC_ARROW);
+                    wcx.hbrBackground = (HBRUSH)COLOR_WINDOW;
+                    wcx.lpszMenuName = NULL;
+                    wcx.lpszClassName = className;
+
+
+                    if (!RegisterClass(&wcx))
+                    {
+                        std::stringstream buf;
+
+                        buf << "Can not register window class, error code: " << GetLastError();
+
+                        throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str());
+                    }
+                }
+
+                CustomWindow::~CustomWindow()
+                {
+                    UnregisterClass(className.c_str(), GetHInstance());
+                }
+
+                std::auto_ptr<Window> CustomWindow::CreateGroupBox(int posX, int posY,
+                    int sizeX, int sizeY, const char* title, int id)
+                {
+                    std::auto_ptr<Window> child(new Window(this, "Button", title));
+
+                    child->Create(WS_CHILD | WS_VISIBLE | BS_GROUPBOX, posX, posY, sizeX, sizeY, id);
+
+                    return child;
+                }
+
+                std::auto_ptr<Window> CustomWindow::CreateLabel(int posX, int posY,
+                    int sizeX, int sizeY, const char* title, int id)
+                {
+                    std::auto_ptr<Window> child(new Window(this, "Static", title));
+
+                    child->Create(WS_CHILD | WS_VISIBLE, posX, posY, sizeX, sizeY, id);
+
+                    return child;
+                }
+
+                std::auto_ptr<Window> CustomWindow::CreateEdit(int posX, int posY,
+                    int sizeX, int sizeY, const char* title, int id, int style)
+                {
+                    std::auto_ptr<Window> child(new Window(this, "Edit", title));
+
+                    child->Create(WS_CHILD | WS_VISIBLE | WS_BORDER | ES_AUTOHSCROLL | WS_TABSTOP | style,
+                        posX, posY, sizeX, sizeY, id);
+
+                    return child;
+                }
+
+                std::auto_ptr<Window> CustomWindow::CreateButton(int posX, int posY,
+                    int sizeX, int sizeY, const char* title, int id)
+                {
+                    std::auto_ptr<Window> child(new Window(this, "Button", title));
+
+                    child->Create(WS_CHILD | WS_VISIBLE | WS_TABSTOP, posX, posY, sizeX, sizeY, id);
+
+                    return child;
+                }
+
+                std::auto_ptr<Window> CustomWindow::CreateCheckBox(int posX, int posY,
+                    int sizeX, int sizeY, const char* title, int id, bool state)
+                {
+                    std::auto_ptr<Window> child(new Window(this, "Button", title));
+
+                    child->Create(WS_CHILD | WS_VISIBLE | BS_CHECKBOX, posX, posY, sizeX, sizeY, id);
+
+                    child->SetChecked(state);
+
+                    return child;
+                }
+
+                std::auto_ptr<Window> CustomWindow::CreateComboBox(int posX, int posY,
+                    int sizeX, int sizeY, const char * title, int id)
+                {
+                    std::auto_ptr<Window> child(new Window(this, "Combobox", title));
+
+                    child->Create(WS_CHILD | WS_VISIBLE | CBS_DROPDOWNLIST, posX, posY, sizeX, sizeY, id);
+
+                    return child;
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
new file mode 100644
index 0000000..663333a
--- /dev/null
+++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
@@ -0,0 +1,305 @@
+/*
+ * 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.
+ */
+
+#include <Windowsx.h>
+
+#include "ignite/odbc/system/ui/dsn_configuration_window.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        namespace system
+        {
+            namespace ui
+            {
+                DsnConfigurationWindow::DsnConfigurationWindow(Window* parent, config::Configuration& config):
+                    CustomWindow(parent, "IgniteConfigureDsn", "Configure Apache Ignite DSN"),
+                    width(360),
+                    height(230),
+                    connectionSettingsGroupBox(),
+                    nameLabel(),
+                    nameEdit(),
+                    addressLabel(),
+                    addressEdit(),
+                    cacheLabel(),
+                    cacheEdit(),
+                    distributedJoinsCheckBox(),
+                    enforceJoinOrderCheckBox(),
+                    protocolVersionLabel(),
+                    protocolVersionComboBox(),
+                    okButton(),
+                    cancelButton(),
+                    config(config),
+                    accepted(false)
+                {
+                    // No-op.
+                }
+
+                DsnConfigurationWindow::~DsnConfigurationWindow()
+                {
+                    // No-op.
+                }
+
+                void DsnConfigurationWindow::Create()
+                {
+                    // Finding out parent position.
+                    RECT parentRect;
+                    GetWindowRect(parent->GetHandle(), &parentRect);
+
+                    // Positioning window to the center of parent window.
+                    const int posX = parentRect.left + (parentRect.right - parentRect.left - width) / 2;
+                    const int posY = parentRect.top + (parentRect.bottom - parentRect.top - height) / 2;
+
+                    RECT desiredRect = {posX, posY, posX + width, posY + height};
+                    AdjustWindowRect(&desiredRect, WS_BORDER | WS_CAPTION | WS_SYSMENU | WS_THICKFRAME, FALSE);
+
+                    Window::Create(WS_OVERLAPPED | WS_SYSMENU, desiredRect.left, desiredRect.top,
+                        desiredRect.right - desiredRect.left, desiredRect.bottom - desiredRect.top, 0);
+
+                    if (!handle)
+                    {
+                        std::stringstream buf;
+
+                        buf << "Can not create window, error code: " << GetLastError();
+
+                        throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str());
+                    }
+                }
+
+                void DsnConfigurationWindow::OnCreate()
+                {
+                    int margin = 10;
+                    int interval = 10;
+
+                    int labelSizeX = 80;
+                    int labelPosX = margin + interval;
+
+                    int editSizeX = width - labelSizeX - 2 * margin - 3 * interval;
+                    int editPosX = margin + labelSizeX + 2 * interval;
+
+                    int rowSize = 20;
+                    int rowPos = margin + 2 * interval;
+
+                    int checkBoxSize = (editSizeX - interval) / 2;
+
+                    int sectionBegin = margin;
+
+                    const char* val = config.GetDsn().c_str();
+                    nameLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "DSN name:", ID_NAME_LABEL);
+                    nameEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_NAME_EDIT);
+
+                    rowPos += interval + rowSize;
+
+                    val = config.GetAddress().c_str();
+                    addressLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "Address:", ID_ADDRESS_LABEL);
+                    addressEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_ADDRESS_EDIT);
+
+                    rowPos += interval + rowSize;
+
+                    val = config.GetCache().c_str();
+                    cacheLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize, "Cache name:", ID_CACHE_LABEL);
+                    cacheEdit = CreateEdit(editPosX, rowPos, editSizeX, rowSize, val, ID_CACHE_EDIT);
+
+                    rowPos += interval + rowSize;
+
+                    protocolVersionLabel = CreateLabel(labelPosX, rowPos, labelSizeX, rowSize,
+                        "Protocol version:", ID_PROTOCOL_VERSION_LABEL);
+                    protocolVersionComboBox = CreateComboBox(editPosX, rowPos, editSizeX, rowSize,
+                        "Protocol version", ID_PROTOCOL_VERSION_COMBO_BOX);
+
+                    int id = 0;
+
+                    const ProtocolVersion::StringToVersionMap& versionMap = ProtocolVersion::GetMap();
+
+                    ProtocolVersion::StringToVersionMap::const_iterator it;
+                    for (it = versionMap.begin(); it != versionMap.end(); ++it)
+                    {
+                        protocolVersionComboBox->AddString(it->first);
+
+                        if (it->second == config.GetProtocolVersion())
+                            protocolVersionComboBox->SetSelection(id);
+
+                        ++id;
+                    }
+
+                    rowPos += interval + rowSize;
+
+                    distributedJoinsCheckBox = CreateCheckBox(editPosX, rowPos, checkBoxSize, rowSize,
+                        "Distributed Joins", ID_DISTRIBUTED_JOINS_CHECK_BOX, config.IsDistributedJoins());
+
+                    enforceJoinOrderCheckBox = CreateCheckBox(editPosX + checkBoxSize + interval, rowPos, checkBoxSize,
+                        rowSize, "Enforce Join Order", ID_ENFORCE_JOIN_ORDER_CHECK_BOX, config.IsEnforceJoinOrder());
+
+                    if (!config.GetProtocolVersion().IsDistributedJoinsSupported())
+                    {
+                        distributedJoinsCheckBox->SetEnabled(false);
+                        enforceJoinOrderCheckBox->SetEnabled(false);
+                    }
+
+                    rowPos += interval * 2 + rowSize;
+
+                    connectionSettingsGroupBox = CreateGroupBox(margin, sectionBegin, width - 2 * margin,
+                        rowPos - interval - sectionBegin, "Connection settings", ID_CONNECTION_SETTINGS_GROUP_BOX);
+
+                    int buttonSizeX = 80;
+                    int cancelPosX = width - margin - buttonSizeX;
+                    int okPosX = cancelPosX - interval - buttonSizeX;
+
+                    rowSize = 25;
+
+                    okButton = CreateButton(okPosX, rowPos, buttonSizeX, rowSize, "Ok", ID_OK_BUTTON);
+                    cancelButton = CreateButton(cancelPosX, rowPos, buttonSizeX, rowSize, "Cancel", ID_CANCEL_BUTTON);
+                }
+
+                bool DsnConfigurationWindow::OnMessage(UINT msg, WPARAM wParam, LPARAM lParam)
+                {
+                    switch (msg)
+                    {
+                        case WM_COMMAND:
+                        {
+                            switch (LOWORD(wParam))
+                            {
+                                case ID_OK_BUTTON:
+                                {
+                                    try
+                                    {
+                                        RetrieveParameters(config);
+
+                                        accepted = true;
+
+                                        PostMessage(GetHandle(), WM_CLOSE, 0, 0);
+                                    }
+                                    catch (IgniteError& err)
+                                    {
+                                        MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK);
+                                    }
+
+                                    break;
+                                }
+
+                                case ID_PROTOCOL_VERSION_COMBO_BOX:
+                                {
+                                    if (HIWORD(wParam) == CBN_SELCHANGE)
+                                    {
+                                        std::string text;
+
+                                        protocolVersionComboBox->GetText(text);
+
+                                        ProtocolVersion version = ProtocolVersion::FromString(text);
+
+                                        if (!version.IsUnknown() && !version.IsDistributedJoinsSupported())
+                                        {
+                                            distributedJoinsCheckBox->SetEnabled(false);
+                                            enforceJoinOrderCheckBox->SetEnabled(false);
+                                        }
+                                        else
+                                        {
+                                            distributedJoinsCheckBox->SetEnabled(true);
+                                            enforceJoinOrderCheckBox->SetEnabled(true);
+                                        }
+                                    }
+
+                                    break;
+                                }
+
+                                case IDCANCEL:
+                                case ID_CANCEL_BUTTON:
+                                {
+                                    PostMessage(GetHandle(), WM_CLOSE, 0, 0);
+
+                                    break;
+                                }
+
+                                case ID_DISTRIBUTED_JOINS_CHECK_BOX:
+                                {
+                                    distributedJoinsCheckBox->SetChecked(!distributedJoinsCheckBox->IsChecked());
+
+                                    break;
+                                }
+
+                                case ID_ENFORCE_JOIN_ORDER_CHECK_BOX:
+                                {
+                                    enforceJoinOrderCheckBox->SetChecked(!enforceJoinOrderCheckBox->IsChecked());
+
+                                    break;
+                                }
+
+                                default:
+                                    return false;
+                            }
+
+                            break;
+                        }
+
+                        case WM_DESTROY:
+                        {
+                            PostQuitMessage(accepted ? RESULT_OK : RESULT_CANCEL);
+
+                            break;
+                        }
+
+                        default:
+                            return false;
+                    }
+
+                    return true;
+                }
+
+                void DsnConfigurationWindow::RetrieveParameters(config::Configuration& cfg) const
+                {
+                    std::string dsn;
+                    std::string address;
+                    std::string cache;
+                    std::string version;
+
+                    bool distributedJoins;
+                    bool enforceJoinOrder;
+
+                    nameEdit->GetText(dsn);
+                    addressEdit->GetText(address);
+                    cacheEdit->GetText(cache);
+                    protocolVersionComboBox->GetText(version);
+
+                    common::StripSurroundingWhitespaces(address);
+                    common::StripSurroundingWhitespaces(dsn);
+
+                    distributedJoins = distributedJoinsCheckBox->IsEnabled() && distributedJoinsCheckBox->IsChecked();
+                    enforceJoinOrder = enforceJoinOrderCheckBox->IsEnabled() && enforceJoinOrderCheckBox->IsChecked();
+
+                    LOG_MSG("Retriving arguments:\n");
+                    LOG_MSG("DSN:                %s\n", dsn.c_str());
+                    LOG_MSG("Address:            %s\n", address.c_str());
+                    LOG_MSG("Cache:              %s\n", cache.c_str());
+                    LOG_MSG("Protocol version:   %s\n", version.c_str());
+                    LOG_MSG("Distributed Joins:  %s\n", distributedJoins ? "true" : "false");
+                    LOG_MSG("Enforce Join Order: %s\n", enforceJoinOrder ? "true" : "false");
+
+                    if (dsn.empty())
+                        throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "DSN name can not be empty.");
+
+                    cfg.SetDsn(dsn);
+                    cfg.SetAddress(address);
+                    cfg.SetCache(cache);
+                    cfg.SetProtocolVersion(version);
+                    cfg.SetDistributedJoins(distributedJoins);
+                    cfg.SetEnforceJoinOrder(enforceJoinOrder);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp
new file mode 100644
index 0000000..1143f01
--- /dev/null
+++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/window.cpp
@@ -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.
+ */
+
+#include <Windowsx.h>
+
+#include "ignite/odbc/system/ui/window.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        namespace system
+        {
+            namespace ui
+            {
+                HINSTANCE GetHInstance()
+                {
+                    HINSTANCE hInstance = GetModuleHandle(TARGET_MODULE_FULL_NAME);
+
+                    if (hInstance == NULL)
+                    {
+                        std::stringstream buf;
+
+                        buf << "Can not get hInstance for the module, error code: " << GetLastError();
+
+                        throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str());
+                    }
+
+                    return hInstance;
+                }
+
+                Window::Window(Window* parent, const char* className, const char* title) :
+                    className(className),
+                    title(title),
+                    handle(NULL),
+                    created(false),
+                    parent(parent)
+                {
+                    // No-op.
+                }
+
+                Window::Window(HWND handle) :
+                    className(),
+                    title(),
+                    handle(handle),
+                    created(false),
+                    parent(0)
+                {
+                    // No-op.
+                }
+
+                Window::~Window()
+                {
+                    if (created)
+                        Destroy();
+                }
+
+                void Window::Create(DWORD style, int posX, int posY, int width, int height, int id)
+                {
+                    if (handle)
+                    {
+                        std::stringstream buf;
+
+                        buf << "Window already created, error code: " << GetLastError();
+
+                        throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str());
+                    }
+
+                    handle = CreateWindow(
+                        className.c_str(),
+                        title.c_str(),
+                        style,
+                        posX,
+                        posY,
+                        width,
+                        height,
+                        parent ? parent->GetHandle() : NULL,
+                        reinterpret_cast<HMENU>(id),
+                        GetHInstance(),
+                        this
+                    );
+
+                    if (!handle)
+                    {
+                        std::stringstream buf;
+
+                        buf << "Can not create window, error code: " << GetLastError();
+
+                        throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str());
+                    }
+
+                    created = true;
+
+                    HGDIOBJ hfDefault = GetStockObject(DEFAULT_GUI_FONT);
+
+                    SendMessage(GetHandle(), WM_SETFONT, (WPARAM)hfDefault, MAKELPARAM(FALSE, 0));
+                }
+
+                void Window::Show()
+                {
+                    ShowWindow(handle, SW_SHOW);
+                }
+
+                void Window::Update()
+                {
+                    UpdateWindow(handle);
+                }
+
+                void Window::Destroy()
+                {
+                    if (handle)
+                        DestroyWindow(handle);
+
+                    handle = NULL;
+                }
+
+                void Window::GetText(std::string& text) const
+                {
+                    int len = GetWindowTextLength(handle);
+
+                    if (len <= 0)
+                    {
+                        text.clear();
+
+                        return;
+                    }
+
+                    text.resize(len + 1);
+
+                    if (!GetWindowText(handle, &text[0], len + 1))
+                        text.clear();
+
+                    text.resize(len);
+                }
+
+                void Window::SetText(const std::string& text) const
+                {
+                    SNDMSG(handle, WM_SETTEXT, 0, reinterpret_cast<LPARAM>(text.c_str()));
+                }
+
+                bool Window::IsChecked() const
+                {
+                    return Button_GetCheck(handle) == BST_CHECKED;
+                }
+
+                void Window::SetChecked(bool state)
+                {
+                    Button_SetCheck(handle, state ? BST_CHECKED : BST_UNCHECKED);
+                }
+
+                void Window::AddString(const std::string & str)
+                {
+                    SNDMSG(handle, CB_ADDSTRING, 0, reinterpret_cast<LPARAM>(str.c_str()));
+                }
+
+                void Window::SetSelection(int idx)
+                {
+                    SNDMSG(handle, CB_SETCURSEL, static_cast<WPARAM>(idx), 0);
+                }
+
+                int Window::GetSelection() const
+                {
+                    return static_cast<int>(SNDMSG(handle, CB_GETCURSEL, 0, 0));
+                }
+
+                void Window::SetEnabled(bool enabled)
+                {
+                    EnableWindow(GetHandle(), enabled);
+                }
+
+                bool Window::IsEnabled() const
+                {
+                    return IsWindowEnabled(GetHandle()) != 0;
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
new file mode 100644
index 0000000..f432a40
--- /dev/null
+++ b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
@@ -0,0 +1,218 @@
+/*
+ * 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.
+ */
+
+#include "ignite/odbc/utility.h"
+#include "ignite/odbc/system/odbc_constants.h"
+
+#include "ignite/odbc/dsn_config.h"
+#include "ignite/odbc/system/ui/window.h"
+#include "ignite/odbc/system/ui/dsn_configuration_window.h"
+
+using ignite::odbc::config::Configuration;
+
+/**
+ * Display configuration window for user to configure DSN.
+ *
+ * @param hwndParent Parent window handle.
+ * @param config Output configuration.
+ * @return True on success and false on fail.
+ */
+bool DisplayConfigureDsnWindow(HWND hwndParent, Configuration& config)
+{
+    using namespace ignite::odbc::system::ui;
+
+    if (!hwndParent)
+        return false;
+
+    try
+    {
+        Window parent(hwndParent);
+
+        DsnConfigurationWindow window(&parent, config);
+
+        window.Create();
+
+        window.Show();
+        window.Update();
+
+        return ProcessMessages(window) == RESULT_OK;
+    }
+    catch (const ignite::IgniteError& err)
+    {
+        std::stringstream buf;
+
+        buf << "Message: " << err.GetText() << ", Code: " << err.GetCode();
+
+        std::string message = buf.str();
+
+        MessageBox(NULL, message.c_str(), "Error!", MB_ICONEXCLAMATION | MB_OK);
+
+        SQLPostInstallerError(err.GetCode(), err.GetText());
+    }
+
+    return false;
+}
+
+/**
+ * Register DSN with specified configuration.
+ *
+ * @param config Configuration.
+ * @param driver Driver.
+ * @return True on success and false on fail.
+ */
+bool RegisterDsn(const Configuration& config, LPCSTR driver)
+{
+    using namespace ignite::odbc::config;
+    using ignite::common::LexicalCast;
+
+    typedef Configuration::ArgumentMap ArgMap;
+
+    const char* dsn = config.GetDsn().c_str();
+
+    try
+    {
+        if (!SQLWriteDSNToIni(dsn, driver))
+            ignite::odbc::ThrowLastSetupError();
+
+        const ArgMap& map = config.GetMap();
+
+        std::set<std::string> ignore;
+
+        ignore.insert(Configuration::Key::dsn);
+        ignore.insert(Configuration::Key::driver);
+
+        for (ArgMap::const_iterator it = map.begin(); it != map.end(); ++it)
+        {
+            const std::string& key = it->first;
+            const std::string& value = it->second;
+
+            if (ignore.find(key) != ignore.end())
+                continue;
+
+            ignite::odbc::WriteDsnString(dsn, key.c_str(), value.c_str());
+        }
+
+        return true;
+    }
+    catch (ignite::IgniteError& err)
+    {
+        MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK);
+
+        SQLPostInstallerError(err.GetCode(), err.GetText());
+    }
+
+    return false;
+}
+
+/**
+ * Unregister specified DSN.
+ *
+ * @param dsn DSN name.
+ * @return True on success and false on fail.
+ */
+bool UnregisterDsn(const char* dsn)
+{
+    try
+    {
+        if (!SQLRemoveDSNFromIni(dsn))
+            ignite::odbc::ThrowLastSetupError();
+
+        return true;
+    }
+    catch (ignite::IgniteError& err)
+    {
+        MessageBox(NULL, err.GetText(), "Error!", MB_ICONEXCLAMATION | MB_OK);
+
+        SQLPostInstallerError(err.GetCode(), err.GetText());
+    }
+
+    return false;
+}
+
+BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attributes)
+{
+    using namespace ignite::odbc;
+
+    LOG_MSG("ConfigDSN called\n");
+
+    Configuration config;
+
+    LOG_MSG("Attributes: %s\n", attributes);
+
+    config.FillFromConfigAttributes(attributes);
+
+    if (!SQLValidDSN(config.GetDsn().c_str()))
+        return FALSE;
+
+    LOG_MSG("Driver: %s\n", driver);
+    LOG_MSG("Attributes: %s\n", attributes);
+
+    LOG_MSG("DSN: %s\n", config.GetDsn().c_str());
+
+    switch (req)
+    {
+        case ODBC_ADD_DSN:
+        {
+            LOG_MSG("ODBC_ADD_DSN\n");
+
+            if (!DisplayConfigureDsnWindow(hwndParent, config))
+                return FALSE;
+
+            if (!RegisterDsn(config, driver))
+                return FALSE;
+
+            break;
+        }
+
+        case ODBC_CONFIG_DSN:
+        {
+            LOG_MSG("ODBC_CONFIG_DSN\n");
+
+            std::string dsn = config.GetDsn();
+
+            Configuration loaded(config);
+
+            ReadDsnConfiguration(dsn.c_str(), loaded);
+
+            if (!DisplayConfigureDsnWindow(hwndParent, loaded))
+                return FALSE;
+
+            if (!RegisterDsn(loaded, driver))
+                return FALSE;
+
+            if (loaded.GetDsn() != dsn && !UnregisterDsn(dsn.c_str()))
+                return FALSE;
+
+            break;
+        }
+
+        case ODBC_REMOVE_DSN:
+        {
+            LOG_MSG("ODBC_REMOVE_DSN\n");
+
+            if (!UnregisterDsn(config.GetDsn().c_str()))
+                return FALSE;
+
+            break;
+        }
+
+        default:
+            return FALSE;
+    }
+
+    return TRUE;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
index 0e0f0d3..d48db21 100644
--- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
+++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
@@ -93,8 +93,8 @@
       <WarningLevel>Level3</WarningLevel>
       <Optimization>Disabled</Optimization>
       <SDLCheck>false</SDLCheck>
-      <AdditionalIncludeDirectories>$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src</AdditionalIncludeDirectories>
-      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;_DEBUG;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src</AdditionalIncludeDirectories>
+      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";_DEBUG;ODBC_DEBUG;ODBC_LOG_PATH="D:\\odbc.log";%(PreprocessorDefinitions)</PreprocessorDefinitions>
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
@@ -107,8 +107,8 @@
       <WarningLevel>Level3</WarningLevel>
       <Optimization>Disabled</Optimization>
       <SDLCheck>false</SDLCheck>
-      <AdditionalIncludeDirectories>$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src</AdditionalIncludeDirectories>
-      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;_DEBUG;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src</AdditionalIncludeDirectories>
+      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";_DEBUG;ODBC_DEBUG;ODBC_LOG_PATH="D:\\odbc.log";%(PreprocessorDefinitions)</PreprocessorDefinitions>
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
@@ -123,8 +123,8 @@
       <FunctionLevelLinking>true</FunctionLevelLinking>
       <IntrinsicFunctions>true</IntrinsicFunctions>
       <SDLCheck>false</SDLCheck>
-      <AdditionalIncludeDirectories>$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src</AdditionalIncludeDirectories>
-      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src</AdditionalIncludeDirectories>
+      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";%(PreprocessorDefinitions)</PreprocessorDefinitions>
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
@@ -141,8 +141,8 @@
       <FunctionLevelLinking>true</FunctionLevelLinking>
       <IntrinsicFunctions>true</IntrinsicFunctions>
       <SDLCheck>false</SDLCheck>
-      <AdditionalIncludeDirectories>$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win;$(ProjectDir)\..\..\src</AdditionalIncludeDirectories>
-      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;%(PreprocessorDefinitions)</PreprocessorDefinitions>
+      <AdditionalIncludeDirectories>$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\..\binary\include;$(ProjectDir)\..\..\..\binary\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;$(ProjectDir)\..\..\src</AdditionalIncludeDirectories>
+      <PreprocessorDefinitions>_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;IGNITE_FRIEND;TARGET_MODULE_FULL_NAME="$(TargetFileName)";%(PreprocessorDefinitions)</PreprocessorDefinitions>
     </ClCompile>
     <Link>
       <GenerateDebugInformation>true</GenerateDebugInformation>
@@ -153,7 +153,11 @@
     </Link>
   </ItemDefinitionGroup>
   <ItemGroup>
+    <ClCompile Include="..\..\os\win\src\system_dsn.cpp" />
     <ClCompile Include="..\..\os\win\src\system\socket_client.cpp" />
+    <ClCompile Include="..\..\os\win\src\system\ui\custom_window.cpp" />
+    <ClCompile Include="..\..\os\win\src\system\ui\dsn_configuration_window.cpp" />
+    <ClCompile Include="..\..\os\win\src\system\ui\window.cpp" />
     <ClCompile Include="..\..\src\app\application_data_buffer.cpp" />
     <ClCompile Include="..\..\src\app\parameter.cpp" />
     <ClCompile Include="..\..\src\column.cpp" />
@@ -165,6 +169,7 @@
     <ClCompile Include="..\..\src\diagnostic\diagnosable_adapter.cpp" />
     <ClCompile Include="..\..\src\diagnostic\diagnostic_record.cpp" />
     <ClCompile Include="..\..\src\diagnostic\diagnostic_record_storage.cpp" />
+    <ClCompile Include="..\..\src\dsn_config.cpp" />
     <ClCompile Include="..\..\src\entry_points.cpp" />
     <ClCompile Include="..\..\src\environment.cpp" />
     <ClCompile Include="..\..\src\meta\column_meta.cpp" />
@@ -201,6 +206,7 @@
     <ClInclude Include="..\..\include\ignite\odbc\diagnostic\diagnosable_adapter.h" />
     <ClInclude Include="..\..\include\ignite\odbc\diagnostic\diagnostic_record.h" />
     <ClInclude Include="..\..\include\ignite\odbc\diagnostic\diagnostic_record_storage.h" />
+    <ClInclude Include="..\..\include\ignite\odbc\dsn_config.h" />
     <ClInclude Include="..\..\include\ignite\odbc\environment.h" />
     <ClInclude Include="..\..\include\ignite\odbc\message.h" />
     <ClInclude Include="..\..\include\ignite\odbc\meta\column_meta.h" />
@@ -221,8 +227,11 @@
     <ClInclude Include="..\..\include\ignite\odbc\statement.h" />
     <ClInclude Include="..\..\include\ignite\odbc\system\odbc_constants.h" />
     <ClInclude Include="..\..\include\ignite\odbc\system\socket_client.h" />
+    <ClInclude Include="..\..\include\ignite\odbc\system\ui\dsn_configuration_window.h" />
     <ClInclude Include="..\..\include\ignite\odbc\type_traits.h" />
     <ClInclude Include="..\..\include\ignite\odbc\utility.h" />
+    <ClInclude Include="..\..\os\win\include\ignite\odbc\system\ui\custom_window.h" />
+    <ClInclude Include="..\..\os\win\include\ignite\odbc\system\ui\window.h" />
   </ItemGroup>
   <ItemGroup>
     <ProjectReference Include="..\..\..\binary\project\vs\binary.vcxproj">

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
index 9caf483..58764e4 100644
--- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
+++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
@@ -26,6 +26,9 @@
     <Filter Include="Code\diagnostic">
       <UniqueIdentifier>{df33e506-b5d8-423f-bcc5-1825242a3e28}</UniqueIdentifier>
     </Filter>
+    <Filter Include="Code\system\ui">
+      <UniqueIdentifier>{ff144e89-0a10-42c3-97dd-d22bfdbc7abb}</UniqueIdentifier>
+    </Filter>
   </ItemGroup>
   <ItemGroup>
     <ClCompile Include="..\..\src\odbc.cpp">
@@ -115,6 +118,21 @@
     <ClCompile Include="..\..\src\entry_points.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\os\win\src\system\ui\window.cpp">
+      <Filter>Code\system\ui</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\os\win\src\system\ui\custom_window.cpp">
+      <Filter>Code\system\ui</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\os\win\src\system\ui\dsn_configuration_window.cpp">
+      <Filter>Code\system\ui</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\os\win\src\system_dsn.cpp">
+      <Filter>Code\system</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\dsn_config.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\protocol_version.cpp">
       <Filter>Code</Filter>
     </ClCompile>
@@ -227,6 +245,18 @@
     <ClInclude Include="..\..\include\ignite\odbc.h">
       <Filter>Code</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\os\win\include\ignite\odbc\system\ui\window.h">
+      <Filter>Code\system\ui</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\os\win\include\ignite\odbc\system\ui\custom_window.h">
+      <Filter>Code\system\ui</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\odbc\system\ui\dsn_configuration_window.h">
+      <Filter>Code\system\ui</Filter>
+    </ClInclude>
+    <ClInclude Include="..\..\include\ignite\odbc\dsn_config.h">
+      <Filter>Code</Filter>
+    </ClInclude>
     <ClInclude Include="..\..\include\ignite\odbc\protocol_version.h">
       <Filter>Code</Filter>
     </ClInclude>


[03/24] ignite git commit: IGNITE-3736: ODBC: Added support for string scalar functions. This closes #979.

Posted by sb...@apache.org.
IGNITE-3736: ODBC: Added support for string scalar functions. This closes #979.


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

Branch: refs/heads/master
Commit: c02ad87f863bd730e60fb7052547fa0848e7eb77
Parents: 118db2f
Author: isapego <is...@gridgain.com>
Authored: Wed Aug 24 17:21:48 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 24 17:21:48 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/odbc-test/Makefile.am     |   2 +
 .../cpp/odbc-test/config/queries-test.xml       |   7 +-
 .../platforms/cpp/odbc-test/include/Makefile.am |   3 +-
 .../cpp/odbc-test/include/test_utils.h          |  45 ++
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   3 +
 .../project/vs/odbc-test.vcxproj.filters        |   9 +
 .../cpp/odbc-test/src/queries_test.cpp          |  41 +-
 .../odbc-test/src/sql_string_functions_test.cpp | 515 +++++++++++++++++++
 .../platforms/cpp/odbc-test/src/test_utils.cpp  |  36 ++
 .../cpp/odbc/src/config/connection_info.cpp     |  64 +--
 .../platforms/cpp/odbc/src/query/data_query.cpp |   6 +-
 11 files changed, 653 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index 48b193a..cc84a7f 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -61,6 +61,8 @@ ignite_odbc_tests_SOURCES = \
     src/row_test.cpp \
     src/utility_test.cpp \
     src/queries_test.cpp \
+    src/test_utils.cpp \
+    src/sql_string_functions_test.cpp \
     ../odbc/src/cursor.cpp \
     ../odbc/src/config/connection_info.cpp \
     ../odbc/src/app/application_data_buffer.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/config/queries-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml
index f08f86d..67415fb 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml
@@ -26,8 +26,8 @@
         http://www.springframework.org/schema/util
         http://www.springframework.org/schema/util/spring-util.xsd">
     <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <!-- Set to true to enable distributed class loading for examples, default is false. -->
-        <property name="peerClassLoadingEnabled" value="true"/>
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
 
         <!-- Enabling ODBC. -->
         <property name="odbcConfiguration">
@@ -84,11 +84,12 @@
                         <property name="addresses">
                             <list>
                                 <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500..47501</value>
+                                <value>127.0.0.1:47500</value>
                             </list>
                         </property>
                     </bean>
                 </property>
+                <property name="socketTimeout" value="300" />
             </bean>
         </property>
     </bean>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/Makefile.am b/modules/platforms/cpp/odbc-test/include/Makefile.am
index 6b2d8b6..7b17102 100644
--- a/modules/platforms/cpp/odbc-test/include/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/include/Makefile.am
@@ -19,5 +19,6 @@ ACLOCAL_AMFLAGS =-I m4
 
 noinst_HEADERS = \
     teamcity/teamcity_messages.h \
-    test_type.h
+    test_type.h \
+    test_utils.h
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/include/test_utils.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/test_utils.h b/modules/platforms/cpp/odbc-test/include/test_utils.h
new file mode 100644
index 0000000..e8cd089
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/include/test_utils.h
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_TEST_TEST_UTILS
+#define _IGNITE_ODBC_TEST_TEST_UTILS
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <sql.h>
+#include <sqlext.h>
+
+#include <string>
+
+namespace ignite
+{
+    /** Read buffer size. */
+    enum { ODBC_BUFFER_SIZE = 1024 };
+
+    /**
+     * Extract error message.
+     *
+     * @param handleType Type of the handle.
+     * @param handle Handle.
+     * @return Error message.
+     */
+    std::string GetOdbcErrorMessage(SQLSMALLINT handleType, SQLHANDLE handle);
+}
+
+#endif // _IGNITE_ODBC_TEST_TEST_UTILS
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index fbc0929..619b4b2 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -168,13 +168,16 @@
     <ClCompile Include="..\..\src\queries_test.cpp" />
     <ClCompile Include="..\..\src\parser_test.cpp" />
     <ClCompile Include="..\..\src\row_test.cpp" />
+    <ClCompile Include="..\..\src\sql_string_functions_test.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_messages.cpp" />
+    <ClCompile Include="..\..\src\test_utils.cpp" />
     <ClCompile Include="..\..\src\utility_test.cpp" />
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\teamcity\teamcity_messages.h" />
     <ClInclude Include="..\..\include\test_type.h" />
+    <ClInclude Include="..\..\include\test_utils.h" />
   </ItemGroup>
   <ItemGroup>
     <ProjectReference Include="..\..\..\binary\project\vs\binary.vcxproj">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index 2e38c24..1871149 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -76,6 +76,12 @@
     <ClCompile Include="..\..\src\queries_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\sql_string_functions_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\test_utils.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\test_type.h">
@@ -84,6 +90,9 @@
     <ClInclude Include="..\..\include\teamcity\teamcity_messages.h">
       <Filter>Code\TeamCity</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\test_utils.h">
+      <Filter>Code</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <None Include="..\..\config\queries-test.xml">

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
index cc3fa8e..2d9bd58 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -36,6 +36,7 @@
 #include "ignite/impl/binary/binary_utils.h"
 
 #include "test_type.h"
+#include "test_utils.h"
 
 using namespace ignite;
 using namespace ignite::cache;
@@ -46,30 +47,6 @@ using namespace boost::unit_test;
 
 using ignite::impl::binary::BinaryUtils;
 
-/** Read buffer size. */
-enum { ODBC_BUFFER_SIZE = 1024 };
-
-/**
- * Extract error message.
- *
- * @param handleType Type of the handle.
- * @param handle Handle.
- * @return Error message.
- */
-std::string GetOdbcErrorMessage(SQLSMALLINT handleType, SQLHANDLE handle)
-{
-    SQLCHAR sqlstate[7] = {};
-    SQLINTEGER nativeCode;
-
-    SQLCHAR message[ODBC_BUFFER_SIZE];
-    SQLSMALLINT reallen = 0;
-
-    SQLGetDiagRec(handleType, handle, 1, sqlstate, &nativeCode, message, ODBC_BUFFER_SIZE, &reallen);
-
-    return std::string(reinterpret_cast<char*>(sqlstate)) + ": " +
-        std::string(reinterpret_cast<char*>(message), reallen);
-}
-
 /**
  * Test setup fixture.
  */
@@ -98,7 +75,9 @@ struct QueriesTestSuiteFixture
 
         char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
 
-        cfg.springCfgPath = std::string(cfgPath).append("/").append("queries-test.xml");
+        BOOST_REQUIRE(cfgPath != 0);
+
+        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
 
         IgniteError err;
 
@@ -178,7 +157,7 @@ struct QueriesTestSuiteFixture
 
         T columns[columnsCnt] = { 0 };
 
-        // Binding colums.
+        // Binding columns.
         for (SQLSMALLINT i = 0; i < columnsCnt; ++i)
         {
             ret = SQLBindCol(stmt, i + 1, type, &columns[i], sizeof(columns[i]), 0);
@@ -214,7 +193,7 @@ struct QueriesTestSuiteFixture
 
         SQLLEN columnLens[columnsCnt] = { 0 };
 
-        // Binding colums.
+        // Binding columns.
         for (SQLSMALLINT i = 0; i < columnsCnt; ++i)
         {
             ret = SQLBindCol(stmt, i + 1, type, &columns[i], sizeof(columns[i]), &columnLens[i]);
@@ -327,7 +306,7 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString)
 
     SQLCHAR columns[columnsCnt][ODBC_BUFFER_SIZE] = { 0 };
 
-    // Binding colums.
+    // Binding columns.
     for (SQLSMALLINT i = 0; i < columnsCnt; ++i)
     {
         ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, &columns[i], ODBC_BUFFER_SIZE, 0);
@@ -364,7 +343,7 @@ BOOST_AUTO_TEST_CASE(TestTwoRowsString)
 
     SQLLEN columnLens[columnsCnt] = { 0 };
 
-    // Binding colums.
+    // Binding columns.
     for (SQLSMALLINT i = 0; i < columnsCnt; ++i)
     {
         ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, &columns[i], ODBC_BUFFER_SIZE, &columnLens[i]);
@@ -420,7 +399,7 @@ BOOST_AUTO_TEST_CASE(TestOneRowString)
 
     SQLLEN columnLens[columnsCnt] = { 0 };
 
-    // Binding colums.
+    // Binding columns.
     for (SQLSMALLINT i = 0; i < columnsCnt; ++i)
     {
         ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, &columns[i], ODBC_BUFFER_SIZE, &columnLens[i]);
@@ -479,7 +458,7 @@ BOOST_AUTO_TEST_CASE(TestOneRowStringLen)
 
     SQLLEN columnLens[columnsCnt] = { 0 };
 
-    // Binding colums.
+    // Binding columns.
     for (SQLSMALLINT i = 0; i < columnsCnt; ++i)
     {
         ret = SQLBindCol(stmt, i + 1, SQL_C_CHAR, 0, 0, &columnLens[i]);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
new file mode 100644
index 0000000..e84aba6
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
@@ -0,0 +1,515 @@
+/*
+ * 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.
+ */
+
+#define _USE_MATH_DEFINES
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <sql.h>
+#include <sqlext.h>
+
+#include <cmath>
+
+#include <vector>
+#include <string>
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "ignite/ignite.h"
+#include "ignite/ignition.h"
+#include "ignite/impl/binary/binary_utils.h"
+
+#include "test_type.h"
+#include "test_utils.h"
+
+using namespace ignite;
+using namespace ignite::cache;
+using namespace ignite::cache::query;
+using namespace ignite::common;
+
+using namespace boost::unit_test;
+
+using ignite::impl::binary::BinaryUtils;
+
+/**
+ * Test setup fixture.
+ */
+struct SqlStringFunctionTestSuiteFixture
+{
+    /**
+     * Constructor.
+     */
+    SqlStringFunctionTestSuiteFixture() :
+        testCache(0),
+        env(NULL),
+        dbc(NULL),
+        stmt(NULL)
+    {
+        IgniteConfiguration cfg;
+
+        cfg.jvmOpts.push_back("-Xdebug");
+        cfg.jvmOpts.push_back("-Xnoagent");
+        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
+        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
+        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
+
+#ifdef IGNITE_TESTS_32
+        cfg.jvmInitMem = 256;
+        cfg.jvmMaxMem = 768;
+#else
+        cfg.jvmInitMem = 1024;
+        cfg.jvmMaxMem = 4096;
+#endif
+
+        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
+
+        BOOST_REQUIRE(cfgPath != 0);
+
+        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
+
+        IgniteError err;
+
+        grid = Ignition::Start(cfg, &err);
+
+        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+            BOOST_FAIL(err.GetText());
+
+        testCache = grid.GetCache<int64_t, TestType>("cache");
+
+        // Allocate an environment handle
+        SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
+
+        BOOST_REQUIRE(env != NULL);
+
+        // We want ODBC 3 support
+        SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
+
+        // Allocate a connection handle
+        SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
+
+        BOOST_REQUIRE(dbc != NULL);
+
+        // Connect string
+        SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache";
+
+        SQLCHAR outstr[ODBC_BUFFER_SIZE];
+        SQLSMALLINT outstrlen;
+
+        // Connecting to ODBC server.
+        SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast<SQLSMALLINT>(sizeof(connectStr)),
+            outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
+
+        if (!SQL_SUCCEEDED(ret))
+        {
+            Ignition::Stop(grid.GetName(), true);
+
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc));
+        }
+
+        // Allocate a statement handle
+        SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
+
+        BOOST_REQUIRE(stmt != NULL);
+    }
+
+    /**
+     * Destructor.
+     */
+    ~SqlStringFunctionTestSuiteFixture()
+    {
+        // Releasing statement handle.
+        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+
+        // Disconneting from the server.
+        SQLDisconnect(dbc);
+
+        // Releasing allocated handles.
+        SQLFreeHandle(SQL_HANDLE_DBC, dbc);
+        SQLFreeHandle(SQL_HANDLE_ENV, env);
+
+        Ignition::Stop(grid.GetName(), true);
+    }
+
+    void CheckSingleResult0(const char* request, SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize)
+    {
+        SQLRETURN ret;
+
+        ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        ret = SQLExecDirect(stmt, reinterpret_cast<SQLCHAR*>(const_cast<char*>(request)), SQL_NTS);
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        ret = SQLFetch(stmt);
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+        ret = SQLFetch(stmt);
+        BOOST_CHECK(ret == SQL_NO_DATA);
+    }
+
+    /**
+     * Run query returning single result and check it to be equal to expected.
+     *
+     * @param request SQL request.
+     * @param expected Expected result.
+     */
+    template<typename T>
+    void CheckSingleResult(const char* request, const T& expected)
+    {
+        BOOST_FAIL("Function is not defined for the type.");
+    }
+    
+    /**
+     * Run query returning single result and check it to be equal to expected.
+     *
+     * @param request SQL request.
+     * @param expected Expected result.
+     * @param type Result type.
+     */
+    template<typename T>
+    void CheckSingleResultNum0(const char* request, const T& expected, SQLSMALLINT type)
+    {
+        T res = 0;
+
+        CheckSingleResult0(request, type, &res, 0, 0);
+
+        BOOST_CHECK_EQUAL(res, expected);
+    }
+
+
+    /** Node started during the test. */
+    Ignite grid;
+
+    /** Test cache instance. */
+    Cache<int64_t, TestType> testCache;
+
+    /** ODBC Environment. */
+    SQLHENV env;
+
+    /** ODBC Connect. */
+    SQLHDBC dbc;
+
+    /** ODBC Statement. */
+    SQLHSTMT stmt;
+};
+
+template<>
+void SqlStringFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request, const std::string& expected)
+{
+    SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
+    SQLLEN resLen = 0;
+
+    CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
+
+    BOOST_CHECK_EQUAL(std::string(reinterpret_cast<char*>(res), static_cast<size_t>(resLen)), expected);
+}
+
+template<>
+void SqlStringFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request, const int64_t& expected)
+{
+    CheckSingleResultNum0<int64_t>(request, expected, SQL_C_SBIGINT);
+}
+
+template<>
+void SqlStringFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request, const int32_t& expected)
+{
+    CheckSingleResultNum0<int32_t>(request, expected, SQL_C_SLONG);
+}
+
+template<>
+void SqlStringFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request, const int16_t& expected)
+{
+    CheckSingleResultNum0<int16_t>(request, expected, SQL_C_SSHORT);
+}
+
+template<>
+void SqlStringFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request, const int8_t& expected)
+{
+    CheckSingleResultNum0<int8_t>(request, expected, SQL_C_STINYINT);
+}
+
+template<>
+void SqlStringFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected)
+{
+    SQLFLOAT res = 0;
+
+    CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
+
+    BOOST_CHECK_CLOSE(static_cast<float>(res), expected, 1E-6f);
+}
+
+template<>
+void SqlStringFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected)
+{
+    SQLDOUBLE res = 0;
+
+    CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0);
+
+    BOOST_CHECK_CLOSE(static_cast<double>(res), expected, 1E-6);
+}
+
+BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, SqlStringFunctionTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionAscii)
+{
+    TestType in;
+
+    in.strField = "Hi";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int32_t>("SELECT {fn ASCII(strField)} FROM TestType", static_cast<int32_t>('H'));
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionBitLength)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn BIT_LENGTH(strField)} FROM TestType", in.strField.size() * 16);
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionChar)
+{
+    TestType in;
+
+    in.i32Field = static_cast<int32_t>('H');
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn CHAR(i32Field)} FROM TestType", "H");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionCharLength)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn CHAR_LENGTH(strField)} FROM TestType", in.strField.size());
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionCharacterLength)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn CHARACTER_LENGTH(strField)} FROM TestType", in.strField.size());
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionConcat)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor sit amet,";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn CONCAT(strField, \' consectetur adipiscing elit\')} FROM TestType",
+        in.strField + " consectetur adipiscing elit");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionDifference)
+{
+    TestType in;
+    in.strField = "Hello";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int32_t>("SELECT {fn DIFFERENCE(strField, \'Hola!\')} FROM TestType", 4);
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionInsert)
+{
+    TestType in;
+    in.strField = "Hello World!";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn INSERT(strField, 7, 5, \'Ignite\')} FROM TestType", "Hello Ignite!");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionLcase)
+{
+    TestType in;
+    in.strField = "Hello World!";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn LCASE(strField)} FROM TestType", "hello world!");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionLeft)
+{
+    TestType in;
+    in.strField = "Hello World!";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn LEFT(strField, 5)} FROM TestType", "Hello");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionLength)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn LENGTH(strField)} FROM TestType", in.strField.size());
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionLocate)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn LOCATE(\'ip\', strField)} FROM TestType", 7);
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionLocate2)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn LOCATE(\'ip\', strField, 10)} FROM TestType", 43);
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionLtrim)
+{
+    TestType in;
+    in.strField = "    Lorem ipsum  ";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn LTRIM(strField)} FROM TestType", "Lorem ipsum  ");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionOctetLength)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn OCTET_LENGTH(strField)} FROM TestType", in.strField.size() * 2);
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionPosition)
+{
+    TestType in;
+    in.strField = "Lorem ipsum dolor sit amet, consectetur adipiscing elit";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn POSITION(\'sit\', strField)} FROM TestType", 19);
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionRepeat)
+{
+    TestType in;
+    in.strField = "Test";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn REPEAT(strField,4)} FROM TestType", "TestTestTestTest");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionReplace)
+{
+    TestType in;
+    in.strField = "Hello Ignite!";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn REPLACE(strField, \'Ignite\', \'World\')} FROM TestType", "Hello World!");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionRight)
+{
+    TestType in;
+    in.strField = "Hello World!";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn RIGHT(strField, 6)} FROM TestType", "World!");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionRtrim)
+{
+    TestType in;
+    in.strField = "    Lorem ipsum  ";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn RTRIM(strField)} FROM TestType", "    Lorem ipsum");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionSoundex)
+{
+    TestType in;
+    in.strField = "Hello Ignite!";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn SOUNDEX(strField)} FROM TestType", "H425");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionSpace)
+{
+    CheckSingleResult<std::string>("SELECT {fn SPACE(10)}", "          ");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionSubstring)
+{
+    TestType in;
+    in.strField = "Hello Ignite!";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn SUBSTRING(strField, 7, 6)} FROM TestType", "Ignite");
+}
+
+BOOST_AUTO_TEST_CASE(TestStringFunctionUcase)
+{
+    TestType in;
+    in.strField = "Hello World!";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT {fn UCASE(strField)} FROM TestType", "HELLO WORLD!");
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc-test/src/test_utils.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/test_utils.cpp b/modules/platforms/cpp/odbc-test/src/test_utils.cpp
new file mode 100644
index 0000000..48f409b
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/test_utils.cpp
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+#include "test_utils.h"
+
+namespace ignite
+{
+
+    std::string GetOdbcErrorMessage(SQLSMALLINT handleType, SQLHANDLE handle)
+    {
+        SQLCHAR sqlstate[7] = {};
+        SQLINTEGER nativeCode;
+
+        SQLCHAR message[ODBC_BUFFER_SIZE];
+        SQLSMALLINT reallen = 0;
+
+        SQLGetDiagRec(handleType, handle, 1, sqlstate, &nativeCode, message, ODBC_BUFFER_SIZE, &reallen);
+
+        return std::string(reinterpret_cast<char*>(sqlstate)) + ": " +
+            std::string(reinterpret_cast<char*>(message), reallen);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc/src/config/connection_info.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
index 7f0e3bd..f35b790 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -124,7 +124,7 @@ namespace ignite
 #ifdef SQL_COLUMN_ALIAS
                 // A character string: "Y" if the data source supports column 
                 // aliases; otherwise, "N".
-                strParams[SQL_COLUMN_ALIAS] = "N";
+                strParams[SQL_COLUMN_ALIAS] = "Y";
 #endif // SQL_COLUMN_ALIAS
 
 #ifdef SQL_IDENTIFIER_QUOTE_CHAR
@@ -192,7 +192,7 @@ namespace ignite
 
 #ifdef SQL_GETDATA_EXTENSIONS
                 // Bitmask enumerating extensions to SQLGetData.
-                intParams[SQL_GETDATA_EXTENSIONS] = SQL_GD_ANY_COLUMN;
+                intParams[SQL_GETDATA_EXTENSIONS] = SQL_GD_ANY_COLUMN | SQL_GD_ANY_ORDER | SQL_GD_BOUND;
 #endif // SQL_GETDATA_EXTENSIONS
 
 #ifdef SQL_ODBC_INTERFACE_CONFORMANCE
@@ -203,7 +203,7 @@ namespace ignite
 
 #ifdef SQL_SQL_CONFORMANCE
                 // Indicates the level of SQL-92 supported by the driver.
-                intParams[SQL_SQL_CONFORMANCE] = 0; // SQL_SC_SQL92_ENTRY;
+                intParams[SQL_SQL_CONFORMANCE] = SQL_SC_SQL92_ENTRY;
 #endif // SQL_SQL_CONFORMANCE
 
 #ifdef SQL_CATALOG_USAGE
@@ -213,7 +213,8 @@ namespace ignite
 
 #ifdef SQL_SCHEMA_USAGE
                 // Bitmask enumerating the statements in which schemas can be used.
-                intParams[SQL_SCHEMA_USAGE] = 0;
+                intParams[SQL_SCHEMA_USAGE] = SQL_SU_DML_STATEMENTS |
+                    SQL_SU_TABLE_DEFINITION | SQL_SU_PRIVILEGE_DEFINITION;
 #endif // SQL_SCHEMA_USAGE
 
 #ifdef SQL_MAX_IDENTIFIER_LEN
@@ -224,21 +225,24 @@ namespace ignite
 
 #ifdef SQL_AGGREGATE_FUNCTIONS
                 // Bitmask enumerating support for aggregation functions.
-                intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_ALL | SQL_AF_AVG | 
-                    SQL_AF_COUNT | SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN |
-                    SQL_AF_SUM;
+                intParams[SQL_AGGREGATE_FUNCTIONS] = 0;
 #endif // SQL_AGGREGATE_FUNCTIONS
 
 #ifdef SQL_NUMERIC_FUNCTIONS
                 // Bitmask enumerating the scalar numeric functions supported by
                 // the driver and associated data source.
-                intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS;
+                intParams[SQL_NUMERIC_FUNCTIONS] = 0;
 #endif // SQL_NUMERIC_FUNCTIONS
 
 #ifdef SQL_STRING_FUNCTIONS
                 // Bitmask enumerating the scalar string functions supported by the
                 // driver and associated data source.
-                intParams[SQL_STRING_FUNCTIONS] = 0;
+                intParams[SQL_STRING_FUNCTIONS] = SQL_FN_STR_ASCII | SQL_FN_STR_BIT_LENGTH | SQL_FN_STR_CHAR |
+                    SQL_FN_STR_CONCAT | SQL_FN_STR_DIFFERENCE | SQL_FN_STR_INSERT | SQL_FN_STR_LEFT |
+                    SQL_FN_STR_LENGTH | SQL_FN_STR_LOCATE | SQL_FN_STR_LTRIM | SQL_FN_STR_OCTET_LENGTH |
+                    SQL_FN_STR_POSITION | SQL_FN_STR_REPEAT | SQL_FN_STR_REPLACE | SQL_FN_STR_RIGHT | SQL_FN_STR_RTRIM |
+                    SQL_FN_STR_SOUNDEX | SQL_FN_STR_SPACE | SQL_FN_STR_SUBSTRING | SQL_FN_STR_LCASE | SQL_FN_STR_UCASE |
+                    SQL_FN_STR_LOCATE_2 | SQL_FN_STR_CHAR_LENGTH | SQL_FN_STR_CHARACTER_LENGTH;
 #endif // SQL_STRING_FUNCTIONS
 
 #ifdef SQL_TIMEDATE_FUNCTIONS
@@ -262,15 +266,7 @@ namespace ignite
 #ifdef SQL_DATETIME_LITERALS
                 // Bitmask enumerating the SQL-92 datetime literals supported by
                 // the data source.
-                intParams[SQL_DATETIME_LITERALS] = SQL_DL_SQL92_INTERVAL_HOUR |
-                    SQL_DL_SQL92_DATE | SQL_DL_SQL92_INTERVAL_MINUTE_TO_SECOND |
-                    SQL_DL_SQL92_TIME | SQL_DL_SQL92_INTERVAL_HOUR_TO_SECOND |
-                    SQL_DL_SQL92_TIMESTAMP | SQL_DL_SQL92_INTERVAL_HOUR_TO_MINUTE |
-                    SQL_DL_SQL92_INTERVAL_YEAR | SQL_DL_SQL92_INTERVAL_DAY_TO_SECOND |
-                    SQL_DL_SQL92_INTERVAL_MONTH | SQL_DL_SQL92_INTERVAL_DAY_TO_HOUR |
-                    SQL_DL_SQL92_INTERVAL_DAY | SQL_DL_SQL92_INTERVAL_DAY_TO_MINUTE |
-                    SQL_DL_SQL92_INTERVAL_MINUTE | SQL_DL_SQL92_INTERVAL_SECOND |
-                    SQL_DL_SQL92_INTERVAL_YEAR_TO_MONTH;
+                intParams[SQL_DATETIME_LITERALS] = 0;
 #endif // SQL_DATETIME_LITERALS
 
 #ifdef SQL_SYSTEM_FUNCTIONS
@@ -288,9 +284,7 @@ namespace ignite
 #ifdef SQL_OJ_CAPABILITIES
                 // Bitmask enumerating the types of outer joins supported by the 
                 // driver and data source.
-                intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_RIGHT |
-                    SQL_OJ_FULL | SQL_OJ_NESTED | SQL_OJ_INNER | 
-                    SQL_OJ_ALL_COMPARISON_OPS;
+                intParams[SQL_OJ_CAPABILITIES] = 0;
 #endif // SQL_OJ_CAPABILITIES
 
 #ifdef SQL_POS_OPERATIONS
@@ -305,42 +299,36 @@ namespace ignite
 
 #ifdef SQL_SQL92_STRING_FUNCTIONS
                 // Bitmask enumerating the string scalar functions.
-                intParams[SQL_SQL92_STRING_FUNCTIONS] = 0;
+                intParams[SQL_SQL92_STRING_FUNCTIONS] = SQL_SSF_CONVERT | SQL_SSF_LOWER | SQL_SSF_UPPER |
+                    SQL_SSF_SUBSTRING | SQL_SSF_TRANSLATE;
 #endif // SQL_SQL92_STRING_FUNCTIONS
 
 #ifdef SQL_SQL92_DATETIME_FUNCTIONS
                 // Bitmask enumerating the datetime scalar functions.
-                intParams[SQL_SQL92_DATETIME_FUNCTIONS] = SQL_SDF_CURRENT_DATE |
-                    SQL_SDF_CURRENT_TIMESTAMP;
+                intParams[SQL_SQL92_DATETIME_FUNCTIONS] = 0;
 #endif // SQL_SQL92_DATETIME_FUNCTIONS
 
 #ifdef SQL_SQL92_VALUE_EXPRESSIONS
                 // Bitmask enumerating the value expressions supported,
                 // as defined in SQL-92.
                 intParams[SQL_SQL92_VALUE_EXPRESSIONS] = SQL_SVE_CASE | 
-                    SQL_SVE_COALESCE | SQL_SVE_NULLIF;
+                    SQL_SVE_CAST | SQL_SVE_COALESCE | SQL_SVE_NULLIF;
 #endif // SQL_SQL92_VALUE_EXPRESSIONS
 
 #ifdef SQL_SQL92_PREDICATES
                 // Bitmask enumerating the datetime scalar functions.
-                intParams[SQL_SQL92_PREDICATES] = SQL_SP_BETWEEN |
-                    SQL_SP_COMPARISON | SQL_SP_EXISTS | SQL_SP_IN |
-                    SQL_SP_ISNOTNULL | SQL_SP_ISNULL | SQL_SP_LIKE |
-                    SQL_SP_MATCH_FULL | SQL_SP_MATCH_PARTIAL |
-                    SQL_SP_MATCH_UNIQUE_FULL | SQL_SP_MATCH_UNIQUE_PARTIAL |
-                    SQL_SP_OVERLAPS | SQL_SP_QUANTIFIED_COMPARISON |
-                    SQL_SP_UNIQUE;
+                intParams[SQL_SQL92_PREDICATES] = SQL_SP_BETWEEN | SQL_SP_COMPARISON | SQL_SP_EXISTS | SQL_SP_IN |
+                    SQL_SP_ISNOTNULL | SQL_SP_ISNULL | SQL_SP_LIKE | SQL_SP_MATCH_FULL | SQL_SP_MATCH_PARTIAL |
+                    SQL_SP_MATCH_UNIQUE_FULL | SQL_SP_MATCH_UNIQUE_PARTIAL | SQL_SP_OVERLAPS | SQL_SP_UNIQUE |
+                    SQL_SP_QUANTIFIED_COMPARISON;
 #endif // SQL_SQL92_PREDICATES
 
 #ifdef SQL_SQL92_RELATIONAL_JOIN_OPERATORS
                 // Bitmask enumerating the relational join operators supported
                 // in a SELECT statement, as defined in SQL-92.
-                intParams[SQL_SQL92_RELATIONAL_JOIN_OPERATORS] =
-                    SQL_SRJO_CORRESPONDING_CLAUSE | SQL_SRJO_CROSS_JOIN |
-                    SQL_SRJO_EXCEPT_JOIN | SQL_SRJO_EXCEPT_JOIN |
-                    SQL_SRJO_INNER_JOIN | SQL_SRJO_INTERSECT_JOIN |
-                    SQL_SRJO_LEFT_OUTER_JOIN | SQL_SRJO_NATURAL_JOIN |
-                    SQL_SRJO_RIGHT_OUTER_JOIN | SQL_SRJO_UNION_JOIN;
+                intParams[SQL_SQL92_RELATIONAL_JOIN_OPERATORS] = SQL_SRJO_CORRESPONDING_CLAUSE | SQL_SRJO_CROSS_JOIN |
+                    SQL_SRJO_EXCEPT_JOIN | SQL_SRJO_INNER_JOIN | SQL_SRJO_LEFT_OUTER_JOIN| SQL_SRJO_RIGHT_OUTER_JOIN |
+                    SQL_SRJO_NATURAL_JOIN | SQL_SRJO_INTERSECT_JOIN | SQL_SRJO_UNION_JOIN;
 #endif // SQL_SQL92_RELATIONAL_JOIN_OPERATORS
 
                 //========================= Short Params ==========================

http://git-wip-us.apache.org/repos/asf/ignite/blob/c02ad87f/modules/platforms/cpp/odbc/src/query/data_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp
index e96f1da..183bbb5 100644
--- a/modules/platforms/cpp/odbc/src/query/data_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp
@@ -141,11 +141,7 @@ namespace ignite
             SqlResult DataQuery::Close()
             {
                 if (!cursor.get())
-                {
-                    diag.AddStatusRecord(SQL_STATE_HY010_SEQUENCE_ERROR, "Query cursor is not in open state.");
-
-                    return SQL_RESULT_ERROR;
-                }
+                    return SQL_RESULT_SUCCESS;
 
                 SqlResult result = MakeRequestClose();
 


[12/24] ignite git commit: IGNITE-3739: ODBC: Added GUID escape sequence support. This closes #988.

Posted by sb...@apache.org.
IGNITE-3739: ODBC: Added GUID escape sequence support. This closes #988.


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

Branch: refs/heads/master
Commit: ae0b5ebf02f3eb70d24dd3b0eb63dde9843c82b0
Parents: 8aabd6e
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Aug 26 11:12:31 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 26 11:12:31 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/escape/OdbcEscapeType.java  | 16 ++--
 .../processors/odbc/escape/OdbcEscapeUtils.java | 71 +++++++++++++---
 .../odbc/OdbcEscapeSequenceSelfTest.java        | 86 ++++++++++++++++----
 3 files changed, 136 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ae0b5ebf/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
index 96a2127..3bf0324 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
@@ -63,8 +63,8 @@ public enum OdbcEscapeType {
     /** Escape sequence body. */
     private final String body;
 
-    /** Whether token must be delimited from the rest of escape sequence. */
-    private final boolean delimited;
+    /** Whether this is a standard token with no special handling. */
+    private final boolean standard;
 
     /** Whether empty escape sequence is allowed. */
     private final boolean allowEmpty;
@@ -73,12 +73,12 @@ public enum OdbcEscapeType {
      * Constructor.
      *
      * @param body Escape sequence body.
-     * @param delimited Whether token must be delimited from the rest of escape sequence.
+     * @param standard Whether this is a standard token with no special handling.
      * @param allowEmpty Whether empty escape sequence is allowed.
      */
-    OdbcEscapeType(String body, boolean delimited, boolean allowEmpty) {
+    OdbcEscapeType(String body, boolean standard, boolean allowEmpty) {
         this.body = body;
-        this.delimited = delimited;
+        this.standard = standard;
         this.allowEmpty = allowEmpty;
     }
 
@@ -90,10 +90,10 @@ public enum OdbcEscapeType {
     }
 
     /**
-     * @return Whether token must be delimited from the rest of escape sequence.
+     * @return Whether this is a standard token with no special handling.
      */
-    public boolean delimited() {
-        return delimited;
+    public boolean standard() {
+        return standard;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae0b5ebf/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
index 6299c7e..83ec9d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
@@ -20,11 +20,19 @@ package org.apache.ignite.internal.processors.odbc.escape;
 import org.apache.ignite.IgniteException;
 
 import java.util.LinkedList;
+import java.util.regex.Pattern;
 
 /**
  * ODBC escape sequence parse.
  */
 public class OdbcEscapeUtils {
+
+    /**
+     * GUID regexp pattern: '12345678-9abc-def0-1234-123456789abc'
+     */
+    private static final Pattern GUID_PATTERN =
+        Pattern.compile("^'\\p{XDigit}{8}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{12}'$");
+
     /**
      * Parse escape sequence.
      *
@@ -145,14 +153,11 @@ public class OdbcEscapeUtils {
 
             OdbcEscapeToken token = parseToken(text, startPos, len);
 
-            switch (token.type()) {
-                case SCALAR_FUNCTION:
-                    return parseScalarExpression(text, startPos, len, token);
-
-                default:
-                    throw new IgniteException("Unsupported escape sequence token [text=" +
-                        substring(text, startPos, len) + ", token=" + token.type().body() + ']');
-            }
+            if (token.type().standard())
+                return parseStandardExpression(text, startPos, len, token);
+            else
+                throw new IgniteException("Unsupported escape sequence token [text=" +
+                    substring(text, startPos, len) + ", token=" + token.type().body() + ']');
         }
         else {
             // Nothing to escape, return original string.
@@ -191,7 +196,7 @@ public class OdbcEscapeUtils {
                 else {
                     empty = (startPos + len == pos + 1);
 
-                    if (!empty && typ.delimited()) {
+                    if (!empty && typ.standard()) {
                         char charAfter = text.charAt(pos);
 
                         if (!Character.isWhitespace(charAfter))
@@ -216,21 +221,61 @@ public class OdbcEscapeUtils {
     }
 
     /**
-     * Parse concrete expression.
+     * Parse standard token.
      *
      * @param text Text.
      * @param startPos Start position.
      * @param len Length.
      * @param token Token.
-     * @return Parsed expression.
+     * @return Result.
      */
-    private static String parseScalarExpression(String text, int startPos, int len, OdbcEscapeToken token) {
+    private static String parseStandardExpression(String text, int startPos, int len, OdbcEscapeToken token) {
         assert validSubstring(text, startPos, len);
 
+        // Get expression borders.
         int startPos0 = startPos + 1 /* open brace */ + token.length() /* token. */;
         int len0 = len - 1 /* open brace */ - token.length() /* token */ - 1 /* close brace */;
 
-        return substring(text, startPos0, len0).trim();
+        switch (token.type()) {
+            case SCALAR_FUNCTION:
+                return parseScalarExpression(text, startPos0, len0);
+
+            case GUID:
+                return parseGuidExpression(text, startPos0, len0);
+
+            default:
+                throw new IgniteException("Unsupported escape sequence token [text=" +
+                    substring(text, startPos, len) + ", token=" + token.type().body() + ']');
+        }
+    }
+
+    /**
+     * Parse scalar function expression.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Parsed expression.
+     */
+    private static String parseScalarExpression(String text, int startPos, int len) {
+        return substring(text, startPos, len).trim();
+    }
+
+    /**
+     * Parse concrete expression.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Parsed expression.
+     */
+    private static String parseGuidExpression(String text, int startPos, int len) {
+        String val = substring(text, startPos, len).trim();
+
+        if (!GUID_PATTERN.matcher(val).matches())
+            throw new IgniteException("Invalid GUID escape sequence: " + substring(text, startPos, len));
+
+        return val;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ae0b5ebf/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
index d9be6cc..7225c1a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
@@ -25,18 +25,23 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import java.util.concurrent.Callable;
 
 /**
- * Scalar function escape sequence parser tests.
+ * Escape sequence parser tests.
  */
 public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
     /**
      * Test simple cases.
      */
-    public void testSimple() {
+    public void testTrivial() {
         check(
             "select * from table;",
             "select * from table;"
         );
+    }
 
+    /**
+     * Test escape sequence series.
+     */
+    public void testSimpleFunction() throws Exception {
         check(
             "test()",
             "{fn test()}"
@@ -51,12 +56,7 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
             "select test() from table;",
             "select {fn test()} from table;"
         );
-    }
 
-    /**
-     * Test escape sequence series.
-     */
-    public void testSimpleFunction() throws Exception {
         check(
             "func(field1) func(field2)",
             "{fn func(field1)} {fn func(field2)}"
@@ -139,20 +139,15 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Test non-closed escape sequence.
+     * Test invalid escape sequence.
      */
-    public void testFailedOnNonClosedEscapeSequence() {
+    public void testFailedOnInvalidFunctionSequence() {
         checkFail("select {fn func1(field1, {fn func2(field2), field3)} from SomeTable;");
-    }
 
-    /**
-     * Test closing undeclared escape sequence.
-     */
-    public void testFailedOnClosingNotOpenedSequence() {
-        checkFail("select {fn func1(field1, func2(field2)}, field3)} from SomeTable;");
+        checkFail("select {fn func1(field1, fn func2(field2)}, field3)} from SomeTable;");
     }
 
-    /**
+     /**
      * Test escape sequences with additional whitespace characters
      */
     public void testFunctionEscapeSequenceWithWhitespaces() throws Exception {
@@ -166,6 +161,65 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test guid escape sequences
+     */
+    public void testGuidEscapeSequence() {
+        check(
+            "'12345678-9abc-def0-1234-123456789abc'",
+            "{guid '12345678-9abc-def0-1234-123456789abc'}"
+        );
+
+        check(
+            "select '12345678-9abc-def0-1234-123456789abc' from SomeTable;",
+            "select {guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;"
+        );
+
+        check(
+            "select '12345678-9abc-def0-1234-123456789abc'",
+            "select {guid '12345678-9abc-def0-1234-123456789abc'}"
+        );
+
+        checkFail("select {guid '1234567-1234-1234-1234-123456789abc'}");
+
+        checkFail("select {guid '1234567-8123-4123-4123-4123456789abc'}");
+
+        checkFail("select {guid '12345678-9abc-defg-1234-123456789abc'}");
+
+        checkFail("select {guid '12345678-12345678-1234-1234-1234-123456789abc'}");
+
+        checkFail("select {guid '12345678-1234-1234-1234-123456789abcdef'}");
+    }
+
+    /**
+     * Test invalid escape sequence.
+     */
+    public void testFailedOnInvalidGuidSequence() {
+        checkFail("select {guid '12345678-9abc-def0-1234-123456789abc' from SomeTable;");
+
+        checkFail("select guid '12345678-9abc-def0-1234-123456789abc'} from SomeTable;");
+    }
+
+    /**
+     * Test escape sequences with additional whitespace characters
+     */
+    public void testGuidEscapeSequenceWithWhitespaces() throws Exception {
+        check(
+            "'12345678-9abc-def0-1234-123456789abc'",
+            "{ guid '12345678-9abc-def0-1234-123456789abc'}"
+        );
+
+        check(
+            "'12345678-9abc-def0-1234-123456789abc'",
+            "{    guid  '12345678-9abc-def0-1234-123456789abc'}"
+        );
+
+        check(
+            "'12345678-9abc-def0-1234-123456789abc'",
+            "{  \n guid\n'12345678-9abc-def0-1234-123456789abc'}"
+        );
+    }
+
+    /**
      * Check parsing logic.
      *
      * @param exp Expected result.


[09/24] ignite git commit: IGNITE-3751: ODBC: Added system functions support. This closes #985.

Posted by sb...@apache.org.
IGNITE-3751: ODBC: Added system functions support. This closes #985.


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

Branch: refs/heads/master
Commit: 8aabd6ea65d883d3bbcf37c05c146105dff8a6e2
Parents: e21111f
Author: isapego <ig...@gmail.com>
Authored: Thu Aug 25 16:30:20 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 25 16:30:20 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/odbc-test/Makefile.am     |  1 +
 .../include/sql_function_test_suite_fixture.h   | 49 +++++++++++++-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |  1 +
 .../project/vs/odbc-test.vcxproj.filters        |  3 +
 .../src/sql_function_test_suite_fixture.cpp     | 49 ++++++++++++++
 .../odbc-test/src/sql_system_functions_test.cpp | 68 ++++++++++++++++++++
 .../cpp/odbc/src/config/connection_info.cpp     |  2 +-
 7 files changed, 171 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8aabd6ea/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index 6288637..1e00b6f 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -66,6 +66,7 @@ ignite_odbc_tests_SOURCES = \
     src/sql_string_functions_test.cpp \
     src/sql_numeric_functions_test.cpp \
     src/sql_aggregate_functions_test.cpp \
+    src/sql_system_functions_test.cpp \
     ../odbc/src/cursor.cpp \
     ../odbc/src/config/connection_info.cpp \
     ../odbc/src/app/application_data_buffer.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/8aabd6ea/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
index c09c1ae..eef0a9b 100644
--- a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
+++ b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
@@ -78,7 +78,18 @@ namespace ignite
         {
             BOOST_FAIL("Function is not defined for the type.");
         }
-    
+
+        /**
+         * Run query returning single result and check it to be successful.
+         *
+         * @param request SQL request.
+         */
+        template<typename T>
+        void CheckSingleResult(const char* request)
+        {
+            BOOST_FAIL("Function is not defined for the type.");
+        }
+
         /**
          * Run query returning single result and check it to be equal to expected.
          *
@@ -96,6 +107,21 @@ namespace ignite
             BOOST_CHECK_EQUAL(res, expected);
         }
 
+        /**
+         * Run query returning single result and check it to be equal to expected.
+         *
+         * @param request SQL request.
+         * @param expected Expected result.
+         * @param type Result type.
+         */
+        template<typename T>
+        void CheckSingleResultNum0(const char* request, SQLSMALLINT type)
+        {
+            T res = 0;
+
+            CheckSingleResult0(request, type, &res, 0, 0);
+        }
+
 
         /** Node started during the test. */
         Ignite grid;
@@ -133,6 +159,27 @@ namespace ignite
 
     template<>
     void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request);
 }
 
 #endif //_IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE

http://git-wip-us.apache.org/repos/asf/ignite/blob/8aabd6ea/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index e0b786a..2a94a2a 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -172,6 +172,7 @@
     <ClCompile Include="..\..\src\sql_function_test_suite_fixture.cpp" />
     <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_string_functions_test.cpp" />
+    <ClCompile Include="..\..\src\sql_system_functions_test.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_messages.cpp" />
     <ClCompile Include="..\..\src\test_utils.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/8aabd6ea/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index 763a364..9ca51af 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -91,6 +91,9 @@
     <ClCompile Include="..\..\src\sql_aggregate_functions_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\sql_system_functions_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\test_type.h">

http://git-wip-us.apache.org/repos/asf/ignite/blob/8aabd6ea/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
index 2398393..f848dc5 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
@@ -185,4 +185,53 @@ namespace ignite
 
         BOOST_CHECK_CLOSE(static_cast<double>(res), expected, 1E-6);
     }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request)
+    {
+        SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
+        SQLLEN resLen = 0;
+
+        CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request)
+    {
+        CheckSingleResultNum0<int64_t>(request, SQL_C_SBIGINT);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request)
+    {
+        CheckSingleResultNum0<int32_t>(request, SQL_C_SLONG);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request)
+    {
+        CheckSingleResultNum0<int16_t>(request, SQL_C_SSHORT);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request)
+    {
+        CheckSingleResultNum0<int8_t>(request, SQL_C_STINYINT);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request)
+    {
+        SQLFLOAT res = 0;
+
+        CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request)
+    {
+        SQLDOUBLE res = 0;
+
+        CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8aabd6ea/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
new file mode 100644
index 0000000..d369b0f
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
@@ -0,0 +1,68 @@
+/*
+ * 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.
+ */
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <sql.h>
+#include <sqlext.h>
+
+#include <vector>
+#include <string>
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "ignite/ignite.h"
+#include "ignite/ignition.h"
+#include "ignite/impl/binary/binary_utils.h"
+
+#include "test_type.h"
+#include "test_utils.h"
+#include "sql_function_test_suite_fixture.h"
+
+using namespace ignite;
+using namespace ignite::cache;
+using namespace ignite::cache::query;
+using namespace ignite::common;
+
+using namespace boost::unit_test;
+
+using ignite::impl::binary::BinaryUtils;
+
+BOOST_FIXTURE_TEST_SUITE(SqlSystemFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestSystemFunctionDatabase)
+{
+    CheckSingleResult<std::string>("SELECT {fn DATABASE()}");
+}
+
+BOOST_AUTO_TEST_CASE(TestSystemFunctionUser)
+{
+    CheckSingleResult<std::string>("SELECT {fn USER()}");
+}
+
+BOOST_AUTO_TEST_CASE(TestSystemFunctionIfnull)
+{
+    CheckSingleResult<int32_t>("SELECT {fn IFNULL(NULL, 42)}", 42);
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/8aabd6ea/modules/platforms/cpp/odbc/src/config/connection_info.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
index b2480c2..cff48cf 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -286,7 +286,7 @@ namespace ignite
 #ifdef SQL_SYSTEM_FUNCTIONS
                 // Bitmask enumerating the scalar system functions supported by the
                 // driver and associated data source.
-                intParams[SQL_SYSTEM_FUNCTIONS] = 0;
+                intParams[SQL_SYSTEM_FUNCTIONS] = SQL_FN_SYS_USERNAME | SQL_FN_SYS_DBNAME | SQL_FN_SYS_IFNULL;
 #endif // SQL_SYSTEM_FUNCTIONS
 
 #ifdef SQL_CONVERT_FUNCTIONS


[20/24] ignite git commit: ignite-2968 Deadlock detection for optimistic tx and near caches

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index 63c9919..f9357f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -50,7 +50,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheMappedVersion;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxFinishSync;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture;
@@ -62,6 +61,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxRemo
 import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedLockFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetection.TxDeadlockFuture;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -126,7 +127,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     public static final IgniteProductVersion TX_DEADLOCK_DETECTION_SINCE = IgniteProductVersion.fromString("1.5.19");
 
     /** Deadlock detection maximum iterations. */
-    static final int DEADLOCK_MAX_ITERS =
+    static int DEADLOCK_MAX_ITERS =
         IgniteSystemProperties.getInteger(IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS, 1000);
 
     /** Committing transactions. */
@@ -389,7 +390,13 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      *      {@code false} otherwise.
      */
     public boolean isCompleted(IgniteInternalTx tx) {
-        return completedVersHashMap.containsKey(tx.xidVersion());
+        boolean completed = completedVersHashMap.containsKey(tx.xidVersion());
+
+        // Need check that for tx with timeout rollback message was not received before lock.
+        if (!completed && tx.local() && tx.dht() && tx.timeout() > 0)
+            return completedVersHashMap.containsKey(tx.nearXidVersion());
+
+        return completed;
     }
 
     /**
@@ -495,13 +502,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             return null;
         }
 
-        if (tx.timeout() > 0) {
-            cctx.time().addTimeoutObject(tx);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered transaction with timeout processor: " + tx);
-        }
-
         if (log.isDebugEnabled())
             log.debug("Transaction created: " + tx);
 
@@ -786,7 +786,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      */
     public void prepareTx(IgniteInternalTx tx) throws IgniteCheckedException {
         if (tx.state() == MARKED_ROLLBACK) {
-            if (tx.timedOut())
+            if (tx.remainingTime() == -1)
                 throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this);
 
             throw new IgniteCheckedException("Transaction is marked for rollback: " + tx);
@@ -1081,13 +1081,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         if (log.isDebugEnabled())
             log.debug("Committing from TM [locNodeId=" + cctx.localNodeId() + ", tx=" + tx + ']');
 
-        if (tx.timeout() > 0) {
-            cctx.time().removeTimeoutObject(tx);
-
-            if (log.isDebugEnabled())
-                log.debug("Unregistered transaction with timeout processor: " + tx);
-        }
-
         /*
          * Note that write phase is handled by transaction adapter itself,
          * so we don't do it here.
@@ -2006,17 +1999,45 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             if (!(nearTxLoc || tx instanceof GridDhtTxLocal) || !hasKeys(tx, txKeys))
                 continue;
 
-            Collection<IgniteTxEntry> txEntries = tx.allEntries();
+            IgniteTxState state = tx.txState();
+
+            assert state instanceof IgniteTxStateImpl || state instanceof IgniteTxImplicitSingleStateImpl;
+
+            Collection<IgniteTxEntry> txEntries =
+                state instanceof IgniteTxStateImpl ? ((IgniteTxStateImpl)state).allEntriesCopy() : state.allEntries();
 
-            Set<KeyCacheObject> requestedKeys = null;
+            Set<IgniteTxKey> requestedKeys = null;
 
             // Try to get info about requested keys for detached entries in case of GridNearTxLocal transaction
             // in order to reduce amount of requests to remote nodes.
             if (nearTxLoc) {
-                GridDhtColocatedLockFuture fut = colocatedLockFuture(tx);
+                if (tx.pessimistic()) {
+                    GridDhtColocatedLockFuture fut =
+                        (GridDhtColocatedLockFuture)mvccFuture(tx, GridDhtColocatedLockFuture.class);
+
+                    if (fut != null)
+                        requestedKeys = fut.requestedKeys();
+
+                    GridNearLockFuture nearFut = (GridNearLockFuture)mvccFuture(tx, GridNearLockFuture.class);
 
-                if (fut != null)
-                    requestedKeys = fut.requestedKeys();
+                    if (nearFut != null) {
+                        Set<IgniteTxKey> nearRequestedKeys = nearFut.requestedKeys();
+
+                        if (nearRequestedKeys != null) {
+                            if (requestedKeys == null)
+                                requestedKeys = nearRequestedKeys;
+                            else
+                                requestedKeys = nearRequestedKeys;
+                        }
+                    }
+                }
+                else {
+                    GridNearOptimisticTxPrepareFuture fut =
+                        (GridNearOptimisticTxPrepareFuture)mvccFuture(tx, GridNearOptimisticTxPrepareFuture.class);
+
+                    if (fut != null)
+                        requestedKeys = fut.requestedKeys();
+                }
             }
 
             for (IgniteTxEntry txEntry : txEntries) {
@@ -2073,17 +2094,18 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
 
     /**
      * @param tx Tx. Must be instance of {@link GridNearTxLocal}.
-     * @return Colocated future.
+     * @param cls Future class.
+     * @return Cache future.
      */
-    private GridDhtColocatedLockFuture colocatedLockFuture(IgniteInternalTx tx) {
+    private IgniteInternalFuture mvccFuture(IgniteInternalTx tx, Class<? extends IgniteInternalFuture> cls) {
         assert tx instanceof GridNearTxLocal : tx;
 
         Collection<GridCacheMvccFuture<?>> futs = cctx.mvcc().mvccFutures(tx.nearXidVersion());
 
         if (futs != null) {
             for (GridCacheMvccFuture<?> fut : futs) {
-                if (fut instanceof GridDhtColocatedLockFuture)
-                    return (GridDhtColocatedLockFuture)fut;
+                if (fut.getClass().equals(cls))
+                    return fut;
             }
         }
 
@@ -2115,6 +2137,16 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @return Collection of active transaction deadlock detection futures.
+     */
+    @SuppressWarnings("unchecked")
+    public Collection<IgniteInternalFuture<?>> deadlockDetectionFutures() {
+        Collection<? extends IgniteInternalFuture<?>> values = deadlockDetectFuts.values();
+
+        return (Collection<IgniteInternalFuture<?>>)values;
+    }
+
+    /**
      * Timeout object for node failure handler.
      */
     private final class NodeFailureTimeoutObject extends GridTimeoutObjectAdapter {

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
index c116d0d..f23cca9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxStateImpl.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.transactions;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
@@ -39,6 +40,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentLinkedHashMap;
 
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
@@ -362,6 +364,13 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter {
         return txMap == null ? Collections.<IgniteTxEntry>emptySet() : txMap.values();
     }
 
+    /**
+     * @return All entries. Returned collection is copy of internal collection.
+     */
+    public synchronized Collection<IgniteTxEntry> allEntriesCopy() {
+        return txMap == null ? Collections.<IgniteTxEntry>emptySet() : new HashSet<>(txMap.values());
+    }
+
     /** {@inheritDoc} */
     @Override public IgniteTxEntry entry(IgniteTxKey key) {
         return txMap == null ? null : txMap.get(key);
@@ -408,7 +417,7 @@ public class IgniteTxStateImpl extends IgniteTxLocalStateAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void addEntry(IgniteTxEntry entry) {
+    @Override public synchronized void addEntry(IgniteTxEntry entry) {
         txMap.put(entry.txKey(), entry);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java
index 36843dd..70d938e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetection.java
@@ -36,8 +36,11 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_TIMEOUT;
@@ -106,22 +109,30 @@ public class TxDeadlockDetection {
         stack.push(txId);
 
         while (!stack.isEmpty()) {
-            GridCacheVersion v = stack.pop();
+            GridCacheVersion v = stack.peek();
+
+            if (visited.contains(v)) {
+                stack.pop();
+                inPath.remove(v);
 
-            if (visited.contains(v))
                 continue;
+            }
 
             visited.add(v);
 
             Set<GridCacheVersion> children = wfg.get(v);
 
-            if (children == null || children.isEmpty())
+            if (children == null || children.isEmpty()) {
+                stack.pop();
+                inPath.remove(v);
+
                 continue;
+            }
 
             inPath.add(v);
 
             for (GridCacheVersion w : children) {
-                if (inPath.contains(w)) {
+                if (inPath.contains(w) && visited.contains(w)) {
                     List<GridCacheVersion> cycle = new ArrayList<>();
 
                     for (GridCacheVersion x = v; !x.equals(w); x = edgeTo.get(x))
@@ -158,15 +169,18 @@ public class TxDeadlockDetection {
         private final Set<IgniteTxKey> keys;
 
         /** Processed keys. */
+        @GridToStringInclude
         private final Set<IgniteTxKey> processedKeys = new HashSet<>();
 
         /** Processed nodes. */
         private final Set<UUID> processedNodes = new HashSet<>();
 
         /** Pending keys. */
+        @GridToStringInclude
         private Map<UUID, Set<IgniteTxKey>> pendingKeys = new HashMap<>();
 
         /** Nodes queue. */
+        @GridToStringInclude
         private final UniqueDeque<UUID> nodesQueue = new UniqueDeque<>();
 
         /** Preferred nodes. */
@@ -194,6 +208,7 @@ public class TxDeadlockDetection {
         private int itersCnt;
 
         /** Timeout object. */
+        @GridToStringExclude
         private DeadlockTimeoutObject timeoutObj;
 
         /** Timed out flag. */
@@ -252,8 +267,8 @@ public class TxDeadlockDetection {
 
             if (topVer == null) // Tx manager already stopped
                 onDone();
-
-            map(keys, Collections.<IgniteTxKey, TxLockList>emptyMap());
+            else
+                map(keys, Collections.<IgniteTxKey, TxLockList>emptyMap());
         }
 
         /**
@@ -441,14 +456,17 @@ public class TxDeadlockDetection {
          * @param txLocks Tx locks.
          */
         private void updateWaitForGraph(Map<IgniteTxKey, TxLockList> txLocks) {
+            if (txLocks == null || txLocks.isEmpty())
+                return;
+
             for (Map.Entry<IgniteTxKey, TxLockList> e : txLocks.entrySet()) {
 
                 GridCacheVersion txOwner = null;
 
                 for (TxLock lock : e.getValue().txLocks()) {
-                    if (lock.owner()) {
-                        assert txOwner == null;
-
+                    if (lock.owner() && txOwner == null) {
+                        // Actually we can get lock list with more than one owner. In this case ignore all owners
+                        // except first because likely the first owner was cause of deadlock.
                         txOwner = lock.txId();
 
                         if (keys.contains(e.getKey()) && !txId.equals(lock.txId())) {
@@ -463,7 +481,7 @@ public class TxDeadlockDetection {
                         continue;
                     }
 
-                    if (lock.candiate()) {
+                    if (lock.candiate() || lock.owner()) {
                         GridCacheVersion txId0 = lock.txId();
 
                         Set<GridCacheVersion> waitForTxs = wfg.get(txId0);
@@ -485,9 +503,9 @@ public class TxDeadlockDetection {
 
             if (res != null && set) {
                 if (res.classError() != null) {
-                    IgniteLogger log = cctx.logger(TxDeadlockDetection.class);
+                    IgniteLogger log = cctx.kernalContext().log(this.getClass());
 
-                    log.warning("Failed to finish deadlock detection due to an error: " + nodeId);
+                    U.warn(log, "Failed to finish deadlock detection due to an error: " + nodeId);
 
                     onDone();
                 }
@@ -528,6 +546,11 @@ public class TxDeadlockDetection {
             return false;
         }
 
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TxDeadlockFuture.class, this);
+        }
+
         /**
          * Lock request timeout object.
          */
@@ -543,6 +566,10 @@ public class TxDeadlockDetection {
             @Override public void onTimeout() {
                 timedOut = true;
 
+                IgniteLogger log = cctx.kernalContext().log(this.getClass());
+
+                U.warn(log, "Deadlock detection was timed out [timeout=" + DEADLOCK_TIMEOUT + ", fut=" + this + ']');
+
                 onDone();
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java
index 94b5620..2b524e8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConfigCacheSelfTest.java
@@ -18,15 +18,29 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import javax.cache.CacheException;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.TransactionConfiguration;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 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;
@@ -34,11 +48,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionTimeoutException;
 
-import javax.cache.CacheException;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.EntryProcessorException;
-import javax.cache.processor.MutableEntry;
-
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
@@ -62,6 +71,10 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
 
+        TcpCommunicationSpi commSpi = new TestCommunicationSpi();
+
+        cfg.setCommunicationSpi(commSpi);
+
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setAtomicityMode(atomicityMode());
@@ -87,7 +100,7 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(1);
+        startGrids(2);
     }
 
     /** {@inheritDoc} */
@@ -98,7 +111,7 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
     /**
      * Success if user tx was timed out.
      *
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testUserTxTimeout() throws Exception {
         final Ignite ignite = grid(0);
@@ -112,7 +125,7 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
     /**
      * Success if system caches weren't timed out.
      *
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testSystemCacheTx() throws Exception {
         final Ignite ignite = grid(0);
@@ -143,27 +156,23 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
      * Success if implicit tx fails.
      *
      * @param cache Cache name.
-     * @throws Exception
+     * @throws Exception If failed.
      */
     protected void checkImplicitTxTimeout(final IgniteCache<Object, Object> cache) throws Exception {
-        try {
-            cache.invoke("key", new EntryProcessor<Object, Object, Object>() {
-                @Override public Object process(final MutableEntry<Object, Object> entry, final Object... args)
-                    throws EntryProcessorException {
-                    try {
-                        sleepForTxFailure();
-                    } catch (InterruptedException e) {
-                        throw new EntryProcessorException(e);
-                    }
+        TestCommunicationSpi.delay = true;
 
-                    return null;
-                }
-            });
+        Integer key = primaryKey(ignite(1).cache(CACHE_NAME));
+
+        try {
+            cache.put(key, 0);
 
             fail("Timeout exception must be thrown");
         }
         catch (CacheException e) {
-            // OK
+            // No-op.
+        }
+        finally {
+            TestCommunicationSpi.delay = false;
         }
 
         cache.clear();
@@ -174,7 +183,7 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
      *
      * @param cache Cache name.
      * @param ignite Ignite instance.
-     * @throws Exception
+     * @throws Exception If failed.
      */
     protected void checkExplicitTxTimeout(final IgniteCache<Object, Object> cache, final Ignite ignite)
         throws Exception {
@@ -198,7 +207,7 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
      * Success if explicit tx doesn't fail.
      *
      * @param cache Cache instance.
-     * @throws Exception
+     * @throws Exception If failed.
      */
     protected void checkStartTxSuccess(final IgniteInternalCache<Object, Object> cache) throws Exception {
         try (final IgniteInternalTx tx = CU.txStartInternal(cache.context(), cache, PESSIMISTIC, READ_COMMITTED)) {
@@ -220,7 +229,7 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
      * Success if implicit tx fails.
      *
      * @param cache Cache instance.
-     * @throws Exception
+     * @throws Exception If failed.
      */
     protected void checkImplicitTxSuccess(final IgniteInternalCache<Object, Object> cache) throws Exception {
         cache.invoke("key", new EntryProcessor<Object, Object, Object>() {
@@ -241,9 +250,39 @@ public class IgniteTxConfigCacheSelfTest extends GridCommonAbstractTest {
     /**
      * Sleep multiple {@link #TX_TIMEOUT} times.
      *
-     * @throws InterruptedException
+     * @throws InterruptedException If interrupted.
      */
     private void sleepForTxFailure() throws InterruptedException {
         Thread.sleep(TX_TIMEOUT * 3);
     }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** Delay. */
+        private static volatile boolean delay;
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(
+            final ClusterNode node,
+            final Message msg,
+            final IgniteInClosure<IgniteException> ackC
+        ) throws IgniteSpiException {
+            if (msg instanceof GridIoMessage) {
+                Message msg0 = ((GridIoMessage)msg).message();
+
+                if (msg0 instanceof GridNearTxPrepareRequest && delay) {
+                    try {
+                        U.sleep(TX_TIMEOUT * 2);
+                    }
+                    catch (IgniteInterruptedCheckedException e) {
+                        e.printStackTrace();
+                    }
+                }
+            }
+
+            super.sendMessage(node, msg, ackC);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
index c417821..8475175 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxTimeoutAbstractTest.java
@@ -20,10 +20,10 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
-import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -154,8 +154,8 @@ public class IgniteTxTimeoutAbstractTest extends GridCommonAbstractTest {
 
             assert false : "Timeout never happened for transaction: " + tx;
         }
-        catch (CacheException e) {
-            if (!(e.getCause() instanceof TransactionTimeoutException))
+        catch (Exception e) {
+            if (!(X.hasCause(e, TransactionTimeoutException.class)))
                 throw e;
 
             info("Received expected timeout exception [msg=" + e.getMessage() + ", tx=" + tx + ']');
@@ -164,4 +164,4 @@ public class IgniteTxTimeoutAbstractTest extends GridCommonAbstractTest {
             tx.close();
         }
     }
-}
\ No newline at end of file
+}

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java
index cfa93ac..e27207d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheLocalTxTimeoutSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.local;
 
-import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
@@ -154,7 +153,7 @@ public class GridCacheLocalTxTimeoutSelfTest extends GridCommonAbstractTest {
 
             tx.commit();
         }
-        catch (CacheException e) {
+        catch (Exception e) {
             assertTrue(X.hasCause(e, TransactionTimeoutException.class));
 
             info("Received expected optimistic exception: " + e.getMessage());
@@ -166,4 +165,4 @@ public class GridCacheLocalTxTimeoutSelfTest extends GridCommonAbstractTest {
 
         assert wasEx;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java
index 20467c2..b0a407c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/DepthFirstSearchTest.java
@@ -24,10 +24,12 @@ import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 import java.util.Set;
 import junit.framework.TestCase;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetection.findCycle;
 
@@ -95,6 +97,14 @@ public class DepthFirstSearchTest extends TestCase {
 
         wfg = new HashMap<GridCacheVersion, Set<GridCacheVersion>>() {{
             put(T1, new HashSet<GridCacheVersion>(){{add(T2);}});
+            put(T2, new HashSet<GridCacheVersion>(){{add(T3);}});
+            put(T4, new HashSet<GridCacheVersion>(){{add(T1); add(T2); add(T3);}});
+        }};
+
+        assertAllNull(wfg);
+
+        wfg = new HashMap<GridCacheVersion, Set<GridCacheVersion>>() {{
+            put(T1, new HashSet<GridCacheVersion>(){{add(T2);}});
             put(T3, new HashSet<GridCacheVersion>(){{add(T4);}});
             put(T4, new HashSet<GridCacheVersion>(){{add(T1);}});
         }};
@@ -228,6 +238,94 @@ public class DepthFirstSearchTest extends TestCase {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testFindCycle4() throws Exception {
+        Map<GridCacheVersion, Set<GridCacheVersion>> wfg = new HashMap<GridCacheVersion, Set<GridCacheVersion>>() {{
+            put(T1, Collections.singleton(T2));
+            put(T2, asLinkedHashSet(T3, T4));
+            put(T3, Collections.singleton(T4));
+            put(T4, Collections.singleton(T5));
+            put(T6, Collections.singleton(T3));
+        }};
+
+        assertNull(findCycle(wfg, T1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomNoExceptions() throws Exception {
+        int maxNodesCnt = 100;
+        int minNodesCnt = 10;
+        int maxWaitForNodesCnt = 20;
+
+        int cyclesFound = 0;
+        int cyclesNotFound = 0;
+
+        Random seedRnd = new Random();
+
+        Random rnd = new Random();
+
+        for (int i = 0; i < 50000; i++) {
+            long seed = seedRnd.nextLong();
+
+            rnd.setSeed(seed);
+
+            System.out.println(">>> Iteration " + i + " with seed " + seed);
+
+            int nodesCnt = rnd.nextInt(maxNodesCnt - minNodesCnt) + minNodesCnt;
+
+            Map<GridCacheVersion, Set<GridCacheVersion>> wfg = new HashMap<>();
+
+            for (int j = 0; j < nodesCnt; j++) {
+                if (rnd.nextInt(100) > 30) {
+                    int waitForNodesCnt = rnd.nextInt(maxWaitForNodesCnt);
+
+                    Set<GridCacheVersion> waitForNodes = null;
+
+                    if (waitForNodesCnt > 0) {
+                        waitForNodes = new LinkedHashSet<>();
+
+                        for (int k = 0; k < waitForNodesCnt;) {
+                            int n = rnd.nextInt(nodesCnt);
+
+                            if (n != j) {
+                                waitForNodes.add(new GridCacheVersion(n, 0, 0, 0));
+                                k++;
+                            }
+                        }
+                    }
+
+                    wfg.put(new GridCacheVersion(j, 0, 0, 0), waitForNodes);
+                }
+            }
+
+            for (int j = 0; j < nodesCnt; j++) {
+                try {
+                    List<GridCacheVersion> cycle = findCycle(wfg, new GridCacheVersion(j, 0, 0, 0));
+
+                    if (cycle == null)
+                        cyclesNotFound++;
+                    else
+                        cyclesFound++;
+                }
+                catch (Throwable e) {
+                    U.error(null, "Error during finding cycle in graph: ", e);
+
+                    U.warn(null, "Seed: " + seed);
+
+                    U.warn(null, "Wait-for-graph: " + wfg);
+
+                    fail();
+                }
+            }
+        }
+
+        System.out.println(">>> Test finished. Cycles found: " + cyclesFound + ", cycles not found: " + cyclesNotFound);
+    }
+
+    /**
      * @param wfg Wait-for-graph.
      */
     private static void assertAllNull(Map<GridCacheVersion, Set<GridCacheVersion>> wfg, GridCacheVersion... ignore) {
@@ -249,4 +347,4 @@ public class DepthFirstSearchTest extends TestCase {
 
         return set;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
new file mode 100644
index 0000000..c9d18eb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
@@ -0,0 +1,246 @@
+/*
+ * 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.transactions;
+
+import java.util.Collection;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_TIMEOUT;
+import static org.apache.ignite.IgniteSystemProperties.getInteger;
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ *
+ */
+public class TxDeadlockDetectionNoHangsTest extends GridCommonAbstractTest {
+    /** Nodes count. */
+    private static final int NODES_CNT = 3;
+
+    /** Cache. */
+    private static final String CACHE = "cache";
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setName(CACHE);
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setNearConfiguration(null);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGridsMultiThreaded(NODES_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        GridTestUtils.setFieldValue(null, TxDeadlockDetection.class, "DEADLOCK_TIMEOUT", (int)(getTestTimeout() * 2));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        GridTestUtils.setFieldValue(null, TxDeadlockDetection.class, "DEADLOCK_TIMEOUT",
+            getInteger(IGNITE_TX_DEADLOCK_DETECTION_TIMEOUT, 60000));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 10 * 60 * 1000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoHangsPessimistic() throws Exception {
+        assertTrue(grid(0).context().cache().context().tm().deadlockDetectionEnabled());
+
+        doTest(PESSIMISTIC);
+
+        try {
+            GridTestUtils.setFieldValue(null, IgniteTxManager.class, "DEADLOCK_MAX_ITERS", 0);
+
+            assertFalse(grid(0).context().cache().context().tm().deadlockDetectionEnabled());
+
+            doTest(PESSIMISTIC);
+        }
+        finally {
+            GridTestUtils.setFieldValue(null, IgniteTxManager.class, "DEADLOCK_MAX_ITERS",
+                IgniteSystemProperties.getInteger(IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS, 1000));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoHangsOptimistic() throws Exception {
+        assertTrue(grid(0).context().cache().context().tm().deadlockDetectionEnabled());
+
+        doTest(OPTIMISTIC);
+
+        try {
+            GridTestUtils.setFieldValue(null, IgniteTxManager.class, "DEADLOCK_MAX_ITERS", 0);
+
+            assertFalse(grid(0).context().cache().context().tm().deadlockDetectionEnabled());
+
+            doTest(OPTIMISTIC);
+        }
+        finally {
+            GridTestUtils.setFieldValue(null, IgniteTxManager.class, "DEADLOCK_MAX_ITERS",
+                IgniteSystemProperties.getInteger(IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS, 1000));
+        }
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void doTest(final TransactionConcurrency concurrency) throws IgniteCheckedException {
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        IgniteInternalFuture<Long> restartFut = null;
+
+        try {
+            restartFut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+                @Override public void run() {
+                    while (!stop.get()) {
+                        try {
+                            U.sleep(500);
+
+                            startGrid(NODES_CNT);
+
+                            awaitPartitionMapExchange();
+
+                            U.sleep(500);
+
+                            stopGrid(NODES_CNT);
+                        }
+                        catch (Exception e) {
+                            // No-op.
+                        }
+                    }
+                }
+            }, 1, "restart-thread");
+
+            long stopTime = System.currentTimeMillis() + 2 * 60_000L;
+
+            for (int i = 0; System.currentTimeMillis() < stopTime; i++) {
+                boolean detectionEnabled = grid(0).context().cache().context().tm().deadlockDetectionEnabled();
+
+                log.info(">>> Iteration " + i + " (detection is " + (detectionEnabled ? "enabled" : "disabled") + ')');
+
+                final AtomicInteger threadCnt = new AtomicInteger();
+
+                IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+                    @Override public void run() {
+                        int threadNum = threadCnt.getAndIncrement();
+
+                        Ignite ignite = ignite(threadNum % NODES_CNT);
+
+                        IgniteCache<Integer, Integer> cache = ignite.cache(CACHE);
+
+                        try (Transaction tx = ignite.transactions().txStart(concurrency, REPEATABLE_READ, 500, 0)) {
+                            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                            for (int i = 0; i < 50; i++) {
+                                int key = rnd.nextInt(50);
+
+                                if (log.isDebugEnabled()) {
+                                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                                        ", tx=" + tx + ", key=" + key + ']');
+                                }
+
+                                cache.put(key, 0);
+                            }
+
+                            tx.commit();
+                        }
+                        catch (Exception e) {
+                            e.printStackTrace();
+                        }
+                    }
+                }, NODES_CNT * 3, "tx-thread");
+
+                fut.get();
+            }
+        }
+        finally {
+            stop.set(true);
+
+            if (restartFut != null)
+                restartFut.get();
+
+            checkDetectionFutures();
+        }
+    }
+
+    /**
+     *
+     */
+    private void checkDetectionFutures() {
+        for (int i = 0; i < NODES_CNT ; i++) {
+            Ignite ignite = ignite(i);
+
+            IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
+
+            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
+
+            assertTrue(futs.isEmpty());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java
index 3d0beac..87bc70f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionTest.java
@@ -21,8 +21,9 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import java.util.concurrent.ConcurrentMap;
+import java.util.Collection;
 import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -47,7 +48,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionDeadlockException;
 import org.apache.ignite.transactions.TransactionTimeoutException;
-import org.jsr166.ThreadLocalRandom8;
 
 import static org.apache.ignite.internal.util.typedef.X.hasCause;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
@@ -152,7 +152,7 @@ public class TxDeadlockDetectionTest extends GridCommonAbstractTest {
                         IgniteCache<Integer, Integer> cache = ignite.cache(CACHE);
 
                         try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 700, 0)) {
-                            ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+                            ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
                             for (int i = 0; i < 50; i++) {
                                 int key = rnd.nextInt(50);
@@ -217,7 +217,7 @@ public class TxDeadlockDetectionTest extends GridCommonAbstractTest {
 
                     cache.put(key, 0);
 
-                    barrier.await(timeout + 100, TimeUnit.MILLISECONDS);
+                    barrier.await(timeout + 1000, TimeUnit.MILLISECONDS);
 
                     tx.commit();
                 }
@@ -281,7 +281,7 @@ public class TxDeadlockDetectionTest extends GridCommonAbstractTest {
                             log.info(">>> Performs sleep. [node=" + ((IgniteKernal)ignite).localNode() +
                                 ", tx=" + tx + ']');
 
-                            U.sleep(timeout * 2);
+                            U.sleep(timeout * 3);
                         }
                         else {
                             int key2 = threadNum + 1;
@@ -406,8 +406,7 @@ public class TxDeadlockDetectionTest extends GridCommonAbstractTest {
 
             IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
 
-            ConcurrentMap<Long, TxDeadlockDetection.TxDeadlockFuture> futs =
-                GridTestUtils.getFieldValue(txMgr, IgniteTxManager.class, "deadlockDetectFuts");
+            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
 
             assertTrue(futs.isEmpty());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java
new file mode 100644
index 0000000..7b40da2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionCrossCacheTest.java
@@ -0,0 +1,257 @@
+/*
+ * 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.transactions;
+
+import java.util.Collection;
+import java.util.Set;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionDeadlockException;
+import org.apache.ignite.transactions.TransactionTimeoutException;
+
+import static org.apache.ignite.internal.util.typedef.X.hasCause;
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ *
+ */
+public class TxOptimisticDeadlockDetectionCrossCacheTest extends GridCommonAbstractTest {
+    /** Nodes count. */
+    private static final int NODES_CNT = 2;
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (isDebug()) {
+            TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+            discoSpi.failureDetectionTimeoutEnabled(false);
+
+            cfg.setDiscoverySpi(discoSpi);
+        }
+
+        TcpCommunicationSpi commSpi = new TestCommunicationSpi();
+
+        cfg.setCommunicationSpi(commSpi);
+
+        CacheConfiguration ccfg0 = defaultCacheConfiguration();
+
+        ccfg0.setName("cache0");
+        ccfg0.setCacheMode(CacheMode.PARTITIONED);
+        ccfg0.setBackups(1);
+        ccfg0.setNearConfiguration(null);
+
+        CacheConfiguration ccfg1 = defaultCacheConfiguration();
+
+        ccfg1.setName("cache1");
+        ccfg1.setCacheMode(CacheMode.PARTITIONED);
+        ccfg1.setBackups(1);
+        ccfg1.setNearConfiguration(null);
+
+        cfg.setCacheConfiguration(ccfg0, ccfg1);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODES_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlock() throws Exception {
+        // Sometimes boh transactions perform commit, so we repeat attempt.
+        while (!doTestDeadlock()) {}
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private boolean doTestDeadlock() throws Exception {
+        TestCommunicationSpi.init(2);
+
+        final CyclicBarrier barrier = new CyclicBarrier(2);
+
+        final AtomicInteger threadCnt = new AtomicInteger();
+
+        final AtomicBoolean deadlock = new AtomicBoolean();
+
+        final AtomicInteger commitCnt = new AtomicInteger();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                int threadNum = threadCnt.getAndIncrement();
+
+                Ignite ignite = ignite(0);
+
+                IgniteCache<Integer, Integer> cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1));
+
+                IgniteCache<Integer, Integer> cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0));
+
+                try (Transaction tx =
+                         ignite.transactions().txStart(OPTIMISTIC, REPEATABLE_READ, 500, 0)
+                ) {
+                    int key1 = primaryKey(cache1);
+
+                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                        ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']');
+
+                    cache1.put(key1, 0);
+
+                    barrier.await();
+
+                    int key2 = primaryKey(cache2);
+
+                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                        ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']');
+
+                    cache2.put(key2, 1);
+
+                    tx.commit();
+
+                    commitCnt.incrementAndGet();
+                }
+                catch (Throwable e) {
+                    // At least one stack trace should contain TransactionDeadlockException.
+                    if (hasCause(e, TransactionTimeoutException.class) &&
+                        hasCause(e, TransactionDeadlockException.class)
+                        ) {
+                        if (deadlock.compareAndSet(false, true))
+                            U.error(log, "At least one stack trace should contain " +
+                                TransactionDeadlockException.class.getSimpleName(), e);
+                    }
+                }
+            }
+        }, 2, "tx-thread");
+
+        fut.get();
+
+        if (commitCnt.get() == 2)
+            return false;
+
+        assertTrue(deadlock.get());
+
+        for (int i = 0; i < NODES_CNT ; i++) {
+            Ignite ignite = ignite(i);
+
+            IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
+
+            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
+
+            assertTrue(futs.isEmpty());
+        }
+
+        return true;
+    }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** Tx count. */
+        private static volatile int TX_CNT;
+
+        /** Tx ids. */
+        private static final Set<GridCacheVersion> TX_IDS = new GridConcurrentHashSet<>();
+
+        /**
+         * @param txCnt Tx count.
+         */
+        private static void init(int txCnt) {
+            TX_CNT = txCnt;
+            TX_IDS.clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(
+            final ClusterNode node,
+            final Message msg,
+            final IgniteInClosure<IgniteException> ackC
+        ) throws IgniteSpiException {
+            if (msg instanceof GridIoMessage) {
+                Message msg0 = ((GridIoMessage)msg).message();
+
+                if (msg0 instanceof GridNearTxPrepareRequest) {
+                    final GridNearTxPrepareRequest req = (GridNearTxPrepareRequest)msg0;
+
+                    GridCacheVersion txId = req.version();
+
+                    if (TX_IDS.contains(txId)) {
+                        while (TX_IDS.size() < TX_CNT) {
+                            try {
+                                U.sleep(50);
+                            }
+                            catch (IgniteInterruptedCheckedException e) {
+                                e.printStackTrace();
+                            }
+                        }
+                    }
+                }
+                else if (msg0 instanceof GridNearTxPrepareResponse) {
+                    GridNearTxPrepareResponse res = (GridNearTxPrepareResponse)msg0;
+
+                    GridCacheVersion txId = res.version();
+
+                    TX_IDS.add(txId);
+                }
+            }
+
+            super.sendMessage(node, msg, ackC);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
new file mode 100644
index 0000000..aa240aa
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
@@ -0,0 +1,574 @@
+/*
+ * 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.transactions;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
+import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionDeadlockException;
+import org.apache.ignite.transactions.TransactionTimeoutException;
+
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE;
+import static org.apache.ignite.internal.util.typedef.X.cause;
+import static org.apache.ignite.internal.util.typedef.X.hasCause;
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ *
+ */
+public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** Nodes count (actually two times more nodes will started: server + client). */
+    private static final int NODES_CNT = 4;
+
+    /** No op transformer. */
+    private static final NoOpTransformer NO_OP_TRANSFORMER = new NoOpTransformer();
+
+    /** Wrapping transformer. */
+    private static final WrappingTransformer WRAPPING_TRANSFORMER = new WrappingTransformer();
+
+    /** Client mode flag. */
+    private static boolean client;
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (isDebug()) {
+            TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+            discoSpi.failureDetectionTimeoutEnabled(false);
+
+            cfg.setDiscoverySpi(discoSpi);
+        }
+
+        TcpCommunicationSpi commSpi = new TestCommunicationSpi();
+
+        cfg.setCommunicationSpi(commSpi);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        client = false;
+
+        startGrids(NODES_CNT);
+
+        client = true;
+
+        for (int i = 0; i < NODES_CNT; i++)
+            startGrid(i + NODES_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlocksPartitioned() throws Exception {
+        for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
+            doTestDeadlocks(createCache(PARTITIONED, syncMode, false), NO_OP_TRANSFORMER);
+            doTestDeadlocks(createCache(PARTITIONED, syncMode, false), WRAPPING_TRANSFORMER);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlocksPartitionedNear() throws Exception {
+        for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
+            doTestDeadlocks(createCache(PARTITIONED, syncMode, true), NO_OP_TRANSFORMER);
+            doTestDeadlocks(createCache(PARTITIONED, syncMode, true), WRAPPING_TRANSFORMER);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlocksReplicated() throws Exception {
+        for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
+            doTestDeadlocks(createCache(REPLICATED, syncMode, false), NO_OP_TRANSFORMER);
+            doTestDeadlocks(createCache(REPLICATED, syncMode, false), WRAPPING_TRANSFORMER);
+        }
+    }
+
+    /**
+     * @param cacheMode Cache mode.
+     * @param syncMode Write sync mode.
+     * @param near Near.
+     * @return Created cache.
+     */
+    @SuppressWarnings("unchecked")
+    private IgniteCache createCache(CacheMode cacheMode, CacheWriteSynchronizationMode syncMode, boolean near) {
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setName(CACHE_NAME);
+        ccfg.setCacheMode(cacheMode);
+        ccfg.setBackups(1);
+        ccfg.setNearConfiguration(near ? new NearCacheConfiguration() : null);
+        ccfg.setWriteSynchronizationMode(syncMode);
+
+        IgniteCache cache = ignite(0).createCache(ccfg);
+
+        if (near) {
+            for (int i = 0; i < NODES_CNT; i++) {
+                Ignite client = ignite(i + NODES_CNT);
+
+                assertTrue(client.configuration().isClientMode());
+
+                client.createNearCache(ccfg.getName(), new NearCacheConfiguration<>());
+            }
+        }
+
+        return cache;
+    }
+
+    /**
+     * @param cache Cache.
+     * @param transformer Transformer closure.
+     * @throws Exception If failed.
+     */
+    private void doTestDeadlocks(IgniteCache cache, IgniteClosure<Integer, Object> transformer) throws Exception {
+        try {
+            awaitPartitionMapExchange();
+
+            doTestDeadlock(3, false, true, true, transformer);
+            doTestDeadlock(3, false, false, false, transformer);
+            doTestDeadlock(3, false, false, true, transformer);
+
+            doTestDeadlock(4, false, true, true, transformer);
+            doTestDeadlock(4, false, false, false, transformer);
+            doTestDeadlock(4, false, false, true, transformer);
+        }
+        catch (Throwable e) {
+            U.error(log, "Unexpected exception: ", e);
+
+            fail();
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestDeadlock(
+        final int txCnt,
+        final boolean loc,
+        boolean lockPrimaryFirst,
+        final boolean clientTx,
+        final IgniteClosure<Integer, Object> transformer
+    ) throws Exception {
+        log.info(">>> Test deadlock [txCnt=" + txCnt + ", loc=" + loc + ", lockPrimaryFirst=" + lockPrimaryFirst +
+            ", clientTx=" + clientTx + ", transformer=" + transformer.getClass().getName() + ']');
+
+        TestCommunicationSpi.init(txCnt);
+
+        final AtomicInteger threadCnt = new AtomicInteger();
+
+        final CyclicBarrier barrier = new CyclicBarrier(txCnt);
+
+        final AtomicReference<TransactionDeadlockException> deadlockErr = new AtomicReference<>();
+
+        final List<List<Integer>> keySets = generateKeys(txCnt, loc, !lockPrimaryFirst);
+
+        final Set<Integer> involvedKeys = new GridConcurrentHashSet<>();
+        final Set<Integer> involvedLockedKeys = new GridConcurrentHashSet<>();
+        final Set<IgniteInternalTx> involvedTxs = new GridConcurrentHashSet<>();
+
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+            @Override public void run() {
+                int threadNum = threadCnt.incrementAndGet();
+
+                Ignite ignite = loc ? ignite(0) : ignite(clientTx ? threadNum - 1 + txCnt : threadNum - 1);
+
+                IgniteCache<Object, Integer> cache = ignite.cache(CACHE_NAME);
+
+                List<Integer> keys = keySets.get(threadNum - 1);
+
+                int txTimeout = 500 + txCnt * 100;
+
+                try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, REPEATABLE_READ, txTimeout, 0)) {
+                    IgniteInternalTx tx0 = ((TransactionProxyImpl)tx).tx();
+
+                    involvedTxs.add(tx0);
+
+                    Integer key = keys.get(0);
+
+                    involvedKeys.add(key);
+
+                    Object k;
+
+                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                        ", tx=" + tx + ", key=" + transformer.apply(key) + ']');
+
+                    cache.put(transformer.apply(key), 0);
+
+                    involvedLockedKeys.add(key);
+
+                    barrier.await();
+
+                    key = keys.get(1);
+
+                    ClusterNode primaryNode =
+                        ((IgniteCacheProxy)cache).context().affinity().primary(key, NONE);
+
+                    List<Integer> primaryKeys =
+                        primaryKeys(grid(primaryNode).cache(CACHE_NAME), 5, key + (100 * threadNum));
+
+                    Map<Object, Integer> entries = new HashMap<>();
+
+                    involvedKeys.add(key);
+
+                    entries.put(transformer.apply(key), 0);
+
+                    for (Integer i : primaryKeys) {
+                        involvedKeys.add(i);
+
+                        entries.put(transformer.apply(i), 1);
+
+                        k = transformer.apply(i + 13);
+
+                        involvedKeys.add(i + 13);
+
+                        entries.put(k, 2);
+                    }
+
+                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                        ", tx=" + tx + ", entries=" + entries + ']');
+
+                    cache.putAll(entries);
+
+                    tx.commit();
+                }
+                catch (Throwable e) {
+                    U.error(log, "Expected exception: ", e);
+
+                    // At least one stack trace should contain TransactionDeadlockException.
+                    if (hasCause(e, TransactionTimeoutException.class) &&
+                        hasCause(e, TransactionDeadlockException.class)
+                        ) {
+                        if (deadlockErr.compareAndSet(null, cause(e, TransactionDeadlockException.class)))
+                            U.error(log, "At least one stack trace should contain " +
+                                TransactionDeadlockException.class.getSimpleName(), e);
+                    }
+                }
+            }
+        }, loc ? 2 : txCnt, "tx-thread");
+
+        try {
+            fut.get();
+        }
+        catch (IgniteCheckedException e) {
+            U.error(null, "Unexpected exception", e);
+
+            fail();
+        }
+
+        U.sleep(1000);
+
+        TransactionDeadlockException deadlockE = deadlockErr.get();
+
+        assertNotNull(deadlockE);
+
+        boolean fail = false;
+
+        // Check transactions, futures and entry locks state.
+        for (int i = 0; i < NODES_CNT * 2; i++) {
+            Ignite ignite = ignite(i);
+
+            int cacheId = ((IgniteCacheProxy)ignite.cache(CACHE_NAME)).context().cacheId();
+
+            GridCacheSharedContext<Object, Object> cctx = ((IgniteKernal)ignite).context().cache().context();
+
+            IgniteTxManager txMgr = cctx.tm();
+
+            Collection<IgniteInternalTx> activeTxs = txMgr.activeTransactions();
+
+            for (IgniteInternalTx tx : activeTxs) {
+                Collection<IgniteTxEntry> entries = tx.allEntries();
+
+                for (IgniteTxEntry entry : entries) {
+                    if (entry.cacheId() == cacheId) {
+                        fail = true;
+
+                        U.error(log, "Transaction still exists: " + "\n" + tx.xidVersion() +
+                            "\n" + tx.nearXidVersion() + "\n nodeId=" + cctx.localNodeId() + "\n tx=" + tx);
+                    }
+                }
+            }
+
+            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
+
+            assertTrue(futs.isEmpty());
+
+            GridCacheAdapter<Object, Integer> intCache = internalCache(i, CACHE_NAME);
+
+            GridCacheConcurrentMap map = intCache.map();
+
+            for (Integer key : involvedKeys) {
+                Object key0 = transformer.apply(key);
+
+                KeyCacheObject keyCacheObj = intCache.context().toCacheKeyObject(key0);
+
+                GridCacheMapEntry entry = map.getEntry(keyCacheObj);
+
+                if (entry != null)
+                    assertNull("Entry still has locks " + entry, entry.mvccAllLocal());
+            }
+        }
+
+        if (fail)
+            fail("Some transactions still exist");
+
+        // Check deadlock report
+        String msg = deadlockE.getMessage();
+
+        for (IgniteInternalTx tx : involvedTxs)
+            assertTrue(msg.contains(
+                "[txId=" + tx.xidVersion() + ", nodeId=" + tx.nodeId() + ", threadId=" + tx.threadId() + ']'));
+
+        for (Integer key : involvedKeys) {
+            if (involvedLockedKeys.contains(key))
+                assertTrue(msg.contains("[key=" + transformer.apply(key) + ", cache=" + CACHE_NAME + ']'));
+            else
+                assertFalse(msg.contains("[key=" + transformer.apply(key)));
+        }
+    }
+
+    /**
+     * @param nodesCnt Nodes count.
+     * @param loc Local cache.
+     */
+    private List<List<Integer>> generateKeys(int nodesCnt, boolean loc, boolean reverse) throws IgniteCheckedException {
+        List<List<Integer>> keySets = new ArrayList<>();
+
+        if (loc) {
+            List<Integer> keys = primaryKeys(ignite(0).cache(CACHE_NAME), 2);
+
+            keySets.add(new ArrayList<>(keys));
+
+            Collections.reverse(keys);
+
+            keySets.add(keys);
+        }
+        else {
+            for (int i = 0; i < nodesCnt; i++) {
+                List<Integer> keys = new ArrayList<>(2);
+
+                int n1 = i + 1;
+                int n2 = n1 + 1;
+
+                int i1 = n1 < nodesCnt ? n1 : n1 - nodesCnt;
+                int i2 = n2 < nodesCnt ? n2 : n2 - nodesCnt;
+
+                keys.add(primaryKey(ignite(i1).cache(CACHE_NAME)));
+                keys.add(primaryKey(ignite(i2).cache(CACHE_NAME)));
+
+                if (reverse)
+                    Collections.reverse(keys);
+
+                keySets.add(keys);
+            }
+        }
+
+        return keySets;
+    }
+
+    /**
+     *
+     */
+    private static class NoOpTransformer implements IgniteClosure<Integer, Object> {
+        /** {@inheritDoc} */
+        @Override public Object apply(Integer val) {
+            return val;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class WrappingTransformer implements IgniteClosure<Integer, Object> {
+        /** {@inheritDoc} */
+        @Override public Object apply(Integer val) {
+            return new KeyObject(val);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class KeyObject implements Serializable {
+        /** Id. */
+        private int id;
+
+        /** Name. */
+        private String name;
+
+        /**
+         * @param id Id.
+         */
+        public KeyObject(int id) {
+            this.id = id;
+            this.name = "KeyObject" + id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "KeyObject{" +
+                "id=" + id +
+                ", name='" + name + '\'' +
+                '}';
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            KeyObject obj = (KeyObject)o;
+
+            return id == obj.id && name.equals(obj.name);
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return id;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** Tx count. */
+        private static volatile int TX_CNT;
+
+        /** Tx ids. */
+        private static final Set<GridCacheVersion> TX_IDS = new GridConcurrentHashSet<>();
+
+        /**
+         * @param txCnt Tx count.
+         */
+        private static void init(int txCnt) {
+            TX_CNT = txCnt;
+            TX_IDS.clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(
+            final ClusterNode node,
+            final Message msg,
+            final IgniteInClosure<IgniteException> ackC
+        ) throws IgniteSpiException {
+            if (msg instanceof GridIoMessage) {
+                Message msg0 = ((GridIoMessage)msg).message();
+
+                if (msg0 instanceof GridNearTxPrepareRequest) {
+                    final GridNearTxPrepareRequest req = (GridNearTxPrepareRequest)msg0;
+
+                    GridCacheVersion txId = req.version();
+
+                    if (TX_IDS.contains(txId)) {
+                        while (TX_IDS.size() < TX_CNT) {
+                            try {
+                                U.sleep(50);
+                            }
+                            catch (IgniteInterruptedCheckedException e) {
+                                e.printStackTrace();
+                            }
+                        }
+                    }
+                }
+                else if (msg0 instanceof GridNearTxPrepareResponse) {
+                    GridNearTxPrepareResponse res = (GridNearTxPrepareResponse)msg0;
+
+                    GridCacheVersion txId = res.version();
+
+                    TX_IDS.add(txId);
+                }
+            }
+
+            super.sendMessage(node, msg, ackC);
+        }
+    }
+}


[17/24] ignite git commit: IGNITE-3670 IGFS: Improved symlink handling for delete operation and added more tests. This closes #975.

Posted by sb...@apache.org.
IGNITE-3670 IGFS: Improved symlink handling for delete operation and added more tests. This closes #975.


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

Branch: refs/heads/master
Commit: b5757642e135908d9baa027a605035dd0d4acfc9
Parents: 92f18bf
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Fri Aug 26 15:47:02 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 26 15:47:02 2016 +0300

----------------------------------------------------------------------
 .../local/LocalIgfsSecondaryFileSystem.java     |   38 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          | 1067 ++++++++++++++++++
 .../processors/igfs/IgfsAbstractSelfTest.java   | 1012 +----------------
 ...SecondaryFileSystemDualAbstractSelfTest.java |  143 +++
 4 files changed, 1239 insertions(+), 1021 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b5757642/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
index 3d3a350..519f472 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
@@ -41,6 +41,8 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.file.Files;
+import java.nio.file.LinkOption;
+import java.nio.file.attribute.BasicFileAttributes;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
@@ -108,35 +110,43 @@ public class LocalIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, Li
     @Override public boolean delete(IgfsPath path, boolean recursive) {
         File f = fileForPath(path);
 
-        if (!recursive || !f.isDirectory())
+        if (!recursive)
             return f.delete();
         else
-            return deleteDirectory(f);
+            return deleteRecursive(f);
     }
 
     /**
      * Delete directory recursively.
      *
-     * @param dir Directory.
+     * @param f Directory.
      * @return {@code true} if successful.
      */
-    private boolean deleteDirectory(File dir) {
-        File[] entries = dir.listFiles();
+    private boolean deleteRecursive(File f) {
+        BasicFileAttributes attrs;
+
+        try {
+            attrs = Files.readAttributes(f.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS);
+        }
+        catch (IOException ignore) {
+            return false;
+        }
+
+        if (!attrs.isDirectory() || attrs.isSymbolicLink())
+            return f.delete();
+
+        File[] entries = f.listFiles();
 
         if (entries != null) {
             for (File entry : entries) {
-                if (entry.isDirectory())
-                    deleteDirectory(entry);
-                else if (entry.isFile()) {
-                    if (!entry.delete())
-                        return false;
-                }
-                else
-                    throw new UnsupportedOperationException("Symlink deletion is not yet supported: " + entry);
+                boolean res = deleteRecursive(entry);
+
+                if (!res)
+                    return false;
             }
         }
 
-        return dir.delete();
+        return f.delete();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b5757642/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java
new file mode 100644
index 0000000..9575bd0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractBaseSelfTest.java
@@ -0,0 +1,1067 @@
+/*
+ * 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.igfs;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Field;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.FileSystemConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
+import org.apache.ignite.igfs.IgfsInputStream;
+import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
+import org.apache.ignite.igfs.IgfsIpcEndpointType;
+import org.apache.ignite.igfs.IgfsMode;
+import org.apache.ignite.igfs.IgfsOutputStream;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+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.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
+import static org.apache.ignite.igfs.IgfsMode.PROXY;
+
+/**
+ * Test fo regular igfs operations.
+ */
+@SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "ConstantConditions"})
+public abstract class IgfsAbstractBaseSelfTest extends IgfsCommonAbstractTest {
+    /** IGFS block size. */
+    protected static final int IGFS_BLOCK_SIZE = 512 * 1024;
+
+    /** Default block size (32Mb). */
+    protected static final long BLOCK_SIZE = 32 * 1024 * 1024;
+
+    /** Default repeat count. */
+    protected static final int REPEAT_CNT = 5; // Diagnostic: up to 500; Regression: 5
+
+    /** Concurrent operations count. */
+    protected static final int OPS_CNT = 16;
+
+    /** Renames count. */
+    protected static final int RENAME_CNT = OPS_CNT;
+
+    /** Deletes count. */
+    protected static final int DELETE_CNT = OPS_CNT;
+
+    /** Updates count. */
+    protected static final int UPDATE_CNT = OPS_CNT;
+
+    /** Mkdirs count. */
+    protected static final int MKDIRS_CNT = OPS_CNT;
+
+    /** Create count. */
+    protected static final int CREATE_CNT = OPS_CNT;
+
+    /** Time to wait until the caches get empty after format. */
+    private static final long CACHE_EMPTY_TIMEOUT = 30_000L;
+
+    /** Seed to generate random numbers. */
+    protected static final long SEED = System.currentTimeMillis();
+
+    /** Amount of blocks to prefetch. */
+    protected static final int PREFETCH_BLOCKS = 1;
+
+    /** Amount of sequential block reads before prefetch is triggered. */
+    protected static final int SEQ_READS_BEFORE_PREFETCH = 2;
+
+    /** Primary file system REST endpoint configuration map. */
+    protected static final IgfsIpcEndpointConfiguration PRIMARY_REST_CFG;
+
+    /** Secondary file system REST endpoint configuration map. */
+    protected static final IgfsIpcEndpointConfiguration SECONDARY_REST_CFG;
+
+    /** Directory. */
+    protected static final IgfsPath DIR = new IgfsPath("/dir");
+
+    /** Sub-directory. */
+    protected static final IgfsPath SUBDIR = new IgfsPath(DIR, "subdir");
+
+    /** Another sub-directory in the same directory. */
+    protected static final IgfsPath SUBDIR2 = new IgfsPath(DIR, "subdir2");
+
+    /** Sub-directory of the sub-directory. */
+    protected static final IgfsPath SUBSUBDIR = new IgfsPath(SUBDIR, "subsubdir");
+
+    /** File. */
+    protected static final IgfsPath FILE = new IgfsPath(SUBDIR, "file");
+
+    /** Another file in the same directory. */
+    protected static final IgfsPath FILE2 = new IgfsPath(SUBDIR, "file2");
+
+    /** Other directory. */
+    protected static final IgfsPath DIR_NEW = new IgfsPath("/dirNew");
+
+    /** Other subdirectory. */
+    protected static final IgfsPath SUBDIR_NEW = new IgfsPath(DIR_NEW, "subdirNew");
+
+    /** Other sub-directory of the sub-directory. */
+    protected static final IgfsPath SUBSUBDIR_NEW = new IgfsPath(SUBDIR_NEW, "subsubdirNew");
+
+    /** Other file. */
+    protected static final IgfsPath FILE_NEW = new IgfsPath(SUBDIR_NEW, "fileNew");
+
+    /** Default data chunk (128 bytes). */
+    protected static final byte[] chunk = createChunk(128);
+
+    /** Primary IGFS. */
+    protected static IgfsImpl igfs;
+
+    /** Secondary IGFS */
+    protected static IgfsSecondaryFileSystem igfsSecondaryFileSystem;
+
+    /** Secondary file system lower layer "backdoor" wrapped in UniversalFileSystemAdapter: */
+    protected static IgfsSecondaryFileSystemTestAdapter igfsSecondary;
+
+    /** IGFS mode. */
+    protected final IgfsMode mode;
+
+    /** Dual mode flag. */
+    protected final boolean dual;
+
+    /** Memory mode. */
+    protected final CacheMemoryMode memoryMode;
+
+    /** IP finder for primary topology. */
+    protected final TcpDiscoveryVmIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** IP finder for secondary topology. */
+    protected final TcpDiscoveryVmIpFinder secondaryIpFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Ignite nodes of cluster, excluding the secondary file system node, if any. */
+    protected Ignite[] nodes;
+
+    static {
+        PRIMARY_REST_CFG = new IgfsIpcEndpointConfiguration();
+
+        PRIMARY_REST_CFG.setType(IgfsIpcEndpointType.TCP);
+        PRIMARY_REST_CFG.setPort(10500);
+
+        SECONDARY_REST_CFG = new IgfsIpcEndpointConfiguration();
+
+        SECONDARY_REST_CFG.setType(IgfsIpcEndpointType.TCP);
+        SECONDARY_REST_CFG.setPort(11500);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param mode IGFS mode.
+     */
+    protected IgfsAbstractBaseSelfTest(IgfsMode mode) {
+        this(mode, ONHEAP_TIERED);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param mode IGFS mode.
+     * @param memoryMode Memory mode.
+     */
+    protected IgfsAbstractBaseSelfTest(IgfsMode mode, CacheMemoryMode memoryMode) {
+        assert mode != null && mode != PROXY;
+
+        this.mode = mode;
+        this.memoryMode = memoryMode;
+
+        dual = mode != PRIMARY;
+    }
+
+    /**
+     * @return Relaxed consistency flag.
+     */
+    protected boolean relaxedConsistency() {
+        return false;
+    }
+
+    /**
+     * @return Relaxed consistency flag.
+     */
+    protected boolean initializeDefaultPathModes() {
+        return false;
+    }
+
+    /**
+     * @return Client flag.
+     */
+    protected boolean client() {
+        return false;
+    }
+
+    /**
+     * @return Use optimized marshaller flag.
+     */
+    protected boolean useOptimizedMarshaller() {
+        return false;
+    }
+
+    /**
+     * @return Whether append is supported.
+     */
+    protected boolean appendSupported() {
+        return true;
+    }
+
+    /**
+     * @return Whether permissions are supported.
+     */
+    protected boolean permissionsSupported() {
+        return true;
+    }
+
+    /**
+     * @return Whether properties are supported.
+     */
+    protected boolean propertiesSupported() {
+        return true;
+    }
+
+    /**
+     * @return Whether times are supported.
+     */
+    protected boolean timesSupported() {
+        return true;
+    }
+
+    /**
+     * @return Amount of nodes to start.
+     */
+    protected int nodeCount() {
+        return 1;
+    }
+
+    /**
+     * Data chunk.
+     *
+     * @param len Length.
+     * @return Data chunk.
+     */
+    static byte[] createChunk(int len) {
+        byte[] chunk = new byte[len];
+
+        for (int i = 0; i < chunk.length; i++)
+            chunk[i] = (byte)i;
+
+        return chunk;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        igfsSecondaryFileSystem = createSecondaryFileSystemStack();
+
+        nodes = new Ignite[nodeCount()];
+
+        for (int i = 0; i < nodes.length; i++) {
+            String nodeName = i == 0 ? "ignite" : "ignite" + i;
+
+            nodes[i] = startGridWithIgfs(nodeName, "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG,
+                primaryIpFinder);
+        }
+
+        igfs = (IgfsImpl) nodes[0].fileSystem("igfs");
+
+        if (client()) {
+            // Start client.
+            Ignition.setClientMode(true);
+
+            try {
+                Ignite ignite = startGridWithIgfs("ignite-client", "igfs", mode, igfsSecondaryFileSystem,
+                    PRIMARY_REST_CFG, primaryIpFinder);
+
+                igfs = (IgfsImpl) ignite.fileSystem("igfs");
+            }
+            finally {
+                Ignition.setClientMode(false);
+            }
+        }
+    }
+
+    /**
+     * Creates secondary file system stack.
+     *
+     * @return The secondary file system.
+     * @throws Exception On error.
+     */
+    protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
+        Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null,
+            SECONDARY_REST_CFG, secondaryIpFinder);
+
+        IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary");
+
+        igfsSecondary = new DefaultIgfsSecondaryFileSystemTestAdapter(secondaryIgfsImpl);
+
+        return secondaryIgfsImpl.asSecondary();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        clear(igfs, igfsSecondary);
+
+        assert igfs.listFiles(new IgfsPath("/")).isEmpty();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        G.stopAll(true);
+    }
+
+    /**
+     * Start grid with IGFS.
+     *
+     * @param gridName Grid name.
+     * @param igfsName IGFS name
+     * @param mode IGFS mode.
+     * @param secondaryFs Secondary file system (optional).
+     * @param restCfg Rest configuration string (optional).
+     * @param ipFinder IP finder.
+     * @return Started grid instance.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
+        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg,
+        TcpDiscoveryIpFinder ipFinder) throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("dataCache");
+        igfsCfg.setMetaCacheName("metaCache");
+        igfsCfg.setName(igfsName);
+        igfsCfg.setBlockSize(IGFS_BLOCK_SIZE);
+        igfsCfg.setDefaultMode(mode);
+        igfsCfg.setIpcEndpointConfiguration(restCfg);
+        igfsCfg.setSecondaryFileSystem(secondaryFs);
+        igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS);
+        igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH);
+        igfsCfg.setRelaxedConsistency(relaxedConsistency());
+
+        igfsCfg.setInitializeDefaultPathModes(initializeDefaultPathModes());
+
+        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setName("dataCache");
+        dataCacheCfg.setNearConfiguration(null);
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setNearConfiguration(null);
+        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2));
+        dataCacheCfg.setBackups(0);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+        dataCacheCfg.setMemoryMode(memoryMode);
+        dataCacheCfg.setOffHeapMaxMemory(0);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("metaCache");
+        metaCacheCfg.setNearConfiguration(null);
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        if (useOptimizedMarshaller())
+            cfg.setMarshaller(new OptimizedMarshaller());
+
+        cfg.setGridName(gridName);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(ipFinder);
+
+        prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setConnectorConfiguration(null);
+
+        return G.start(cfg);
+    }
+
+    /**
+     * Prepare cache configuration.
+     *
+     * @param dataCacheCfg Data cache configuration.
+     * @param metaCacheCfg Meta cache configuration.
+     */
+    protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, CacheConfiguration metaCacheCfg) {
+        // Noop
+    }
+
+    /**
+     * Execute provided task in a separate thread.
+     *
+     * @param task Task to execute.
+     * @return Result.
+     */
+    protected static <T> IgniteInternalFuture<T> execute(final Callable<T> task) {
+        final GridFutureAdapter<T> fut = new GridFutureAdapter<>();
+
+        new Thread(new Runnable() {
+            @Override public void run() {
+                try {
+                    fut.onDone(task.call());
+                }
+                catch (Throwable e) {
+                    fut.onDone(e);
+                }
+            }
+        }).start();
+
+        return fut;
+    }
+
+
+    /**
+     * Create the given directories and files in the given IGFS.
+     *
+     * @param igfs IGFS.
+     * @param dirs Directories.
+     * @param files Files.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("EmptyTryBlock")
+    public static void create(IgfsImpl igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception {
+        if (dirs != null) {
+            for (IgfsPath dir : dirs)
+                igfs.mkdirs(dir);
+        }
+
+        if (files != null) {
+            for (IgfsPath file : files) {
+                try (OutputStream ignored = igfs.create(file, true)) {
+                    // No-op.
+                }
+
+                igfs.await(file);
+            }
+        }
+    }
+
+    /**
+     * Creates specified files/directories
+     *
+     * @param uni The file system to operate on.
+     * @param dirs The directories to create.
+     * @param files The files to create.
+     * @throws Exception On error.
+     */
+    @SuppressWarnings("EmptyTryBlock")
+    public void create(IgfsSecondaryFileSystemTestAdapter uni, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files)
+        throws Exception {
+        if (dirs != null) {
+            for (IgfsPath dir : dirs)
+                uni.mkdirs(dir.toString());
+        }
+
+        if (files != null) {
+            for (IgfsPath file : files)
+                try (OutputStream ignore = uni.openOutputStream(file.toString(), false)) {
+                    // No-op
+                }
+        }
+    }
+
+    /**
+     * Create the file in the given IGFS and write provided data chunks to it.
+     *
+     * @param igfs IGFS.
+     * @param file File.
+     * @param overwrite Overwrite flag.
+     * @param chunks Data chunks.
+     * @throws IOException In case of IO exception.
+     */
+    protected static void createFile(IgfsEx igfs, IgfsPath file, boolean overwrite, @Nullable byte[]... chunks)
+        throws IOException {
+        OutputStream os = null;
+
+        try {
+            os = igfs.create(file, overwrite);
+
+            writeFileChunks(os, chunks);
+        }
+        finally {
+            U.closeQuiet(os);
+
+            awaitFileClose(igfs, file);
+        }
+    }
+
+    /**
+     * Create the file in the given IGFS and write provided data chunks to it.
+     *
+     * @param file File.
+     * @param chunks Data chunks.
+     * @throws IOException In case of IO exception.
+     */
+    protected static void createFile(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath file, @Nullable byte[]... chunks)
+        throws IOException {
+        OutputStream os = null;
+
+        try {
+            os = uni.openOutputStream(file.toString(), false);
+
+            writeFileChunks(os, chunks);
+        }
+        finally {
+            U.closeQuiet(os);
+
+            IgfsEx igfsEx = uni.igfs();
+
+            if (igfsEx != null)
+                awaitFileClose(igfsEx, file);
+        }
+    }
+
+    /**
+     * Create the file in the given IGFS and write provided data chunks to it.
+     *
+     * @param igfs IGFS.
+     * @param file File.
+     * @param overwrite Overwrite flag.
+     * @param blockSize Block size.
+     * @param chunks Data chunks.
+     * @throws Exception If failed.
+     */
+    protected static void createFile(IgfsImpl igfs, IgfsPath file, boolean overwrite, long blockSize,
+        @Nullable byte[]... chunks) throws Exception {
+        IgfsOutputStream os = null;
+
+        try {
+            os = igfs.create(file, 256, overwrite, null, 0, blockSize, null);
+
+            writeFileChunks(os, chunks);
+        }
+        finally {
+            U.closeQuiet(os);
+
+            awaitFileClose(igfs, file);
+        }
+    }
+
+    /**
+     * Append to the file in the given IGFS provided data chunks.
+     *
+     * @param igfs IGFS.
+     * @param file File.
+     * @param chunks Data chunks.
+     * @throws Exception If failed.
+     */
+    protected static void appendFile(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
+        throws Exception {
+        IgfsOutputStream os = null;
+
+        try {
+            os = igfs.append(file, false);
+
+            writeFileChunks(os, chunks);
+        }
+        finally {
+            U.closeQuiet(os);
+
+            awaitFileClose(igfs, file);
+        }
+    }
+
+    /**
+     * Write provided data chunks to the file output stream.
+     *
+     * @param os Output stream.
+     * @param chunks Data chunks.
+     * @throws IOException If failed.
+     */
+    protected static void writeFileChunks(OutputStream os, @Nullable byte[]... chunks) throws IOException {
+        if (chunks != null && chunks.length > 0) {
+            for (byte[] chunk : chunks)
+                os.write(chunk);
+        }
+    }
+
+    /**
+     * Await for previously opened output stream to close. This is achieved by requesting dummy update on the file.
+     *
+     * @param igfs IGFS.
+     * @param file File.
+     */
+    public static void awaitFileClose(IgfsSecondaryFileSystem igfs, IgfsPath file) {
+        try {
+            igfs.update(file, Collections.singletonMap("prop", "val"));
+        }
+        catch (IgniteException ignore) {
+            // No-op.
+        }
+    }
+
+    /**
+     * Await for previously opened output stream to close.
+     *
+     * @param igfs IGFS.
+     * @param file File.
+     */
+    public static void awaitFileClose(@Nullable IgfsEx igfs, IgfsPath file) {
+        igfs.await(file);
+    }
+
+    /**
+     * Ensure that the given paths exist in the given IGFSs.
+     *
+     * @param igfs First IGFS.
+     * @param igfsSecondary Second IGFS.
+     * @param paths Paths.
+     * @throws Exception If failed.
+     */
+    protected void checkExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths)
+        throws Exception {
+        checkExist(igfs, paths);
+
+        if (dual)
+            checkExist(igfsSecondary, paths);
+    }
+
+    /**
+     * Ensure that the given paths exist in the given IGFS.
+     *
+     * @param igfs IGFS.
+     * @param paths Paths.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected static void checkExist(IgfsImpl igfs, IgfsPath... paths) throws IgniteCheckedException {
+        for (IgfsPath path : paths)
+            assert igfs.exists(path) : "Path doesn't exist [igfs=" + igfs.name() + ", path=" + path + ']';
+    }
+
+    /**
+     * Ensure that the given paths exist in the given IGFS.
+     *
+     * @param uni filesystem.
+     * @param paths Paths.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void checkExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws IgniteCheckedException {
+        IgfsEx ex = uni.igfs();
+
+        for (IgfsPath path : paths) {
+            if (ex != null)
+                assert ex.context().meta().fileId(path) != null : "Path doesn't exist [igfs=" + ex.name() +
+                    ", path=" + path + ']';
+
+            try {
+                assert uni.exists(path.toString()) : "Path doesn't exist [igfs=" + uni.name() + ", path=" + path + ']';
+            }
+            catch (IOException ioe) {
+                throw new IgniteCheckedException(ioe);
+            }
+        }
+    }
+
+    /**
+     * Ensure that the given paths don't exist in the given IGFSs.
+     *
+     * @param igfs First IGFS.
+     * @param igfsSecondary Second IGFS.
+     * @param paths Paths.
+     * @throws Exception If failed.
+     */
+    protected void checkNotExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths)
+        throws Exception {
+        checkNotExist(igfs, paths);
+
+        if (dual)
+            checkNotExist(igfsSecondary, paths);
+    }
+
+    /**
+     * Ensure that the given paths don't exist in the given IGFS.
+     *
+     * @param igfs IGFS.
+     * @param paths Paths.
+     * @throws Exception If failed.
+     */
+    protected void checkNotExist(IgfsImpl igfs, IgfsPath... paths) throws Exception {
+        for (IgfsPath path : paths)
+            assert !igfs.exists(path) : "Path exists [igfs=" + igfs.name() + ", path=" + path + ']';
+    }
+
+    /**
+     * Ensure that the given paths don't exist in the given IGFS.
+     *
+     * @param uni secondary FS.
+     * @param paths Paths.
+     * @throws Exception If failed.
+     */
+    protected void checkNotExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws Exception {
+        IgfsEx ex = uni.igfs();
+
+        for (IgfsPath path : paths) {
+            if (ex != null)
+                assert !ex.exists(path) : "Path exists [igfs=" + ex.name() + ", path=" + path + ']';
+
+            assert !uni.exists(path.toString()) : "Path exists [igfs=" + uni.name() + ", path=" + path + ']';
+        }
+    }
+
+    /**
+     * Ensure that the given file exists in the given IGFSs and that it has exactly the same content as provided in the
+     * "data" parameter.
+     *
+     * @param igfs First IGFS.
+     * @param igfsSecondary Second IGFS.
+     * @param file File.
+     * @param chunks Expected data.
+     * @throws Exception If failed.
+     */
+    protected void checkFile(@Nullable IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath file,
+        @Nullable byte[]... chunks) throws Exception {
+        if (igfs != null) {
+            checkExist(igfs, file);
+            checkFileContent(igfs, file, chunks);
+        }
+
+        if (dual) {
+            checkExist(igfsSecondary, file);
+            checkFileContent(igfsSecondary, file.toString(), chunks);
+        }
+    }
+
+    /**
+     * Ensure that the given file has exactly the same content as provided in the "data" parameter.
+     *
+     * @param igfs IGFS.
+     * @param file File.
+     * @param chunks Expected data.
+     * @throws IOException In case of IO exception.
+     * @throws IgniteCheckedException In case of Grid exception.
+     */
+    protected static void checkFileContent(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
+        throws IOException, IgniteCheckedException {
+        if (chunks != null && chunks.length > 0) {
+            IgfsInputStream is = null;
+
+            try {
+                is = igfs.open(file);
+
+                int chunkIdx = 0;
+                int pos = 0;
+
+                for (byte[] chunk : chunks) {
+                    byte[] buf = new byte[chunk.length];
+
+                    is.readFully(pos, buf);
+
+                    assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + igfs.name() + ", chunkIdx=" + chunkIdx +
+                        ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']';
+
+                    chunkIdx++;
+                    pos += chunk.length;
+                }
+
+                is.close();
+            }
+            finally {
+                U.closeQuiet(is);
+            }
+        }
+    }
+
+    /**
+     * Ensure that the given file has exactly the same content as provided in the "data" parameter.
+     *
+     * @param uni FS.
+     * @param path File.
+     * @param chunks Expected data.
+     * @throws IOException In case of IO exception.
+     * @throws IgniteCheckedException In case of Grid exception.
+     */
+    protected void checkFileContent(IgfsSecondaryFileSystemTestAdapter uni, String path, @Nullable byte[]... chunks)
+        throws IOException, IgniteCheckedException {
+        if (chunks != null && chunks.length > 0) {
+            InputStream is = null;
+
+            try {
+                is = uni.openInputStream(path);
+
+                int chunkIdx = 0;
+
+                int read;
+                for (byte[] chunk: chunks) {
+                    byte[] buf = new byte[chunk.length];
+
+                    read = 0;
+
+                    while (true) {
+                        int r = is.read(buf, read, buf.length - read);
+
+                        read += r;
+
+                        if (read == buf.length || r <= 0)
+                            break;
+                    }
+
+                    assert read == chunk.length : "Chunk #" + chunkIdx + " was not read fully:" +
+                            " read=" + read + ", expected=" + chunk.length;
+                    assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + uni.name() + ", chunkIdx=" + chunkIdx +
+                        ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']';
+
+                    chunkIdx++;
+                }
+
+                is.close();
+            }
+            finally {
+                U.closeQuiet(is);
+            }
+        }
+    }
+
+    /**
+     * Create map with properties.
+     *
+     * @param username User name.
+     * @param grpName Group name.
+     * @param perm Permission.
+     * @return Map with properties.
+     */
+    protected Map<String, String> properties(@Nullable String username, @Nullable String grpName,
+        @Nullable String perm) {
+        Map<String, String> props = new HashMap<>();
+
+        if (username != null)
+            props.put(IgfsUtils.PROP_USER_NAME, username);
+
+        if (grpName != null)
+            props.put(IgfsUtils.PROP_GROUP_NAME, grpName);
+
+        if (perm != null)
+            props.put(IgfsUtils.PROP_PERMISSION, perm);
+
+        return props;
+    }
+
+    /**
+     * Convenient method to group paths.
+     *
+     * @param paths Paths to group.
+     * @return Paths as array.
+     */
+    protected static IgfsPath[] paths(IgfsPath... paths) {
+        return paths;
+    }
+
+    /**
+     * Safely clear IGFSs.
+     *
+     * @param igfs First IGFS.
+     * @param igfsSecondary Second IGFS.
+     * @throws Exception If failed.
+     */
+    protected void clear(IgniteFileSystem igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary) throws Exception {
+        clear(igfs);
+
+        if (dual)
+            clear(igfsSecondary);
+    }
+
+    /**
+     * Gets the data cache instance for this IGFS instance.
+     *
+     * @param igfs The IGFS unstance.
+     * @return The data cache.
+     */
+    protected static GridCacheAdapter<IgfsBlockKey, byte[]> getDataCache(IgniteFileSystem igfs) {
+        String dataCacheName = igfs.configuration().getDataCacheName();
+
+        IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
+
+        return ((IgniteKernal)igniteEx).internalCache(dataCacheName);
+    }
+
+    /**
+     * Gets meta cache.
+     *
+     * @param igfs The IGFS instance.
+     * @return The data cache.
+     */
+    protected static GridCacheAdapter<IgniteUuid, IgfsEntryInfo> getMetaCache(IgniteFileSystem igfs) {
+        String dataCacheName = igfs.configuration().getMetaCacheName();
+
+        IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
+
+        return ((IgniteKernal)igniteEx).internalCache(dataCacheName);
+    }
+
+    /**
+     * Clear particular IGFS.
+     *
+     * @param igfs IGFS.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public static void clear(IgniteFileSystem igfs) throws Exception {
+        Field workerMapFld = IgfsImpl.class.getDeclaredField("workerMap");
+
+        workerMapFld.setAccessible(true);
+
+        // Wait for all workers to finish.
+        Map<IgfsPath, IgfsFileWorkerBatch> workerMap = (Map<IgfsPath, IgfsFileWorkerBatch>)workerMapFld.get(igfs);
+
+        for (Map.Entry<IgfsPath, IgfsFileWorkerBatch> entry : workerMap.entrySet()) {
+            entry.getValue().cancel();
+
+            try {
+                entry.getValue().await();
+            }
+            catch (IgniteCheckedException e) {
+                if (!(e instanceof IgfsFileWorkerBatchCancelledException))
+                    throw e;
+            }
+        }
+
+        // Clear igfs.
+        igfs.format();
+
+        int prevDifferentSize = Integer.MAX_VALUE; // Previous different size.
+        int constCnt = 0, totalCnt = 0;
+        final int constThreshold = 20;
+        final long sleepPeriod = 500L;
+        final long totalThreshold = CACHE_EMPTY_TIMEOUT / sleepPeriod;
+
+        while (true) {
+            int metaSize = 0;
+
+            for (IgniteUuid metaId : getMetaCache(igfs).keySet()) {
+                if (!IgfsUtils.isRootOrTrashId(metaId))
+                    metaSize++;
+            }
+
+            int dataSize = getDataCache(igfs).size();
+
+            int size = metaSize + dataSize;
+
+            if (size <= 2)
+                return; // Caches are cleared, we're done. (2 because ROOT & TRASH always exist).
+
+            X.println("Sum size: " + size);
+
+            if (size > prevDifferentSize) {
+                X.println("Summary cache size has grown unexpectedly: size=" + size + ", prevSize=" + prevDifferentSize);
+
+                break;
+            }
+
+            if (totalCnt > totalThreshold) {
+                X.println("Timeout exceeded.");
+
+                break;
+            }
+
+            if (size == prevDifferentSize) {
+                constCnt++;
+
+                if (constCnt == constThreshold) {
+                    X.println("Summary cache size stays unchanged for too long: size=" + size);
+
+                    break;
+                }
+            } else {
+                constCnt = 0;
+
+                prevDifferentSize = size; // renew;
+            }
+
+            Thread.sleep(sleepPeriod);
+
+            totalCnt++;
+        }
+
+        dumpCache("MetaCache" , getMetaCache(igfs));
+
+        dumpCache("DataCache" , getDataCache(igfs));
+
+        fail("Caches are not empty.");
+    }
+
+    /**
+     * Dumps given cache for diagnostic purposes.
+     *
+     * @param cacheName Name.
+     * @param cache The cache.
+     */
+    private static void dumpCache(String cacheName, GridCacheAdapter<?,?> cache) {
+        X.println("=============================== " + cacheName + " cache dump: ");
+
+        Iterable<? extends GridCacheEntryEx> entries = cache.entries();
+
+        for (GridCacheEntryEx e: entries)
+            X.println("Lost " + cacheName + " entry = " + e);
+    }
+
+    /**
+     * Clear particular {@link IgfsSecondaryFileSystemTestAdapter}.
+     *
+     * @param uni IGFS.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public static void clear(IgfsSecondaryFileSystemTestAdapter uni) throws Exception {
+        IgfsEx igfsEx = uni.igfs();
+
+        if (igfsEx != null)
+            clear(igfsEx);
+
+        // Clear the filesystem.
+        uni.format();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        clear(igfs, igfsSecondary);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b5757642/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 86c2449..c9b08d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -17,55 +17,32 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteFileSystem;
-import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.FileSystemConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException;
 import org.apache.ignite.igfs.IgfsException;
 import org.apache.ignite.igfs.IgfsFile;
-import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
 import org.apache.ignite.igfs.IgfsInputStream;
-import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
-import org.apache.ignite.igfs.IgfsIpcEndpointType;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsOutputStream;
 import org.apache.ignite.igfs.IgfsParentNotDirectoryException;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.IgfsPathNotFoundException;
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
-import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-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.jetbrains.annotations.Nullable;
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.Field;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -84,142 +61,18 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-import static org.apache.ignite.igfs.IgfsMode.PRIMARY;
-import static org.apache.ignite.igfs.IgfsMode.PROXY;
-
 /**
  * Test fo regular igfs operations.
  */
 @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "ConstantConditions"})
-public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
-    /** IGFS block size. */
-    protected static final int IGFS_BLOCK_SIZE = 512 * 1024;
-
-    /** Default block size (32Mb). */
-    protected static final long BLOCK_SIZE = 32 * 1024 * 1024;
-
-    /** Default repeat count. */
-    protected static final int REPEAT_CNT = 5; // Diagnostic: up to 500; Regression: 5
-
-    /** Concurrent operations count. */
-    protected static final int OPS_CNT = 16;
-
-    /** Renames count. */
-    protected static final int RENAME_CNT = OPS_CNT;
-
-    /** Deletes count. */
-    protected static final int DELETE_CNT = OPS_CNT;
-
-    /** Updates count. */
-    protected static final int UPDATE_CNT = OPS_CNT;
-
-    /** Mkdirs count. */
-    protected static final int MKDIRS_CNT = OPS_CNT;
-
-    /** Create count. */
-    protected static final int CREATE_CNT = OPS_CNT;
-
-    /** Time to wait until the caches get empty after format. */
-    private static final long CACHE_EMPTY_TIMEOUT = 30_000L;
-
-    /** Seed to generate random numbers. */
-    protected static final long SEED = System.currentTimeMillis();
-
-    /** Amount of blocks to prefetch. */
-    protected static final int PREFETCH_BLOCKS = 1;
-
-    /** Amount of sequential block reads before prefetch is triggered. */
-    protected static final int SEQ_READS_BEFORE_PREFETCH = 2;
-
-    /** Primary file system REST endpoint configuration map. */
-    protected static final IgfsIpcEndpointConfiguration PRIMARY_REST_CFG;
-
-    /** Secondary file system REST endpoint configuration map. */
-    protected static final IgfsIpcEndpointConfiguration SECONDARY_REST_CFG;
-
-    /** Directory. */
-    protected static final IgfsPath DIR = new IgfsPath("/dir");
-
-    /** Sub-directory. */
-    protected static final IgfsPath SUBDIR = new IgfsPath(DIR, "subdir");
-
-    /** Another sub-directory in the same directory. */
-    protected static final IgfsPath SUBDIR2 = new IgfsPath(DIR, "subdir2");
-
-    /** Sub-directory of the sub-directory. */
-    protected static final IgfsPath SUBSUBDIR = new IgfsPath(SUBDIR, "subsubdir");
-
-    /** File. */
-    protected static final IgfsPath FILE = new IgfsPath(SUBDIR, "file");
-
-    /** Another file in the same directory. */
-    protected static final IgfsPath FILE2 = new IgfsPath(SUBDIR, "file2");
-
-    /** Other directory. */
-    protected static final IgfsPath DIR_NEW = new IgfsPath("/dirNew");
-
-    /** Other subdirectory. */
-    protected static final IgfsPath SUBDIR_NEW = new IgfsPath(DIR_NEW, "subdirNew");
-
-    /** Other sub-directory of the sub-directory. */
-    protected static final IgfsPath SUBSUBDIR_NEW = new IgfsPath(SUBDIR_NEW, "subsubdirNew");
-
-    /** Other file. */
-    protected static final IgfsPath FILE_NEW = new IgfsPath(SUBDIR_NEW, "fileNew");
-
-    /** Default data chunk (128 bytes). */
-    protected static final byte[] chunk = createChunk(128);
-
-    /** Primary IGFS. */
-    protected static IgfsImpl igfs;
-
-    /** Secondary IGFS */
-    protected static IgfsSecondaryFileSystem igfsSecondaryFileSystem;
-
-    /** Secondary file system lower layer "backdoor" wrapped in UniversalFileSystemAdapter: */
-    protected static IgfsSecondaryFileSystemTestAdapter igfsSecondary;
-
-    /** IGFS mode. */
-    protected final IgfsMode mode;
-
-    /** Dual mode flag. */
-    protected final boolean dual;
-
-    /** Memory mode. */
-    protected final CacheMemoryMode memoryMode;
-
-    /** IP finder for primary topology. */
-    protected final TcpDiscoveryVmIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** IP finder for secondary topology. */
-    protected final TcpDiscoveryVmIpFinder secondaryIpFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Ignite nodes of cluster, excluding the secondary file system node, if any. */
-    protected Ignite[] nodes;
-
-    static {
-        PRIMARY_REST_CFG = new IgfsIpcEndpointConfiguration();
-
-        PRIMARY_REST_CFG.setType(IgfsIpcEndpointType.TCP);
-        PRIMARY_REST_CFG.setPort(10500);
-
-        SECONDARY_REST_CFG = new IgfsIpcEndpointConfiguration();
-
-        SECONDARY_REST_CFG.setType(IgfsIpcEndpointType.TCP);
-        SECONDARY_REST_CFG.setPort(11500);
-    }
-
+public abstract class IgfsAbstractSelfTest extends IgfsAbstractBaseSelfTest {
     /**
      * Constructor.
      *
      * @param mode IGFS mode.
      */
     protected IgfsAbstractSelfTest(IgfsMode mode) {
-        this(mode, ONHEAP_TIERED);
+        super(mode);
     }
 
     /**
@@ -229,258 +82,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param memoryMode Memory mode.
      */
     protected IgfsAbstractSelfTest(IgfsMode mode, CacheMemoryMode memoryMode) {
-        assert mode != null && mode != PROXY;
-
-        this.mode = mode;
-        this.memoryMode = memoryMode;
-
-        dual = mode != PRIMARY;
-    }
-
-    /**
-     * @return Relaxed consistency flag.
-     */
-    protected boolean relaxedConsistency() {
-        return false;
-    }
-
-    /**
-     * @return Relaxed consistency flag.
-     */
-    protected boolean initializeDefaultPathModes() {
-        return false;
-    }
-
-    /**
-     * @return Client flag.
-     */
-    protected boolean client() {
-        return false;
-    }
-
-    /**
-     * @return Use optimized marshaller flag.
-     */
-    protected boolean useOptimizedMarshaller() {
-        return false;
-    }
-
-    /**
-     * @return Whether append is supported.
-     */
-    protected boolean appendSupported() {
-        return true;
-    }
-
-    /**
-     * @return Whether permissions are supported.
-     */
-    protected boolean permissionsSupported() {
-        return true;
-    }
-
-    /**
-     * @return Whether properties are supported.
-     */
-    protected boolean propertiesSupported() {
-        return true;
-    }
-
-    /**
-     * @return Whether times are supported.
-     */
-    protected boolean timesSupported() {
-        return true;
-    }
-
-    /**
-     * @return Amount of nodes to start.
-     */
-    protected int nodeCount() {
-        return 1;
-    }
-
-    /**
-     * Data chunk.
-     *
-     * @param len Length.
-     * @return Data chunk.
-     */
-    static byte[] createChunk(int len) {
-        byte[] chunk = new byte[len];
-
-        for (int i = 0; i < chunk.length; i++)
-            chunk[i] = (byte)i;
-
-        return chunk;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        igfsSecondaryFileSystem = createSecondaryFileSystemStack();
-
-        nodes = new Ignite[nodeCount()];
-
-        for (int i = 0; i < nodes.length; i++) {
-            String nodeName = i == 0 ? "ignite" : "ignite" + i;
-
-            nodes[i] = startGridWithIgfs(nodeName, "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG,
-                primaryIpFinder);
-        }
-
-        igfs = (IgfsImpl) nodes[0].fileSystem("igfs");
-
-        if (client()) {
-            // Start client.
-            Ignition.setClientMode(true);
-
-            try {
-                Ignite ignite = startGridWithIgfs("ignite-client", "igfs", mode, igfsSecondaryFileSystem,
-                    PRIMARY_REST_CFG, primaryIpFinder);
-
-                igfs = (IgfsImpl) ignite.fileSystem("igfs");
-            }
-            finally {
-                Ignition.setClientMode(false);
-            }
-        }
-    }
-
-    /**
-     * Creates secondary file system stack.
-     *
-     * @return The secondary file system.
-     * @throws Exception On error.
-     */
-    protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
-        Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null,
-            SECONDARY_REST_CFG, secondaryIpFinder);
-
-        IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary");
-
-        igfsSecondary = new DefaultIgfsSecondaryFileSystemTestAdapter(secondaryIgfsImpl);
-
-        return secondaryIgfsImpl.asSecondary();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        clear(igfs, igfsSecondary);
-
-        assert igfs.listFiles(new IgfsPath("/")).isEmpty();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        G.stopAll(true);
-    }
-
-    /**
-     * Start grid with IGFS.
-     *
-     * @param gridName Grid name.
-     * @param igfsName IGFS name
-     * @param mode IGFS mode.
-     * @param secondaryFs Secondary file system (optional).
-     * @param restCfg Rest configuration string (optional).
-     * @param ipFinder IP finder.
-     * @return Started grid instance.
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("unchecked")
-    protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
-        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg,
-        TcpDiscoveryIpFinder ipFinder) throws Exception {
-        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
-
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
-        igfsCfg.setName(igfsName);
-        igfsCfg.setBlockSize(IGFS_BLOCK_SIZE);
-        igfsCfg.setDefaultMode(mode);
-        igfsCfg.setIpcEndpointConfiguration(restCfg);
-        igfsCfg.setSecondaryFileSystem(secondaryFs);
-        igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS);
-        igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH);
-        igfsCfg.setRelaxedConsistency(relaxedConsistency());
-
-        igfsCfg.setInitializeDefaultPathModes(initializeDefaultPathModes());
-
-        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
-
-        dataCacheCfg.setName("dataCache");
-        dataCacheCfg.setNearConfiguration(null);
-        dataCacheCfg.setCacheMode(PARTITIONED);
-        dataCacheCfg.setNearConfiguration(null);
-        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2));
-        dataCacheCfg.setBackups(0);
-        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
-        dataCacheCfg.setMemoryMode(memoryMode);
-        dataCacheCfg.setOffHeapMaxMemory(0);
-
-        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
-
-        metaCacheCfg.setName("metaCache");
-        metaCacheCfg.setNearConfiguration(null);
-        metaCacheCfg.setCacheMode(REPLICATED);
-        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        IgniteConfiguration cfg = new IgniteConfiguration();
-
-        if (useOptimizedMarshaller())
-            cfg.setMarshaller(new OptimizedMarshaller());
-
-        cfg.setGridName(gridName);
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(ipFinder);
-
-        prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
-
-        cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
-        cfg.setFileSystemConfiguration(igfsCfg);
-
-        cfg.setLocalHost("127.0.0.1");
-        cfg.setConnectorConfiguration(null);
-
-        return G.start(cfg);
-    }
-
-    /**
-     * Prepare cache configuration.
-     *
-     * @param dataCacheCfg Data cache configuration.
-     * @param metaCacheCfg Meta cache configuration.
-     */
-    protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, CacheConfiguration metaCacheCfg) {
-        // Noop
-    }
-
-    /**
-     * Execute provided task in a separate thread.
-     *
-     * @param task Task to execute.
-     * @return Result.
-     */
-    protected static <T> IgniteInternalFuture<T> execute(final Callable<T> task) {
-        final GridFutureAdapter<T> fut = new GridFutureAdapter<>();
-
-        new Thread(new Runnable() {
-            @Override public void run() {
-                try {
-                    fut.onDone(task.call());
-                }
-                catch (Throwable e) {
-                    fut.onDone(e);
-                }
-            }
-        }).start();
-
-        return fut;
+        super(mode, memoryMode);
     }
 
     /**
@@ -1138,8 +740,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * Check root property update.
      *
-     * @throws Exception
+     * @throws Exception If failed.
      */
     private void checkRootPropertyUpdate(String prop, String setVal, String expGetVal) throws Exception {
         final IgfsPath rootPath = new IgfsPath("/");
@@ -2945,609 +2548,4 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         U.joinThreads(threads, null);
     }
-
-    /**
-     * Create the given directories and files in the given IGFS.
-     *
-     * @param igfs IGFS.
-     * @param dirs Directories.
-     * @param files Files.
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("EmptyTryBlock")
-    public static void create(IgfsImpl igfs, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files) throws Exception {
-        if (dirs != null) {
-            for (IgfsPath dir : dirs)
-                igfs.mkdirs(dir);
-        }
-
-        if (files != null) {
-            for (IgfsPath file : files) {
-                try (OutputStream os = igfs.create(file, true)) {
-                    // No-op.
-                }
-
-                igfs.await(file);
-            }
-        }
-    }
-
-    /**
-     * Creates specified files/directories
-     *
-     * @param uni The file system to operate on.
-     * @param dirs The directories to create.
-     * @param files The files to create.
-     * @throws Exception On error.
-     */
-    @SuppressWarnings("EmptyTryBlock")
-    public void create(IgfsSecondaryFileSystemTestAdapter uni, @Nullable IgfsPath[] dirs, @Nullable IgfsPath[] files)
-        throws Exception {
-        if (dirs != null) {
-            for (IgfsPath dir : dirs)
-                uni.mkdirs(dir.toString());
-        }
-
-        if (files != null) {
-            for (IgfsPath file : files)
-                try (OutputStream ignore = uni.openOutputStream(file.toString(), false)) {
-                    // No-op
-                }
-        }
-    }
-
-    /**
-     * Create the file in the given IGFS and write provided data chunks to it.
-     *
-     * @param igfs IGFS.
-     * @param file File.
-     * @param overwrite Overwrite flag.
-     * @param chunks Data chunks.
-     * @throws IOException In case of IO exception.
-     */
-    protected static void createFile(IgfsEx igfs, IgfsPath file, boolean overwrite, @Nullable byte[]... chunks)
-        throws IOException {
-        OutputStream os = null;
-
-        try {
-            os = igfs.create(file, overwrite);
-
-            writeFileChunks(os, chunks);
-        }
-        finally {
-            U.closeQuiet(os);
-
-            awaitFileClose(igfs, file);
-        }
-    }
-
-    /**
-     * Create the file in the given IGFS and write provided data chunks to it.
-     *
-     * @param file File.
-     * @param chunks Data chunks.
-     * @throws IOException In case of IO exception.
-     */
-    protected static void createFile(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath file, @Nullable byte[]... chunks)
-        throws IOException {
-        OutputStream os = null;
-
-        try {
-            os = uni.openOutputStream(file.toString(), false);
-
-            writeFileChunks(os, chunks);
-        }
-        finally {
-            U.closeQuiet(os);
-
-            IgfsEx igfsEx = uni.igfs();
-
-            if (igfsEx != null)
-                awaitFileClose(igfsEx, file);
-        }
-    }
-
-    /**
-     * Create the file in the given IGFS and write provided data chunks to it.
-     *
-     * @param igfs IGFS.
-     * @param file File.
-     * @param overwrite Overwrite flag.
-     * @param blockSize Block size.
-     * @param chunks Data chunks.
-     * @throws Exception If failed.
-     */
-    protected static void createFile(IgfsImpl igfs, IgfsPath file, boolean overwrite, long blockSize,
-        @Nullable byte[]... chunks) throws Exception {
-        IgfsOutputStream os = null;
-
-        try {
-            os = igfs.create(file, 256, overwrite, null, 0, blockSize, null);
-
-            writeFileChunks(os, chunks);
-        }
-        finally {
-            U.closeQuiet(os);
-
-            awaitFileClose(igfs, file);
-        }
-    }
-
-    /**
-     * Append to the file in the given IGFS provided data chunks.
-     *
-     * @param igfs IGFS.
-     * @param file File.
-     * @param chunks Data chunks.
-     * @throws Exception If failed.
-     */
-    protected static void appendFile(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
-        throws Exception {
-        IgfsOutputStream os = null;
-
-        try {
-            os = igfs.append(file, false);
-
-            writeFileChunks(os, chunks);
-        }
-        finally {
-            U.closeQuiet(os);
-
-            awaitFileClose(igfs, file);
-        }
-    }
-
-    /**
-     * Write provided data chunks to the file output stream.
-     *
-     * @param os Output stream.
-     * @param chunks Data chunks.
-     * @throws IOException If failed.
-     */
-    protected static void writeFileChunks(OutputStream os, @Nullable byte[]... chunks) throws IOException {
-        if (chunks != null && chunks.length > 0) {
-            for (byte[] chunk : chunks)
-                os.write(chunk);
-        }
-    }
-
-    /**
-     * Await for previously opened output stream to close. This is achieved by requesting dummy update on the file.
-     *
-     * @param igfs IGFS.
-     * @param file File.
-     */
-    public static void awaitFileClose(IgfsSecondaryFileSystem igfs, IgfsPath file) {
-        try {
-            igfs.update(file, Collections.singletonMap("prop", "val"));
-        }
-        catch (IgniteException ignore) {
-            // No-op.
-        }
-    }
-
-    /**
-     * Await for previously opened output stream to close.
-     *
-     * @param igfs IGFS.
-     * @param file File.
-     */
-    public static void awaitFileClose(@Nullable IgfsEx igfs, IgfsPath file) {
-        igfs.await(file);
-    }
-
-    /**
-     * Ensure that the given paths exist in the given IGFSs.
-     *
-     * @param igfs First IGFS.
-     * @param igfsSecondary Second IGFS.
-     * @param paths Paths.
-     * @throws Exception If failed.
-     */
-    protected void checkExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths)
-        throws Exception {
-        checkExist(igfs, paths);
-
-        if (dual)
-            checkExist(igfsSecondary, paths);
-    }
-
-    /**
-     * Ensure that the given paths exist in the given IGFS.
-     *
-     * @param igfs IGFS.
-     * @param paths Paths.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected static void checkExist(IgfsImpl igfs, IgfsPath... paths) throws IgniteCheckedException {
-        for (IgfsPath path : paths)
-            assert igfs.exists(path) : "Path doesn't exist [igfs=" + igfs.name() + ", path=" + path + ']';
-    }
-
-    /**
-     * Ensure that the given paths exist in the given IGFS.
-     *
-     * @param uni filesystem.
-     * @param paths Paths.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected void checkExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws IgniteCheckedException {
-        IgfsEx ex = uni.igfs();
-
-        for (IgfsPath path : paths) {
-            if (ex != null)
-                assert ex.context().meta().fileId(path) != null : "Path doesn't exist [igfs=" + ex.name() +
-                    ", path=" + path + ']';
-
-            try {
-                assert uni.exists(path.toString()) : "Path doesn't exist [igfs=" + uni.name() + ", path=" + path + ']';
-            }
-            catch (IOException ioe) {
-                throw new IgniteCheckedException(ioe);
-            }
-        }
-    }
-
-    /**
-     * Ensure that the given paths don't exist in the given IGFSs.
-     *
-     * @param igfs First IGFS.
-     * @param igfsSecondary Second IGFS.
-     * @param paths Paths.
-     * @throws Exception If failed.
-     */
-    protected void checkNotExist(IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath... paths)
-        throws Exception {
-        checkNotExist(igfs, paths);
-
-        if (dual)
-            checkNotExist(igfsSecondary, paths);
-    }
-
-    /**
-     * Ensure that the given paths don't exist in the given IGFS.
-     *
-     * @param igfs IGFS.
-     * @param paths Paths.
-     * @throws Exception If failed.
-     */
-    protected void checkNotExist(IgfsImpl igfs, IgfsPath... paths) throws Exception {
-        for (IgfsPath path : paths)
-            assert !igfs.exists(path) : "Path exists [igfs=" + igfs.name() + ", path=" + path + ']';
-    }
-
-    /**
-     * Ensure that the given paths don't exist in the given IGFS.
-     *
-     * @param uni secondary FS.
-     * @param paths Paths.
-     * @throws Exception If failed.
-     */
-    protected void checkNotExist(IgfsSecondaryFileSystemTestAdapter uni, IgfsPath... paths) throws Exception {
-        IgfsEx ex = uni.igfs();
-
-        for (IgfsPath path : paths) {
-            if (ex != null)
-                assert !ex.exists(path) : "Path exists [igfs=" + ex.name() + ", path=" + path + ']';
-
-            assert !uni.exists(path.toString()) : "Path exists [igfs=" + uni.name() + ", path=" + path + ']';
-        }
-    }
-
-    /**
-     * Ensure that the given file exists in the given IGFSs and that it has exactly the same content as provided in the
-     * "data" parameter.
-     *
-     * @param igfs First IGFS.
-     * @param igfsSecondary Second IGFS.
-     * @param file File.
-     * @param chunks Expected data.
-     * @throws Exception If failed.
-     */
-    protected void checkFile(@Nullable IgfsImpl igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary, IgfsPath file,
-        @Nullable byte[]... chunks) throws Exception {
-        if (igfs != null) {
-            checkExist(igfs, file);
-            checkFileContent(igfs, file, chunks);
-        }
-
-        if (dual) {
-            checkExist(igfsSecondary, file);
-            checkFileContent(igfsSecondary, file.toString(), chunks);
-        }
-    }
-
-    /**
-     * Ensure that the given file has exactly the same content as provided in the "data" parameter.
-     *
-     * @param igfs IGFS.
-     * @param file File.
-     * @param chunks Expected data.
-     * @throws IOException In case of IO exception.
-     * @throws IgniteCheckedException In case of Grid exception.
-     */
-    protected static void checkFileContent(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
-        throws IOException, IgniteCheckedException {
-        if (chunks != null && chunks.length > 0) {
-            IgfsInputStream is = null;
-
-            try {
-                is = igfs.open(file);
-
-                int chunkIdx = 0;
-                int pos = 0;
-
-                for (byte[] chunk : chunks) {
-                    byte[] buf = new byte[chunk.length];
-
-                    is.readFully(pos, buf);
-
-                    assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + igfs.name() + ", chunkIdx=" + chunkIdx +
-                        ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']';
-
-                    chunkIdx++;
-                    pos += chunk.length;
-                }
-
-                is.close();
-            }
-            finally {
-                U.closeQuiet(is);
-            }
-        }
-    }
-
-    /**
-     * Ensure that the given file has exactly the same content as provided in the "data" parameter.
-     *
-     * @param uni FS.
-     * @param path File.
-     * @param chunks Expected data.
-     * @throws IOException In case of IO exception.
-     * @throws IgniteCheckedException In case of Grid exception.
-     */
-    protected void checkFileContent(IgfsSecondaryFileSystemTestAdapter uni, String path, @Nullable byte[]... chunks)
-        throws IOException, IgniteCheckedException {
-        if (chunks != null && chunks.length > 0) {
-            InputStream is = null;
-
-            try {
-                is = uni.openInputStream(path);
-
-                int chunkIdx = 0;
-
-                int read;
-                for (byte[] chunk: chunks) {
-                    byte[] buf = new byte[chunk.length];
-
-                    read = 0;
-
-                    while (true) {
-                        int r = is.read(buf, read, buf.length - read);
-
-                        read += r;
-
-                        if (read == buf.length || r <= 0)
-                            break;
-                    }
-
-                    assert read == chunk.length : "Chunk #" + chunkIdx + " was not read fully:" +
-                            " read=" + read + ", expected=" + chunk.length;
-                    assert Arrays.equals(chunk, buf) : "Bad chunk [igfs=" + uni.name() + ", chunkIdx=" + chunkIdx +
-                        ", expected=" + Arrays.toString(chunk) + ", actual=" + Arrays.toString(buf) + ']';
-
-                    chunkIdx++;
-                }
-
-                is.close();
-            }
-            finally {
-                U.closeQuiet(is);
-            }
-        }
-    }
-
-    /**
-     * Create map with properties.
-     *
-     * @param username User name.
-     * @param grpName Group name.
-     * @param perm Permission.
-     * @return Map with properties.
-     */
-    protected Map<String, String> properties(@Nullable String username, @Nullable String grpName,
-        @Nullable String perm) {
-        Map<String, String> props = new HashMap<>();
-
-        if (username != null)
-            props.put(IgfsUtils.PROP_USER_NAME, username);
-
-        if (grpName != null)
-            props.put(IgfsUtils.PROP_GROUP_NAME, grpName);
-
-        if (perm != null)
-            props.put(IgfsUtils.PROP_PERMISSION, perm);
-
-        return props;
-    }
-
-    /**
-     * Convenient method to group paths.
-     *
-     * @param paths Paths to group.
-     * @return Paths as array.
-     */
-    protected static IgfsPath[] paths(IgfsPath... paths) {
-        return paths;
-    }
-
-    /**
-     * Safely clear IGFSs.
-     *
-     * @param igfs First IGFS.
-     * @param igfsSecondary Second IGFS.
-     * @throws Exception If failed.
-     */
-    protected void clear(IgniteFileSystem igfs, IgfsSecondaryFileSystemTestAdapter igfsSecondary) throws Exception {
-        clear(igfs);
-
-        if (dual)
-            clear(igfsSecondary);
-    }
-
-    /**
-     * Gets the data cache instance for this IGFS instance.
-     *
-     * @param igfs The IGFS unstance.
-     * @return The data cache.
-     */
-    protected static GridCacheAdapter<IgfsBlockKey, byte[]> getDataCache(IgniteFileSystem igfs) {
-        String dataCacheName = igfs.configuration().getDataCacheName();
-
-        IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
-
-        return ((IgniteKernal)igniteEx).internalCache(dataCacheName);
-    }
-
-    /**
-     * Gets meta cache.
-     *
-     * @param igfs The IGFS instance.
-     * @return The data cache.
-     */
-    protected static GridCacheAdapter<IgniteUuid, IgfsEntryInfo> getMetaCache(IgniteFileSystem igfs) {
-        String dataCacheName = igfs.configuration().getMetaCacheName();
-
-        IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid();
-
-        return ((IgniteKernal)igniteEx).internalCache(dataCacheName);
-    }
-
-    /**
-     * Clear particular IGFS.
-     *
-     * @param igfs IGFS.
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("unchecked")
-    public static void clear(IgniteFileSystem igfs) throws Exception {
-        Field workerMapFld = IgfsImpl.class.getDeclaredField("workerMap");
-
-        workerMapFld.setAccessible(true);
-
-        // Wait for all workers to finish.
-        Map<IgfsPath, IgfsFileWorkerBatch> workerMap = (Map<IgfsPath, IgfsFileWorkerBatch>)workerMapFld.get(igfs);
-
-        for (Map.Entry<IgfsPath, IgfsFileWorkerBatch> entry : workerMap.entrySet()) {
-            entry.getValue().cancel();
-
-            try {
-                entry.getValue().await();
-            }
-            catch (IgniteCheckedException e) {
-                if (!(e instanceof IgfsFileWorkerBatchCancelledException))
-                    throw e;
-            }
-        }
-
-        // Clear igfs.
-        igfs.format();
-
-        int prevDifferentSize = Integer.MAX_VALUE; // Previous different size.
-        int constCnt = 0, totalCnt = 0;
-        final int constThreshold = 20;
-        final long sleepPeriod = 500L;
-        final long totalThreshold = CACHE_EMPTY_TIMEOUT / sleepPeriod;
-
-        while (true) {
-            int metaSize = 0;
-
-            for (IgniteUuid metaId : getMetaCache(igfs).keySet()) {
-                if (!IgfsUtils.isRootOrTrashId(metaId))
-                    metaSize++;
-            }
-
-            int dataSize = getDataCache(igfs).size();
-
-            int size = metaSize + dataSize;
-
-            if (size <= 2)
-                return; // Caches are cleared, we're done. (2 because ROOT & TRASH always exist).
-
-            X.println("Sum size: " + size);
-
-            if (size > prevDifferentSize) {
-                X.println("Summary cache size has grown unexpectedly: size=" + size + ", prevSize=" + prevDifferentSize);
-
-                break;
-            }
-
-            if (totalCnt > totalThreshold) {
-                X.println("Timeout exceeded.");
-
-                break;
-            }
-
-            if (size == prevDifferentSize) {
-                constCnt++;
-
-                if (constCnt == constThreshold) {
-                    X.println("Summary cache size stays unchanged for too long: size=" + size);
-
-                    break;
-                }
-            } else {
-                constCnt = 0;
-
-                prevDifferentSize = size; // renew;
-            }
-
-            Thread.sleep(sleepPeriod);
-
-            totalCnt++;
-        }
-
-        dumpCache("MetaCache" , getMetaCache(igfs));
-
-        dumpCache("DataCache" , getDataCache(igfs));
-
-        fail("Caches are not empty.");
-    }
-
-    /**
-     * Dumps given cache for diagnostic purposes.
-     *
-     * @param cacheName Name.
-     * @param cache The cache.
-     */
-    private static void dumpCache(String cacheName, GridCacheAdapter<?,?> cache) {
-        X.println("=============================== " + cacheName + " cache dump: ");
-
-        Iterable<? extends GridCacheEntryEx> entries = cache.entries();
-
-        for (GridCacheEntryEx e: entries)
-            X.println("Lost " + cacheName + " entry = " + e);
-    }
-
-    /**
-     * Clear particular {@link IgfsSecondaryFileSystemTestAdapter}.
-     *
-     * @param uni IGFS.
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("unchecked")
-    public static void clear(IgfsSecondaryFileSystemTestAdapter uni) throws Exception {
-        IgfsEx igfsEx = uni.igfs();
-
-        if (igfsEx != null)
-            clear(igfsEx);
-
-        // Clear the filesystem.
-        uni.format();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        clear(igfs, igfsSecondary);
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b5757642/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java
index c2f5633..1d1ce8d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java
@@ -17,12 +17,21 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsMode;
+import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.secondary.local.LocalIgfsSecondaryFileSystem;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
 
 import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.Collection;
 
 /**
  * Abstract test for Hadoop 1.0 file system stack.
@@ -32,6 +41,24 @@ public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends I
     private static final String FS_WORK_DIR = U.getIgniteHome() + File.separatorChar + "work"
         + File.separatorChar + "fs";
 
+    /** */
+    private static final String FS_EXT_DIR = U.getIgniteHome() + File.separatorChar + "work"
+        + File.separatorChar + "ext";
+
+    /** */
+    private final File dirLinkDest = new File(FS_EXT_DIR + File.separatorChar + "extdir");
+
+    /** */
+    private final File fileLinkDest =
+        new File(FS_EXT_DIR + File.separatorChar + "extdir" + File.separatorChar + "filedest");
+
+    /** */
+    private final File dirLinkSrc = new File(FS_WORK_DIR + File.separatorChar + "dir");
+
+    /** */
+    private final File fileLinkSrc = new File(FS_WORK_DIR + File.separatorChar + "file");
+
+
     /** Constructor.
      * @param mode IGFS mode.
      */
@@ -39,6 +66,19 @@ public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends I
         super(mode);
     }
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        final File extDir = new File(FS_EXT_DIR);
+
+        if (!extDir.exists())
+            assert extDir.mkdirs();
+        else
+            cleanDirectory(extDir);
+    }
+
+
     /**
      * Creates secondary filesystems.
      * @return IgfsSecondaryFileSystem
@@ -73,4 +113,107 @@ public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends I
     @Override protected boolean timesSupported() {
         return false;
     }
+
+    /**
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public void testListPathForSymlink() throws Exception {
+        if (U.isWindows())
+            return;
+
+        createSymlinks();
+
+        assertTrue(igfs.info(DIR).isDirectory());
+
+        Collection<IgfsPath> pathes = igfs.listPaths(DIR);
+        Collection<IgfsFile> files = igfs.listFiles(DIR);
+
+        assertEquals(1, pathes.size());
+        assertEquals(1, files.size());
+
+        assertEquals("filedest", F.first(pathes).name());
+        assertEquals("filedest", F.first(files).path().name());
+    }
+
+    /**
+     *
+     * @throws Exception If failed.
+     */
+    public void testDeleteSymlinkDir() throws Exception {
+        if (U.isWindows())
+            return;
+
+        createSymlinks();
+
+        // Only symlink must be deleted. Destination content must be exist.
+        igfs.delete(DIR, true);
+
+        assertTrue(fileLinkDest.exists());
+    }
+
+    /**
+     *
+     * @throws Exception If failed.
+     */
+    public void testSymlinkToFile() throws Exception {
+        if (U.isWindows())
+            return;
+
+        createSymlinks();
+
+        checkFileContent(igfs, new IgfsPath("/file"), chunk);
+    }
+
+    /**
+     *
+     * @throws Exception If failed.
+     */
+    private void createSymlinks() throws Exception {
+        assert dirLinkDest.mkdir();
+
+        createFile(fileLinkDest, true, chunk);
+
+        Files.createSymbolicLink(dirLinkSrc.toPath(), dirLinkDest.toPath());
+        Files.createSymbolicLink(fileLinkSrc.toPath(), fileLinkDest.toPath());
+    }
+
+    /**
+     * @param dir Directory to clean.
+     */
+    private static void cleanDirectory(File dir){
+        File[] entries = dir.listFiles();
+
+        if (entries != null) {
+            for (File entry : entries) {
+                if (entry.isDirectory()) {
+                    cleanDirectory(entry);
+
+                    assert entry.delete();
+                }
+                else
+                    assert entry.delete();
+            }
+        }
+    }
+
+    /**
+     * @param f File object.
+     * @param overwrite Overwrite flag.
+     * @param chunks File content.
+     * @throws IOException If failed.
+     */
+    private static void createFile(File f, boolean overwrite, @Nullable byte[]... chunks) throws IOException {
+        OutputStream os = null;
+
+        try {
+            os = new FileOutputStream(f, overwrite);
+
+            writeFileChunks(os, chunks);
+        }
+        finally {
+            U.closeQuiet(os);
+        }
+    }
 }
\ No newline at end of file


[19/24] ignite git commit: ignite-2968 Deadlock detection for optimistic tx and near caches

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java
index abbefd0..4bf5a41 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.transactions;
 
-import java.util.concurrent.ConcurrentMap;
+import java.util.Collection;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -26,6 +26,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -60,22 +61,6 @@ public class TxPessimisticDeadlockDetectionCrossCacheTest extends GridCommonAbst
             cfg.setDiscoverySpi(discoSpi);
         }
 
-        CacheConfiguration ccfg0 = defaultCacheConfiguration();
-
-        ccfg0.setName("cache0");
-        ccfg0.setCacheMode(CacheMode.PARTITIONED);
-        ccfg0.setBackups(1);
-        ccfg0.setNearConfiguration(null);
-
-        CacheConfiguration ccfg1 = defaultCacheConfiguration();
-
-        ccfg1.setName("cache1");
-        ccfg1.setCacheMode(CacheMode.PARTITIONED);
-        ccfg1.setBackups(1);
-        ccfg1.setNearConfiguration(null);
-
-        cfg.setCacheConfiguration(ccfg0, ccfg1);
-
         return cfg;
     }
 
@@ -96,70 +81,132 @@ public class TxPessimisticDeadlockDetectionCrossCacheTest extends GridCommonAbst
     /**
      * @throws Exception If failed.
      */
-    public void testDeadlock() throws Exception {
-        final CyclicBarrier barrier = new CyclicBarrier(2);
+    public void testDeadlockNoNear() throws Exception {
+        doTestDeadlock(false, false);
+    }
 
-        final AtomicInteger threadCnt = new AtomicInteger();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlockOneNear() throws Exception {
+        doTestDeadlock(false, true);
+    }
 
-        final AtomicBoolean deadlock = new AtomicBoolean();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlockAnotherNear() throws Exception {
+        doTestDeadlock(true, false);
+        doTestDeadlock(false, true);
+    }
 
-        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-            @Override public void run() {
-                int threadNum = threadCnt.getAndIncrement();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlockBothNear() throws Exception {
+        doTestDeadlock(true, true);
+    }
 
-                Ignite ignite = ignite(0);
+    /**
+     * @param near0 Near flag for cache0.
+     * @param near1 Near flag for cache1.
+     */
+    private void doTestDeadlock(boolean near0, boolean near1) throws Exception {
+        IgniteCache<Integer, Integer> cache0 = null;
+        IgniteCache<Integer, Integer> cache1 = null;
 
-                IgniteCache<Integer, Integer> cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1));
+        try {
+            cache0 = getCache(ignite(0), "cache0", near0);
+            cache1 = getCache(ignite(0), "cache1", near1);
 
-                IgniteCache<Integer, Integer> cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0));
+            awaitPartitionMapExchange();
 
-                try (Transaction tx =
-                         ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 500, 0)
-                ) {
-                    int key1 = primaryKey(cache1);
+            final CyclicBarrier barrier = new CyclicBarrier(2);
 
-                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']');
+            final AtomicInteger threadCnt = new AtomicInteger();
 
-                    cache1.put(key1, 0);
+            final AtomicBoolean deadlock = new AtomicBoolean();
 
-                    barrier.await();
+            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+                @Override public void run() {
+                    int threadNum = threadCnt.getAndIncrement();
 
-                    int key2 = primaryKey(cache2);
+                    Ignite ignite = ignite(0);
 
-                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']');
+                    IgniteCache<Integer, Integer> cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1));
 
-                    cache2.put(key2, 1);
+                    IgniteCache<Integer, Integer> cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0));
 
-                    tx.commit();
-                }
-                catch (Throwable e) {
-                    // At least one stack trace should contain TransactionDeadlockException.
-                    if (hasCause(e, TransactionTimeoutException.class) &&
-                        hasCause(e, TransactionDeadlockException.class)
-                        ) {
-                        if (deadlock.compareAndSet(false, true))
-                            U.error(log, "At least one stack trace should contain " +
-                                TransactionDeadlockException.class.getSimpleName(), e);
+                    try (Transaction tx =
+                             ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 500, 0)
+                    ) {
+                        int key1 = primaryKey(cache1);
+
+                        log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                            ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']');
+
+                        cache1.put(key1, 0);
+
+                        barrier.await();
+
+                        int key2 = primaryKey(cache2);
+
+                        log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                            ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']');
+
+                        cache2.put(key2, 1);
+
+                        tx.commit();
+                    }
+                    catch (Throwable e) {
+                        // At least one stack trace should contain TransactionDeadlockException.
+                        if (hasCause(e, TransactionTimeoutException.class) &&
+                            hasCause(e, TransactionDeadlockException.class)
+                            ) {
+                            if (deadlock.compareAndSet(false, true))
+                                U.error(log, "At least one stack trace should contain " +
+                                    TransactionDeadlockException.class.getSimpleName(), e);
+                        }
                     }
                 }
-            }
-        }, 2, "tx-thread");
+            }, 2, "tx-thread");
 
-        fut.get();
+            fut.get();
 
-        assertTrue(deadlock.get());
+            assertTrue(deadlock.get());
 
-        for (int i = 0; i < NODES_CNT ; i++) {
-            Ignite ignite = ignite(i);
+            for (int i = 0; i < NODES_CNT ; i++) {
+                Ignite ignite = ignite(i);
 
-            IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
+                IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
 
-            ConcurrentMap<Long, TxDeadlockDetection.TxDeadlockFuture> futs =
-                GridTestUtils.getFieldValue(txMgr, IgniteTxManager.class, "deadlockDetectFuts");
+                Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
 
-            assertTrue(futs.isEmpty());
+                assertTrue(futs.isEmpty());
+            }
+        }
+        finally {
+            if (cache0 != null)
+                cache0.destroy();
+
+            if (cache1 != null)
+                cache1.destroy();
         }
     }
+
+    /**
+     * @param ignite Ignite.
+     * @param name Name.
+     * @param near Near.
+     */
+    private IgniteCache<Integer, Integer> getCache(Ignite ignite, String name, boolean near) {
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setName(name);
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setNearConfiguration(near ? new NearCacheConfiguration() : null);
+
+        return ignite.getOrCreateCache(ccfg);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
index ee1a989..83eb908 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
@@ -25,7 +25,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -43,6 +42,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
@@ -135,7 +135,7 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void _testDeadlocksPartitionedNear() throws Exception {
+    public void testDeadlocksPartitionedNear() throws Exception {
         for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
             doTestDeadlocks(createCache(PARTITIONED, syncMode, true), NO_OP_TRANSFORMER);
             doTestDeadlocks(createCache(PARTITIONED, syncMode, true), WRAPPING_TRANSFORMER);
@@ -178,6 +178,7 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
      * @param cacheMode Cache mode.
      * @param syncMode Write sync mode.
      * @param near Near.
+     * @return Created cache.
      */
     @SuppressWarnings("unchecked")
     private IgniteCache createCache(CacheMode cacheMode, CacheWriteSynchronizationMode syncMode, boolean near) {
@@ -189,7 +190,19 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
         ccfg.setNearConfiguration(near ? new NearCacheConfiguration() : null);
         ccfg.setWriteSynchronizationMode(syncMode);
 
-        return ignite(0).getOrCreateCache(ccfg);
+        IgniteCache cache = ignite(0).createCache(ccfg);
+
+        if (near) {
+            for (int i = 0; i < NODES_CNT; i++) {
+                Ignite client = ignite(i + NODES_CNT);
+
+                assertTrue(client.configuration().isClientMode());
+
+                client.createNearCache(ccfg.getName(), new NearCacheConfiguration<>());
+            }
+        }
+
+        return cache;
     }
 
     /**
@@ -323,7 +336,14 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
             }
         }, loc ? 2 : txCnt, "tx-thread");
 
-        fut.get();
+        try {
+            fut.get();
+        }
+        catch (IgniteCheckedException e) {
+            U.error(null, "Unexpected exception", e);
+
+            fail();
+        }
 
         U.sleep(1000);
 
@@ -331,13 +351,17 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         assertNotNull(deadlockE);
 
+        boolean fail = false;
+
         // Check transactions, futures and entry locks state.
         for (int i = 0; i < NODES_CNT * 2; i++) {
             Ignite ignite = ignite(i);
 
             int cacheId = ((IgniteCacheProxy)ignite.cache(CACHE_NAME)).context().cacheId();
 
-            IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
+            GridCacheSharedContext<Object, Object> cctx = ((IgniteKernal)ignite).context().cache().context();
+
+            IgniteTxManager txMgr = cctx.tm();
 
             Collection<IgniteInternalTx> activeTxs = txMgr.activeTransactions();
 
@@ -345,13 +369,16 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
                 Collection<IgniteTxEntry> entries = tx.allEntries();
 
                 for (IgniteTxEntry entry : entries) {
-                    if (entry.cacheId() == cacheId)
-                        fail("Transaction still exists: " + tx);
+                    if (entry.cacheId() == cacheId) {
+                        fail = true;
+
+                        U.error(log, "Transaction still exists: " + "\n" + tx.xidVersion() +
+                            "\n" + tx.nearXidVersion() + "\n nodeId=" + cctx.localNodeId() + "\n tx=" + tx);
+                    }
                 }
             }
 
-            ConcurrentMap<Long, TxDeadlockDetection.TxDeadlockFuture> futs =
-                GridTestUtils.getFieldValue(txMgr, IgniteTxManager.class, "deadlockDetectFuts");
+            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
 
             assertTrue(futs.isEmpty());
 
@@ -371,6 +398,9 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
             }
         }
 
+        if (fail)
+            fail("Some transactions still exist");
+
         // Check deadlock report
         String msg = deadlockE.getMessage();
 
@@ -484,4 +514,4 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
             return id;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 8414461..14e5833 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePart
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredAtomicFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeP2PDisabledFullApiSelfTest;
@@ -74,6 +75,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeCounterSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapMultiNodeFullApiSelfTest;
@@ -167,6 +169,8 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicNearEnabledMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicNearEnabledPrimaryWriteOrderMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(CachePartitionedMultiNodeLongTxTimeoutFullApiTest.class);
+        suite.addTestSuite(CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.class);
 
         suite.addTestSuite(GridCachePartitionedNearDisabledMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledMultiNodeP2PDisabledFullApiSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java
index c057e55..5a1b1ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java
@@ -19,7 +19,10 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.transactions.DepthFirstSearchTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionNoHangsTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticDeadlockDetectionCrossCacheTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticDeadlockDetectionTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPessimisticDeadlockDetectionCrossCacheTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPessimisticDeadlockDetectionTest;
 
@@ -35,9 +38,12 @@ public class TxDeadlockDetectionTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("Ignite Deadlock Detection Test Suite");
 
         suite.addTestSuite(DepthFirstSearchTest.class);
+        suite.addTestSuite(TxOptimisticDeadlockDetectionTest.class);
+        suite.addTestSuite(TxOptimisticDeadlockDetectionCrossCacheTest.class);
         suite.addTestSuite(TxPessimisticDeadlockDetectionTest.class);
         suite.addTestSuite(TxPessimisticDeadlockDetectionCrossCacheTest.class);
         suite.addTestSuite(TxDeadlockDetectionTest.class);
+        suite.addTestSuite(TxDeadlockDetectionNoHangsTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala
index 22fb89d..1b55505 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala
@@ -19,6 +19,7 @@ package org.apache.ignite.visor.commands.cache
 
 import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode}
 import org.apache.ignite.visor.visor._
+
 import org.apache.ignite.internal.visor.cache.VisorCacheStopTask
 import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 
@@ -101,9 +102,7 @@ class VisorCacheStopCommand {
                 return
         }
 
-        val dflt = if (batchMode) "y" else "n"
-
-        ask(s"Are you sure you want to stop cache: ${escapeName(cacheName)}? (y/n) [$dflt]: ", dflt) match {
+        ask(s"Are you sure you want to stop cache: ${escapeName(cacheName)}? (y/n) [n]: ", "n") match {
             case "y" | "Y" =>
                 try {
                     executeRandom(grp, classOf[VisorCacheStopTask], cacheName)


[15/24] ignite git commit: IGNITE-3776: Removed code duplication in GridNearAtomicAbstractUpdateFuture.

Posted by sb...@apache.org.
IGNITE-3776: Removed code duplication in GridNearAtomicAbstractUpdateFuture.


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

Branch: refs/heads/master
Commit: 87a1928a4f90b4f8a221041cfff9d22e3dd801cc
Parents: 99e3e8a
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Aug 26 15:22:15 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 26 15:22:15 2016 +0300

----------------------------------------------------------------------
 .../GridNearAtomicAbstractUpdateFuture.java     | 69 +++++++++++++++++++
 .../GridNearAtomicSingleUpdateFuture.java       | 70 +-------------------
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 68 +------------------
 3 files changed, 73 insertions(+), 134 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/87a1928a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
index 3e69c02..85751bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -29,6 +30,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheReturn;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -246,4 +248,71 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
     protected boolean storeFuture() {
         return cctx.config().getAtomicWriteOrderMode() == CLOCK || syncMode != FULL_ASYNC;
     }
+
+    /**
+     * Maps future to single node.
+     *
+     * @param nodeId Node ID.
+     * @param req Request.
+     */
+    protected void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) {
+        if (cctx.localNodeId().equals(nodeId)) {
+            cache.updateAllAsyncInternal(nodeId, req,
+                new CI2<GridNearAtomicUpdateRequest, GridNearAtomicUpdateResponse>() {
+                    @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) {
+                        onResult(res.nodeId(), res, false);
+                    }
+                });
+        }
+        else {
+            try {
+                cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
+
+                if (msgLog.isDebugEnabled()) {
+                    msgLog.debug("Near update fut, sent request [futId=" + req.futureVersion() +
+                        ", writeVer=" + req.updateVersion() +
+                        ", node=" + req.nodeId() + ']');
+                }
+
+                if (syncMode == FULL_ASYNC)
+                    onDone(new GridCacheReturn(cctx, true, true, null, true));
+            }
+            catch (IgniteCheckedException e) {
+                if (msgLog.isDebugEnabled()) {
+                    msgLog.debug("Near update fut, failed to send request [futId=" + req.futureVersion() +
+                        ", writeVer=" + req.updateVersion() +
+                        ", node=" + req.nodeId() +
+                        ", err=" + e + ']');
+                }
+
+                onSendError(req, e);
+            }
+        }
+    }
+
+    /**
+     * Response callback.
+     *
+     * @param nodeId Node ID.
+     * @param res Update response.
+     * @param nodeErr {@code True} if response was created on node failure.
+     */
+    public abstract void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr);
+
+    /**
+     * @param req Request.
+     * @param e Error.
+     */
+    protected void onSendError(GridNearAtomicUpdateRequest req, IgniteCheckedException e) {
+        synchronized (mux) {
+            GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
+                req.nodeId(),
+                req.futureVersion(),
+                cctx.deploymentEnabled());
+
+            res.addFailedKeys(req.keys(), e);
+
+            onResult(req.nodeId(), res, true);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/87a1928a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
index aad4186..661a178 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
@@ -37,7 +37,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.CI1;
-import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -52,7 +51,6 @@ import java.util.Map;
 import java.util.UUID;
 
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.CLOCK;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.TRANSFORM;
 
 /**
@@ -191,15 +189,9 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
         return false;
     }
 
-    /**
-     * Response callback.
-     *
-     * @param nodeId Node ID.
-     * @param res Update response.
-     * @param nodeErr {@code True} if response was created on node failure.
-     */
+    /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
-    public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
+    @Override  public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
         GridNearAtomicUpdateRequest req;
 
         AffinityTopologyVersion remapTopVer = null;
@@ -441,64 +433,6 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
         map(topVer);
     }
 
-    /**
-     * Maps future to single node.
-     *
-     * @param nodeId Node ID.
-     * @param req Request.
-     */
-    private void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) {
-        if (cctx.localNodeId().equals(nodeId)) {
-            cache.updateAllAsyncInternal(nodeId, req,
-                new CI2<GridNearAtomicUpdateRequest, GridNearAtomicUpdateResponse>() {
-                    @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) {
-                        onResult(res.nodeId(), res, false);
-                    }
-                });
-        }
-        else {
-            try {
-                cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
-
-                if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Near update single fut, sent request [futId=" + req.futureVersion() +
-                        ", writeVer=" + req.updateVersion() +
-                        ", node=" + req.nodeId() + ']');
-                }
-
-                if (syncMode == FULL_ASYNC)
-                    onDone(new GridCacheReturn(cctx, true, true, null, true));
-            }
-            catch (IgniteCheckedException e) {
-                if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Near update single fut, failed to send request [futId=" + req.futureVersion() +
-                        ", writeVer=" + req.updateVersion() +
-                        ", node=" + req.nodeId() +
-                        ", err=" + e + ']');
-                }
-
-                onSendError(req, e);
-            }
-        }
-    }
-
-    /**
-     * @param req Request.
-     * @param e Error.
-     */
-    void onSendError(GridNearAtomicUpdateRequest req, IgniteCheckedException e) {
-        synchronized (mux) {
-            GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
-                req.nodeId(),
-                req.futureVersion(),
-                cctx.deploymentEnabled());
-
-            res.addFailedKeys(req.keys(), e);
-
-            onResult(req.nodeId(), res, true);
-        }
-    }
-
     /** {@inheritDoc} */
     protected void map(AffinityTopologyVersion topVer) {
         Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);

http://git-wip-us.apache.org/repos/asf/ignite/blob/87a1928a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 0d88ef8..2432f63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -253,15 +253,9 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
         return false;
     }
 
-    /**
-     * Response callback.
-     *
-     * @param nodeId Node ID.
-     * @param res Update response.
-     * @param nodeErr {@code True} if response was created on node failure.
-     */
+    /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "ThrowableResultOfMethodCallIgnored"})
-    public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
+    @Override public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res, boolean nodeErr) {
         GridNearAtomicUpdateRequest req;
 
         AffinityTopologyVersion remapTopVer = null;
@@ -552,47 +546,6 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     }
 
     /**
-     * Maps future to single node.
-     *
-     * @param nodeId Node ID.
-     * @param req Request.
-     */
-    private void mapSingle(UUID nodeId, GridNearAtomicUpdateRequest req) {
-        if (cctx.localNodeId().equals(nodeId)) {
-            cache.updateAllAsyncInternal(nodeId, req,
-                new CI2<GridNearAtomicUpdateRequest, GridNearAtomicUpdateResponse>() {
-                    @Override public void apply(GridNearAtomicUpdateRequest req, GridNearAtomicUpdateResponse res) {
-                        onResult(res.nodeId(), res, false);
-                    }
-                });
-        }
-        else {
-            try {
-                cctx.io().send(req.nodeId(), req, cctx.ioPolicy());
-
-                if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Near update fut, sent request [futId=" + req.futureVersion() +
-                        ", writeVer=" + req.updateVersion() +
-                        ", node=" + req.nodeId() + ']');
-                }
-
-                if (syncMode == FULL_ASYNC)
-                    onDone(new GridCacheReturn(cctx, true, true, null, true));
-            }
-            catch (IgniteCheckedException e) {
-                if (msgLog.isDebugEnabled()) {
-                    msgLog.debug("Near update fut, failed to send request [futId=" + req.futureVersion() +
-                        ", writeVer=" + req.updateVersion() +
-                        ", node=" + req.nodeId() +
-                        ", err=" + e + ']');
-                }
-
-                onSendError(req, e);
-            }
-        }
-    }
-
-    /**
      * Sends messages to remote nodes and updates local cache.
      *
      * @param mappings Mappings to send.
@@ -646,23 +599,6 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             onDone(new GridCacheReturn(cctx, true, true, null, true));
     }
 
-    /**
-     * @param req Request.
-     * @param e Error.
-     */
-    void onSendError(GridNearAtomicUpdateRequest req, IgniteCheckedException e) {
-        synchronized (mux) {
-            GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
-                req.nodeId(),
-                req.futureVersion(),
-                cctx.deploymentEnabled());
-
-            res.addFailedKeys(req.keys(), e);
-
-            onResult(req.nodeId(), res, true);
-        }
-    }
-
     /** {@inheritDoc} */
     protected void map(AffinityTopologyVersion topVer) {
         map(topVer, null);


[02/24] ignite git commit: Merge remote-tracking branch 'upstream/ignite-1.6.6' into ignite-1.6.6

Posted by sb...@apache.org.
Merge remote-tracking branch 'upstream/ignite-1.6.6' into ignite-1.6.6


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

Branch: refs/heads/master
Commit: 118db2fcffe3534aa1e5f4b97b8fbe23891752c4
Parents: 4e9e7b8 25b59d7
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 24 12:12:24 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 24 12:12:24 2016 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheLinqTest.cs                | 196 +++++++++++++-
 .../Apache.Ignite.Linq.csproj                   |   2 +-
 .../dotnet/Apache.Ignite.Linq/CompiledQuery.cs  |   1 +
 .../dotnet/Apache.Ignite.Linq/CompiledQuery2.cs | 257 +++++++++++++++++++
 .../Impl/CacheFieldsQueryExecutor.cs            | 116 ++++++++-
 .../Impl/CacheQueryExpressionVisitor.cs         |  15 ++
 .../Impl/CacheQueryModelVisitor.cs              |  15 +-
 .../Impl/CacheQueryableBase.cs                  |  22 ++
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |  10 +
 .../Apache.Ignite.Linq/Impl/ICacheQueryProxy.cs |  40 ---
 .../Impl/ICacheQueryableInternal.cs             |  14 +-
 .../commands/alert/VisorAlertCommand.scala      |  43 ++--
 .../commands/cache/VisorCacheCommand.scala      |  16 +-
 13 files changed, 661 insertions(+), 86 deletions(-)
----------------------------------------------------------------------



[13/24] ignite git commit: IGNITE-3761: ODBC: Added tests for SQL_SQL92_VALUE_EXPRESSIONS. This closes #989.

Posted by sb...@apache.org.
IGNITE-3761: ODBC: Added tests for SQL_SQL92_VALUE_EXPRESSIONS. This closes #989.


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

Branch: refs/heads/master
Commit: 6fd53ea5b50148e5a1156d83ea28acb8faf84035
Parents: ae0b5eb
Author: Igor Sapego <is...@gridgain.com>
Authored: Fri Aug 26 11:19:39 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 26 11:19:39 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/odbc-test/Makefile.am     |  1 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |  1 +
 .../project/vs/odbc-test.vcxproj.filters        |  3 +
 .../src/sql_function_test_suite_fixture.cpp     |  7 +-
 .../odbc-test/src/sql_system_functions_test.cpp | 21 -----
 .../cpp/odbc-test/src/sql_value_expressions.cpp | 94 ++++++++++++++++++++
 modules/platforms/cpp/odbc/src/utility.cpp      | 14 +--
 7 files changed, 112 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6fd53ea5/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index 1e00b6f..1f641be 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -67,6 +67,7 @@ ignite_odbc_tests_SOURCES = \
     src/sql_numeric_functions_test.cpp \
     src/sql_aggregate_functions_test.cpp \
     src/sql_system_functions_test.cpp \
+    src/sql_value_expressions.cpp \
     ../odbc/src/cursor.cpp \
     ../odbc/src/config/connection_info.cpp \
     ../odbc/src/app/application_data_buffer.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fd53ea5/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index 2a94a2a..4d3a9cc 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -173,6 +173,7 @@
     <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_string_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_system_functions_test.cpp" />
+    <ClCompile Include="..\..\src\sql_value_expressions.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_messages.cpp" />
     <ClCompile Include="..\..\src\test_utils.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fd53ea5/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index 9ca51af..c4d70b7 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -94,6 +94,9 @@
     <ClCompile Include="..\..\src\sql_system_functions_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\sql_value_expressions.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\test_type.h">

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fd53ea5/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
index f848dc5..4d8b919 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
@@ -139,7 +139,12 @@ namespace ignite
 
         CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
 
-        BOOST_CHECK_EQUAL(std::string(reinterpret_cast<char*>(res), static_cast<size_t>(resLen)), expected);
+        std::string actual;
+
+        if (resLen > 0)
+            actual.assign(reinterpret_cast<char*>(res), static_cast<size_t>(resLen));
+
+        BOOST_CHECK_EQUAL(actual, expected);
     }
 
     template<>

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fd53ea5/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
index d369b0f..752925d 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
@@ -15,39 +15,18 @@
  * limitations under the License.
  */
 
-#ifdef _WIN32
-#   include <windows.h>
-#endif
-
-#include <sql.h>
-#include <sqlext.h>
-
-#include <vector>
-#include <string>
-
 #ifndef _MSC_VER
 #   define BOOST_TEST_DYN_LINK
 #endif
 
 #include <boost/test/unit_test.hpp>
 
-#include "ignite/ignite.h"
-#include "ignite/ignition.h"
-#include "ignite/impl/binary/binary_utils.h"
-
-#include "test_type.h"
-#include "test_utils.h"
 #include "sql_function_test_suite_fixture.h"
 
 using namespace ignite;
-using namespace ignite::cache;
-using namespace ignite::cache::query;
-using namespace ignite::common;
 
 using namespace boost::unit_test;
 
-using ignite::impl::binary::BinaryUtils;
-
 BOOST_FIXTURE_TEST_SUITE(SqlSystemFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestSystemFunctionDatabase)

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fd53ea5/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp b/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp
new file mode 100644
index 0000000..e7d6598
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "sql_function_test_suite_fixture.h"
+
+using namespace ignite;
+
+using namespace boost::unit_test;
+
+BOOST_FIXTURE_TEST_SUITE(SqlValueExpressionTestSuite, ignite::SqlFunctionTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestCase)
+{
+    TestType in;
+
+    in.i32Field = 82;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int32_t>(
+        "SELECT "
+            "CASE i32Field WHEN 82 "
+                "THEN (i32Field / 2) "
+                "ELSE (i32Field / 3) "
+            "END "
+        "FROM TestType", in.i32Field / 2);
+
+    
+    CheckSingleResult<int32_t>(
+        "SELECT "
+            "CASE i32Field WHEN 22 "
+                "THEN (i32Field / 2) "
+                "ELSE (i32Field / 3) "
+            "END "
+        "FROM TestType", in.i32Field / 3);;
+}
+
+BOOST_AUTO_TEST_CASE(TestCast)
+{
+    TestType in;
+
+    in.i32Field = 12345;
+    in.strField = "54321";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int32_t>("SELECT CAST(strField AS INT) + i32Field FROM TestType", 
+        common::LexicalCast<int32_t>(in.strField) + in.i32Field);
+
+    CheckSingleResult<std::string>("SELECT CAST(i32Field AS VARCHAR) || strField FROM TestType",
+        common::LexicalCast<std::string>(in.i32Field) + in.strField);
+}
+
+BOOST_AUTO_TEST_CASE(TestCoalesce)
+{
+    CheckSingleResult<std::string>("SELECT COALESCE('One', 'Two', 'Three')", "One");
+    CheckSingleResult<std::string>("SELECT COALESCE(NULL, 'Two', 'Three')", "Two");
+    CheckSingleResult<std::string>("SELECT COALESCE(NULL, 'Two', NULL)", "Two");
+    CheckSingleResult<std::string>("SELECT COALESCE(NULL, NULL, 'Three')", "Three");
+}
+
+BOOST_AUTO_TEST_CASE(TestNullif)
+{
+    TestType in;
+
+    in.strField = "SomeValue";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT NULLIF(strField, 'blablabla') FROM TestType", in.strField);
+    CheckSingleResult<std::string>("SELECT NULLIF(strField, 'SomeValue') FROM TestType", "");
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/6fd53ea5/modules/platforms/cpp/odbc/src/utility.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp
index 133f059..19be799 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -56,19 +56,19 @@ namespace ignite
         void ReadString(ignite::impl::binary::BinaryReaderImpl& reader, std::string& str)
         {
             int32_t strLen = reader.ReadString(0, 0);
-            if (!strLen)
+            if (strLen > 0)
             {
-                str.clear();
-
-                char dummy;
+                str.resize(strLen);
 
-                reader.ReadString(&dummy, sizeof(dummy));
+                reader.ReadString(&str[0], static_cast<int32_t>(str.size()));
             }
             else
             {
-                str.resize(strLen);
+                str.clear();
 
-                reader.ReadString(&str[0], static_cast<int32_t>(str.size()));
+                char dummy;
+
+                reader.ReadString(&dummy, sizeof(dummy));
             }
         }
 


[06/24] ignite git commit: IGNITE-3749: ODBC: Added support for numeric scalar functions. This closes #981.

Posted by sb...@apache.org.
IGNITE-3749: ODBC: Added support for numeric scalar functions. This closes #981.


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

Branch: refs/heads/master
Commit: 5a3b3e2c6ecb5d6c96513b79f21828526b4a98a0
Parents: d6449ff
Author: isapego <is...@gridgain.com>
Authored: Thu Aug 25 12:35:07 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 25 12:35:07 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/odbc-test/Makefile.am     |   2 +
 .../platforms/cpp/odbc-test/include/Makefile.am |   3 +-
 .../include/sql_function_test_suite_fixture.h   | 138 +++++++++
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   3 +
 .../project/vs/odbc-test.vcxproj.filters        |   9 +
 .../src/sql_function_test_suite_fixture.cpp     | 188 +++++++++++
 .../src/sql_numeric_functions_test.cpp          | 309 +++++++++++++++++++
 .../odbc-test/src/sql_string_functions_test.cpp | 228 +-------------
 .../cpp/odbc/src/config/connection_info.cpp     |   7 +-
 9 files changed, 659 insertions(+), 228 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index cc84a7f..62bdec3 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -62,7 +62,9 @@ ignite_odbc_tests_SOURCES = \
     src/utility_test.cpp \
     src/queries_test.cpp \
     src/test_utils.cpp \
+    src/sql_function_test_suite_fixture.cpp \
     src/sql_string_functions_test.cpp \
+    src/sql_numeric_functions_test.cpp \
     ../odbc/src/cursor.cpp \
     ../odbc/src/config/connection_info.cpp \
     ../odbc/src/app/application_data_buffer.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc-test/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/Makefile.am b/modules/platforms/cpp/odbc-test/include/Makefile.am
index 7b17102..baca1b7 100644
--- a/modules/platforms/cpp/odbc-test/include/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/include/Makefile.am
@@ -20,5 +20,6 @@ ACLOCAL_AMFLAGS =-I m4
 noinst_HEADERS = \
     teamcity/teamcity_messages.h \
     test_type.h \
-    test_utils.h
+    test_utils.h \
+    sql_function_test_suite_fixture.h
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
new file mode 100644
index 0000000..c09c1ae
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
@@ -0,0 +1,138 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE
+#define _IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <sql.h>
+#include <sqlext.h>
+
+#include <string>
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "ignite/ignite.h"
+#include "ignite/ignition.h"
+
+#include "test_type.h"
+
+namespace ignite
+{
+    /**
+     * Test setup fixture.
+     */
+    struct SqlFunctionTestSuiteFixture
+    {
+        /**
+         * Constructor.
+         */
+        SqlFunctionTestSuiteFixture();
+
+        /**
+         * Destructor.
+         */
+        ~SqlFunctionTestSuiteFixture();
+
+        /**
+         * Run query returning single result and stores it to buffer.
+         *
+         * @param request SQL request.
+         * @param type Result type.
+         * @param column Result buffer.
+         * @param bufSize Result buffer size.
+         * @param resSize Size of received value.
+         */
+        void CheckSingleResult0(const char* request, SQLSMALLINT type,
+            void* column, SQLLEN bufSize, SQLLEN* resSize) const;
+
+        /**
+         * Run query returning single result and check it to be equal to expected.
+         *
+         * @param request SQL request.
+         * @param expected Expected result.
+         */
+        template<typename T>
+        void CheckSingleResult(const char* request, const T& expected)
+        {
+            BOOST_FAIL("Function is not defined for the type.");
+        }
+    
+        /**
+         * Run query returning single result and check it to be equal to expected.
+         *
+         * @param request SQL request.
+         * @param expected Expected result.
+         * @param type Result type.
+         */
+        template<typename T>
+        void CheckSingleResultNum0(const char* request, const T& expected, SQLSMALLINT type)
+        {
+            T res = 0;
+
+            CheckSingleResult0(request, type, &res, 0, 0);
+
+            BOOST_CHECK_EQUAL(res, expected);
+        }
+
+
+        /** Node started during the test. */
+        Ignite grid;
+
+        /** Test cache instance. */
+        cache::Cache<int64_t, TestType> testCache;
+
+        /** ODBC Environment. */
+        SQLHENV env;
+
+        /** ODBC Connect. */
+        SQLHDBC dbc;
+
+        /** ODBC Statement. */
+        SQLHSTMT stmt;
+    };
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request, const std::string& expected);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request, const int64_t& expected);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request, const int32_t& expected);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request, const int16_t& expected);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request, const int8_t& expected);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected);
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected);
+}
+
+#endif //_IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index 619b4b2..a83ed6b 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -168,6 +168,8 @@
     <ClCompile Include="..\..\src\queries_test.cpp" />
     <ClCompile Include="..\..\src\parser_test.cpp" />
     <ClCompile Include="..\..\src\row_test.cpp" />
+    <ClCompile Include="..\..\src\sql_function_test_suite_fixture.cpp" />
+    <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_string_functions_test.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_messages.cpp" />
@@ -175,6 +177,7 @@
     <ClCompile Include="..\..\src\utility_test.cpp" />
   </ItemGroup>
   <ItemGroup>
+    <ClInclude Include="..\..\include\sql_function_test_suite_fixture.h" />
     <ClInclude Include="..\..\include\teamcity\teamcity_messages.h" />
     <ClInclude Include="..\..\include\test_type.h" />
     <ClInclude Include="..\..\include\test_utils.h" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index 1871149..f993ed9 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -82,6 +82,12 @@
     <ClCompile Include="..\..\src\test_utils.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\sql_function_test_suite_fixture.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\test_type.h">
@@ -93,6 +99,9 @@
     <ClInclude Include="..\..\include\test_utils.h">
       <Filter>Code</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\sql_function_test_suite_fixture.h">
+      <Filter>Code</Filter>
+    </ClInclude>
   </ItemGroup>
   <ItemGroup>
     <None Include="..\..\config\queries-test.xml">

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
new file mode 100644
index 0000000..2398393
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
@@ -0,0 +1,188 @@
+/*
+ * 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.
+ */
+
+#include "sql_function_test_suite_fixture.h"
+
+#include "test_utils.h"
+
+namespace ignite
+{
+    SqlFunctionTestSuiteFixture::SqlFunctionTestSuiteFixture():
+        testCache(0),
+        env(NULL),
+        dbc(NULL),
+        stmt(NULL)
+    {
+        IgniteConfiguration cfg;
+
+        cfg.jvmOpts.push_back("-Xdebug");
+        cfg.jvmOpts.push_back("-Xnoagent");
+        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
+        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
+        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
+
+#ifdef IGNITE_TESTS_32
+        cfg.jvmInitMem = 256;
+        cfg.jvmMaxMem = 768;
+#else
+        cfg.jvmInitMem = 1024;
+        cfg.jvmMaxMem = 4096;
+#endif
+
+        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
+
+        BOOST_REQUIRE(cfgPath != 0) ;
+
+        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
+
+        IgniteError err;
+
+        grid = Ignition::Start(cfg, &err);
+
+        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+        BOOST_FAIL(err.GetText()) ;
+
+        testCache = grid.GetCache<int64_t, TestType>("cache");
+
+        // Allocate an environment handle
+        SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
+
+        BOOST_REQUIRE(env != NULL) ;
+
+        // We want ODBC 3 support
+        SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
+
+        // Allocate a connection handle
+        SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
+
+        BOOST_REQUIRE(dbc != NULL) ;
+
+        // Connect string
+        SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache";
+
+        SQLCHAR outstr[ODBC_BUFFER_SIZE];
+        SQLSMALLINT outstrlen;
+
+        // Connecting to ODBC server.
+        SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast<SQLSMALLINT>(sizeof(connectStr)),
+                                         outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
+
+        if (!SQL_SUCCEEDED(ret))
+        {
+            Ignition::Stop(grid.GetName(), true);
+
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)) ;
+        }
+
+        // Allocate a statement handle
+        SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
+
+        BOOST_REQUIRE(stmt != NULL) ;
+    }
+
+    SqlFunctionTestSuiteFixture::~SqlFunctionTestSuiteFixture()
+    {
+        // Releasing statement handle.
+        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+
+        // Disconneting from the server.
+        SQLDisconnect(dbc);
+
+        // Releasing allocated handles.
+        SQLFreeHandle(SQL_HANDLE_DBC, dbc);
+        SQLFreeHandle(SQL_HANDLE_ENV, env);
+
+        ignite::Ignition::Stop(grid.GetName(), true);
+    }
+
+    void SqlFunctionTestSuiteFixture::CheckSingleResult0(const char* request,
+        SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize) const
+    {
+        SQLRETURN ret;
+
+        ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize);
+
+        if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
+
+        ret = SQLExecDirect(stmt, reinterpret_cast<SQLCHAR*>(const_cast<char*>(request)), SQL_NTS);
+        if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
+
+        ret = SQLFetch(stmt);
+        if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
+
+        ret = SQLFetch(stmt);
+        BOOST_CHECK(ret == SQL_NO_DATA) ;
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request, const std::string& expected)
+    {
+        SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
+        SQLLEN resLen = 0;
+
+        CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
+
+        BOOST_CHECK_EQUAL(std::string(reinterpret_cast<char*>(res), static_cast<size_t>(resLen)), expected);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request, const int64_t& expected)
+    {
+        CheckSingleResultNum0<int64_t>(request, expected, SQL_C_SBIGINT);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request, const int32_t& expected)
+    {
+        CheckSingleResultNum0<int32_t>(request, expected, SQL_C_SLONG);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request, const int16_t& expected)
+    {
+        CheckSingleResultNum0<int16_t>(request, expected, SQL_C_SSHORT);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request, const int8_t& expected)
+    {
+        CheckSingleResultNum0<int8_t>(request, expected, SQL_C_STINYINT);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected)
+    {
+        SQLFLOAT res = 0;
+
+        CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
+
+        BOOST_CHECK_CLOSE(static_cast<float>(res), expected, 1E-6f);
+    }
+
+    template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected)
+    {
+        SQLDOUBLE res = 0;
+
+        CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0);
+
+        BOOST_CHECK_CLOSE(static_cast<double>(res), expected, 1E-6);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp
new file mode 100644
index 0000000..80ffd83
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp
@@ -0,0 +1,309 @@
+/*
+ * 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.
+ */
+
+#define _USE_MATH_DEFINES
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <sql.h>
+#include <sqlext.h>
+
+#include <cmath>
+
+#include <vector>
+#include <string>
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "ignite/ignite.h"
+#include "ignite/ignition.h"
+#include "ignite/impl/binary/binary_utils.h"
+
+#include "test_type.h"
+#include "test_utils.h"
+#include "sql_function_test_suite_fixture.h"
+
+using namespace ignite;
+using namespace ignite::cache;
+using namespace ignite::cache::query;
+using namespace ignite::common;
+
+using namespace boost::unit_test;
+
+using ignite::impl::binary::BinaryUtils;
+
+BOOST_FIXTURE_TEST_SUITE(SqlNumericFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionAbs)
+{
+    TestType in;
+
+    in.i32Field = -42;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int32_t>("SELECT {fn ABS(i32Field)} FROM TestType", std::abs(in.i32Field));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionAcos)
+{
+    TestType in;
+
+    in.doubleField = 0.32;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn ACOS(doubleField)} FROM TestType", std::acos(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionAsin)
+{
+    TestType in;
+
+    in.doubleField = 0.12;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn ASIN(doubleField)} FROM TestType", std::asin(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionAtan)
+{
+    TestType in;
+
+    in.doubleField = 0.14;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn ATAN(doubleField)} FROM TestType", std::atan(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionAtan2)
+{
+    TestType in;
+
+    in.doubleField = 0.24;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn ATAN2(doubleField, 0.2)} FROM TestType", std::atan2(in.doubleField, 0.2));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionCeiling)
+{
+    TestType in;
+
+    in.doubleField = 7.31;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn CEILING(doubleField)} FROM TestType", std::ceil(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionCos)
+{
+    TestType in;
+
+    in.doubleField = 2.31;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn COS(doubleField)} FROM TestType", std::cos(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionCot)
+{
+    TestType in;
+
+    in.doubleField = 2.31;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn COT(doubleField)} FROM TestType", 1 / std::tan(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionDegrees)
+{
+    TestType in;
+
+    in.doubleField = 2.31;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn DEGREES(doubleField)} FROM TestType", in.doubleField * M_1_PI * 180);
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionExp)
+{
+    TestType in;
+
+    in.doubleField = 1.23;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn EXP(doubleField)} FROM TestType", std::exp(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionFloor)
+{
+    TestType in;
+
+    in.doubleField = 5.29;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn FLOOR(doubleField)} FROM TestType", std::floor(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionLog)
+{
+    TestType in;
+
+    in.doubleField = 15.3;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn LOG(doubleField)} FROM TestType", std::log(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionLog10)
+{
+    TestType in;
+
+    in.doubleField = 15.3;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn LOG10(doubleField)} FROM TestType", std::log10(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionMod)
+{
+    TestType in;
+
+    in.i64Field = 26;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int64_t>("SELECT {fn MOD(i64Field, 3)} FROM TestType", in.i64Field % 3);
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionPi)
+{
+    CheckSingleResult<double>("SELECT {fn PI()}", M_PI);
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionPower)
+{
+    TestType in;
+
+    in.doubleField = 1.81;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn POWER(doubleField, 2.5)} FROM TestType", std::pow(in.doubleField, 2.5));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionRadians)
+{
+    TestType in;
+
+    in.doubleField = 161;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn RADIANS(doubleField)} FROM TestType", in.doubleField * M_PI / 180.0);
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionRand)
+{
+    CheckSingleResult<double>("SELECT {fn RAND()} * 0", 0);
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionRound)
+{
+    TestType in;
+
+    in.doubleField = 5.29;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn ROUND(doubleField)} FROM TestType", std::floor(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionSign)
+{
+    TestType in;
+
+    in.doubleField = -1.39;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn SIGN(doubleField)} FROM TestType", in.doubleField < 0 ? -1 : in.doubleField == 0 ? 0 : 1);
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionSin)
+{
+    TestType in;
+
+    in.doubleField = 1.01;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn SIN(doubleField)} FROM TestType", std::sin(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionSqrt)
+{
+    TestType in;
+
+    in.doubleField = 2.56;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn SQRT(doubleField)} FROM TestType", std::sqrt(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionTan)
+{
+    TestType in;
+
+    in.doubleField = 0.56;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn TAN(doubleField)} FROM TestType", std::tan(in.doubleField));
+}
+
+BOOST_AUTO_TEST_CASE(TestNumericFunctionTruncate)
+{
+    TestType in;
+
+    in.doubleField = 4.17133;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<double>("SELECT {fn TRUNCATE(doubleField, 3)} FROM TestType", 4.171);
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
index e84aba6..d22bb83 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
@@ -15,8 +15,6 @@
  * limitations under the License.
  */
 
-#define _USE_MATH_DEFINES
-
 #ifdef _WIN32
 #   include <windows.h>
 #endif
@@ -24,8 +22,6 @@
 #include <sql.h>
 #include <sqlext.h>
 
-#include <cmath>
-
 #include <vector>
 #include <string>
 
@@ -41,6 +37,7 @@
 
 #include "test_type.h"
 #include "test_utils.h"
+#include "sql_function_test_suite_fixture.h"
 
 using namespace ignite;
 using namespace ignite::cache;
@@ -51,228 +48,7 @@ using namespace boost::unit_test;
 
 using ignite::impl::binary::BinaryUtils;
 
-/**
- * Test setup fixture.
- */
-struct SqlStringFunctionTestSuiteFixture
-{
-    /**
-     * Constructor.
-     */
-    SqlStringFunctionTestSuiteFixture() :
-        testCache(0),
-        env(NULL),
-        dbc(NULL),
-        stmt(NULL)
-    {
-        IgniteConfiguration cfg;
-
-        cfg.jvmOpts.push_back("-Xdebug");
-        cfg.jvmOpts.push_back("-Xnoagent");
-        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
-        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
-        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
-
-#ifdef IGNITE_TESTS_32
-        cfg.jvmInitMem = 256;
-        cfg.jvmMaxMem = 768;
-#else
-        cfg.jvmInitMem = 1024;
-        cfg.jvmMaxMem = 4096;
-#endif
-
-        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
-
-        BOOST_REQUIRE(cfgPath != 0);
-
-        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
-
-        IgniteError err;
-
-        grid = Ignition::Start(cfg, &err);
-
-        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
-            BOOST_FAIL(err.GetText());
-
-        testCache = grid.GetCache<int64_t, TestType>("cache");
-
-        // Allocate an environment handle
-        SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
-
-        BOOST_REQUIRE(env != NULL);
-
-        // We want ODBC 3 support
-        SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
-
-        // Allocate a connection handle
-        SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
-
-        BOOST_REQUIRE(dbc != NULL);
-
-        // Connect string
-        SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache";
-
-        SQLCHAR outstr[ODBC_BUFFER_SIZE];
-        SQLSMALLINT outstrlen;
-
-        // Connecting to ODBC server.
-        SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast<SQLSMALLINT>(sizeof(connectStr)),
-            outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
-
-        if (!SQL_SUCCEEDED(ret))
-        {
-            Ignition::Stop(grid.GetName(), true);
-
-            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc));
-        }
-
-        // Allocate a statement handle
-        SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
-
-        BOOST_REQUIRE(stmt != NULL);
-    }
-
-    /**
-     * Destructor.
-     */
-    ~SqlStringFunctionTestSuiteFixture()
-    {
-        // Releasing statement handle.
-        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
-
-        // Disconneting from the server.
-        SQLDisconnect(dbc);
-
-        // Releasing allocated handles.
-        SQLFreeHandle(SQL_HANDLE_DBC, dbc);
-        SQLFreeHandle(SQL_HANDLE_ENV, env);
-
-        Ignition::Stop(grid.GetName(), true);
-    }
-
-    void CheckSingleResult0(const char* request, SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize)
-    {
-        SQLRETURN ret;
-
-        ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize);
-
-        if (!SQL_SUCCEEDED(ret))
-            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
-
-        ret = SQLExecDirect(stmt, reinterpret_cast<SQLCHAR*>(const_cast<char*>(request)), SQL_NTS);
-        if (!SQL_SUCCEEDED(ret))
-            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
-
-        ret = SQLFetch(stmt);
-        if (!SQL_SUCCEEDED(ret))
-            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
-
-        ret = SQLFetch(stmt);
-        BOOST_CHECK(ret == SQL_NO_DATA);
-    }
-
-    /**
-     * Run query returning single result and check it to be equal to expected.
-     *
-     * @param request SQL request.
-     * @param expected Expected result.
-     */
-    template<typename T>
-    void CheckSingleResult(const char* request, const T& expected)
-    {
-        BOOST_FAIL("Function is not defined for the type.");
-    }
-    
-    /**
-     * Run query returning single result and check it to be equal to expected.
-     *
-     * @param request SQL request.
-     * @param expected Expected result.
-     * @param type Result type.
-     */
-    template<typename T>
-    void CheckSingleResultNum0(const char* request, const T& expected, SQLSMALLINT type)
-    {
-        T res = 0;
-
-        CheckSingleResult0(request, type, &res, 0, 0);
-
-        BOOST_CHECK_EQUAL(res, expected);
-    }
-
-
-    /** Node started during the test. */
-    Ignite grid;
-
-    /** Test cache instance. */
-    Cache<int64_t, TestType> testCache;
-
-    /** ODBC Environment. */
-    SQLHENV env;
-
-    /** ODBC Connect. */
-    SQLHDBC dbc;
-
-    /** ODBC Statement. */
-    SQLHSTMT stmt;
-};
-
-template<>
-void SqlStringFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request, const std::string& expected)
-{
-    SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
-    SQLLEN resLen = 0;
-
-    CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
-
-    BOOST_CHECK_EQUAL(std::string(reinterpret_cast<char*>(res), static_cast<size_t>(resLen)), expected);
-}
-
-template<>
-void SqlStringFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request, const int64_t& expected)
-{
-    CheckSingleResultNum0<int64_t>(request, expected, SQL_C_SBIGINT);
-}
-
-template<>
-void SqlStringFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request, const int32_t& expected)
-{
-    CheckSingleResultNum0<int32_t>(request, expected, SQL_C_SLONG);
-}
-
-template<>
-void SqlStringFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request, const int16_t& expected)
-{
-    CheckSingleResultNum0<int16_t>(request, expected, SQL_C_SSHORT);
-}
-
-template<>
-void SqlStringFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request, const int8_t& expected)
-{
-    CheckSingleResultNum0<int8_t>(request, expected, SQL_C_STINYINT);
-}
-
-template<>
-void SqlStringFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected)
-{
-    SQLFLOAT res = 0;
-
-    CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
-
-    BOOST_CHECK_CLOSE(static_cast<float>(res), expected, 1E-6f);
-}
-
-template<>
-void SqlStringFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected)
-{
-    SQLDOUBLE res = 0;
-
-    CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0);
-
-    BOOST_CHECK_CLOSE(static_cast<double>(res), expected, 1E-6);
-}
-
-BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, SqlStringFunctionTestSuiteFixture)
+BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestStringFunctionAscii)
 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a3b3e2c/modules/platforms/cpp/odbc/src/config/connection_info.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
index f9a3f96..1fed7c5 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -233,7 +233,12 @@ namespace ignite
 #ifdef SQL_NUMERIC_FUNCTIONS
                 // Bitmask enumerating the scalar numeric functions supported by
                 // the driver and associated data source.
-                intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS;
+                intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS | SQL_FN_NUM_ACOS | SQL_FN_NUM_ASIN | 
+                    SQL_FN_NUM_ATAN | SQL_FN_NUM_ATAN2 | SQL_FN_NUM_CEILING | SQL_FN_NUM_COS | SQL_FN_NUM_COT |
+                    SQL_FN_NUM_EXP | SQL_FN_NUM_FLOOR | SQL_FN_NUM_LOG | SQL_FN_NUM_MOD | SQL_FN_NUM_SIGN |
+                    SQL_FN_NUM_SIN | SQL_FN_NUM_SQRT | SQL_FN_NUM_TAN | SQL_FN_NUM_PI | SQL_FN_NUM_RAND |
+                    SQL_FN_NUM_DEGREES | SQL_FN_NUM_LOG10 | SQL_FN_NUM_POWER | SQL_FN_NUM_RADIANS | SQL_FN_NUM_ROUND |
+                    SQL_FN_NUM_TRUNCATE;
 #endif // SQL_NUMERIC_FUNCTIONS
 
 #ifdef SQL_STRING_FUNCTIONS


[24/24] ignite git commit: Merge branch 'master' of https://github.com/apache/ignite

Posted by sb...@apache.org.
Merge branch 'master' of https://github.com/apache/ignite


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

Branch: refs/heads/master
Commit: d98cd3093f4a494e10ebfe5ee0eb302650716a12
Parents: f89375b 76f26c1
Author: agura <ag...@gridgain.com>
Authored: Mon Aug 29 21:03:05 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Mon Aug 29 21:03:05 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |  5 ---
 .../Cache/Affinity/AffinityFunctionContext.cs   |  4 +--
 .../Apache.Ignite.Core/Events/DiscoveryEvent.cs |  4 +--
 .../Apache.Ignite.Core/Events/EventReader.cs    |  6 ++--
 .../Impl/Binary/DateTimeHolder.cs               |  4 +--
 .../Impl/Binary/Marshaller.cs                   | 36 ++++++++++----------
 .../Impl/Binary/SerializableObjectHolder.cs     |  6 ++--
 .../Affinity/AffinityFunctionSerializer.cs      |  2 +-
 .../Impl/Cache/CacheEntryFilterHolder.cs        | 10 +++---
 .../Impl/Cache/CacheEntryProcessorHolder.cs     |  8 ++---
 .../Continuous/ContinuousQueryFilterHolder.cs   |  8 ++---
 .../Impl/Compute/Closure/ComputeActionJob.cs    |  6 ++--
 .../Impl/Compute/Closure/ComputeFuncJob.cs      |  8 ++---
 .../Impl/Compute/Closure/ComputeOutFuncJob.cs   |  9 ++---
 .../Impl/Compute/ComputeFunc.cs                 |  9 +++--
 .../Impl/Compute/ComputeJob.cs                  | 10 +++---
 .../Impl/Compute/ComputeJobHolder.cs            |  8 ++---
 .../Impl/Compute/ComputeOutFunc.cs              |  8 ++---
 .../Impl/Datastream/StreamReceiverHolder.cs     |  2 +-
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  5 ++-
 .../Impl/Messaging/MessageListenerHolder.cs     |  8 ++---
 .../Apache.Ignite.Linq.csproj                   |  5 ---
 .../Apache.Ignite.NLog.csproj                   |  5 ---
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |  5 ---
 24 files changed, 70 insertions(+), 111 deletions(-)
----------------------------------------------------------------------



[04/24] ignite git commit: IGNITE-3736: ODBC: Reverted back removed "supported" futures.

Posted by sb...@apache.org.
IGNITE-3736: ODBC: Reverted back removed "supported" futures.


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

Branch: refs/heads/master
Commit: 13dfcbe03aca137ee70698f6083df27c10ecdaf9
Parents: c02ad87
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 24 17:31:38 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 24 17:31:38 2016 +0300

----------------------------------------------------------------------
 .../cpp/odbc/src/config/connection_info.cpp     | 23 +++++++++++++++-----
 1 file changed, 18 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/13dfcbe0/modules/platforms/cpp/odbc/src/config/connection_info.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
index f35b790..f9a3f96 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -225,13 +225,15 @@ namespace ignite
 
 #ifdef SQL_AGGREGATE_FUNCTIONS
                 // Bitmask enumerating support for aggregation functions.
-                intParams[SQL_AGGREGATE_FUNCTIONS] = 0;
+                intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_ALL | SQL_AF_AVG |
+                    SQL_AF_COUNT | SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN |
+                    SQL_AF_SUM;
 #endif // SQL_AGGREGATE_FUNCTIONS
 
 #ifdef SQL_NUMERIC_FUNCTIONS
                 // Bitmask enumerating the scalar numeric functions supported by
                 // the driver and associated data source.
-                intParams[SQL_NUMERIC_FUNCTIONS] = 0;
+                intParams[SQL_NUMERIC_FUNCTIONS] = SQL_FN_NUM_ABS;
 #endif // SQL_NUMERIC_FUNCTIONS
 
 #ifdef SQL_STRING_FUNCTIONS
@@ -266,7 +268,15 @@ namespace ignite
 #ifdef SQL_DATETIME_LITERALS
                 // Bitmask enumerating the SQL-92 datetime literals supported by
                 // the data source.
-                intParams[SQL_DATETIME_LITERALS] = 0;
+                intParams[SQL_DATETIME_LITERALS] = SQL_DL_SQL92_INTERVAL_HOUR |
+                    SQL_DL_SQL92_DATE | SQL_DL_SQL92_INTERVAL_MINUTE_TO_SECOND |
+                    SQL_DL_SQL92_TIME | SQL_DL_SQL92_INTERVAL_HOUR_TO_SECOND |
+                    SQL_DL_SQL92_TIMESTAMP | SQL_DL_SQL92_INTERVAL_HOUR_TO_MINUTE |
+                    SQL_DL_SQL92_INTERVAL_YEAR | SQL_DL_SQL92_INTERVAL_DAY_TO_SECOND |
+                    SQL_DL_SQL92_INTERVAL_MONTH | SQL_DL_SQL92_INTERVAL_DAY_TO_HOUR |
+                    SQL_DL_SQL92_INTERVAL_DAY | SQL_DL_SQL92_INTERVAL_DAY_TO_MINUTE |
+                    SQL_DL_SQL92_INTERVAL_MINUTE | SQL_DL_SQL92_INTERVAL_SECOND |
+                    SQL_DL_SQL92_INTERVAL_YEAR_TO_MONTH;
 #endif // SQL_DATETIME_LITERALS
 
 #ifdef SQL_SYSTEM_FUNCTIONS
@@ -284,7 +294,9 @@ namespace ignite
 #ifdef SQL_OJ_CAPABILITIES
                 // Bitmask enumerating the types of outer joins supported by the 
                 // driver and data source.
-                intParams[SQL_OJ_CAPABILITIES] = 0;
+                intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_RIGHT |
+                    SQL_OJ_FULL | SQL_OJ_NESTED | SQL_OJ_INNER |
+                    SQL_OJ_ALL_COMPARISON_OPS;
 #endif // SQL_OJ_CAPABILITIES
 
 #ifdef SQL_POS_OPERATIONS
@@ -305,7 +317,8 @@ namespace ignite
 
 #ifdef SQL_SQL92_DATETIME_FUNCTIONS
                 // Bitmask enumerating the datetime scalar functions.
-                intParams[SQL_SQL92_DATETIME_FUNCTIONS] = 0;
+                intParams[SQL_SQL92_DATETIME_FUNCTIONS] = SQL_SDF_CURRENT_DATE |
+                    SQL_SDF_CURRENT_TIMESTAMP;
 #endif // SQL_SQL92_DATETIME_FUNCTIONS
 
 #ifdef SQL_SQL92_VALUE_EXPRESSIONS


[07/24] ignite git commit: IGNITE-3757: ODBC: Added aggregate functions support. This closes #983.

Posted by sb...@apache.org.
IGNITE-3757: ODBC: Added aggregate functions support. This closes #983.


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

Branch: refs/heads/master
Commit: 0e3a6e2df8b42f255a5a4688d5827dccaabfd3a4
Parents: 5a3b3e2
Author: isapego <is...@gridgain.com>
Authored: Thu Aug 25 14:34:31 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 25 14:34:31 2016 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/odbc-test/Makefile.am     |   1 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   1 +
 .../project/vs/odbc-test.vcxproj.filters        |   3 +
 .../src/sql_aggregate_functions_test.cpp        | 249 +++++++++++++++++++
 .../cpp/odbc/src/config/connection_info.cpp     |   5 +-
 5 files changed, 256 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e3a6e2d/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index 62bdec3..6288637 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -65,6 +65,7 @@ ignite_odbc_tests_SOURCES = \
     src/sql_function_test_suite_fixture.cpp \
     src/sql_string_functions_test.cpp \
     src/sql_numeric_functions_test.cpp \
+    src/sql_aggregate_functions_test.cpp \
     ../odbc/src/cursor.cpp \
     ../odbc/src/config/connection_info.cpp \
     ../odbc/src/app/application_data_buffer.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e3a6e2d/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index a83ed6b..e0b786a 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -168,6 +168,7 @@
     <ClCompile Include="..\..\src\queries_test.cpp" />
     <ClCompile Include="..\..\src\parser_test.cpp" />
     <ClCompile Include="..\..\src\row_test.cpp" />
+    <ClCompile Include="..\..\src\sql_aggregate_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_function_test_suite_fixture.cpp" />
     <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_string_functions_test.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e3a6e2d/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index f993ed9..763a364 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -88,6 +88,9 @@
     <ClCompile Include="..\..\src\sql_function_test_suite_fixture.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\sql_aggregate_functions_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
   </ItemGroup>
   <ItemGroup>
     <ClInclude Include="..\..\include\test_type.h">

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e3a6e2d/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp
new file mode 100644
index 0000000..6254913
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp
@@ -0,0 +1,249 @@
+/*
+ * 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.
+ */
+
+#define _USE_MATH_DEFINES
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <sql.h>
+#include <sqlext.h>
+
+#include <cmath>
+
+#include <vector>
+#include <string>
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "ignite/ignite.h"
+#include "ignite/ignition.h"
+#include "ignite/impl/binary/binary_utils.h"
+
+#include "test_type.h"
+#include "test_utils.h"
+#include "sql_function_test_suite_fixture.h"
+
+using namespace ignite;
+using namespace ignite::cache;
+using namespace ignite::cache::query;
+using namespace ignite::common;
+
+using namespace boost::unit_test;
+
+using ignite::impl::binary::BinaryUtils;
+
+BOOST_FIXTURE_TEST_SUITE(SqlAggregateFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgInt)
+{
+    std::vector<TestType> in(3);
+
+    in[0].i32Field = 43;
+    in[1].i32Field = 311;
+    in[2].i32Field = 7;
+
+    int32_t avg = 0;
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+    {
+        testCache.Put(i, in[i]);
+
+        avg += in[i].i32Field;
+    }
+
+    avg /= static_cast<int32_t>(in.size());
+
+    CheckSingleResult<int64_t>("SELECT {fn AVG(i32Field)} FROM TestType", avg);
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgIntDistinct)
+{
+    std::vector<TestType> in(3);
+
+    in[0].i32Field = 43;
+    in[1].i32Field = 311;
+    in[2].i32Field = 7;
+
+    int32_t avg = 0;
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+    {
+        testCache.Put(i, in[i]);
+
+        avg += in[i].i32Field;
+    }
+
+    avg /= static_cast<int32_t>(in.size());
+
+    testCache.Put(in.size() + 10, in[0]);
+
+    CheckSingleResult<int64_t>("SELECT {fn AVG(DISTINCT i32Field)} FROM TestType", avg);
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgFloat)
+{
+    std::vector<TestType> in(3);
+
+    in[0].floatField = 43.0;
+    in[1].floatField = 311.0;
+    in[2].floatField = 7.0;
+
+    float avg = 0;
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+    {
+        testCache.Put(i, in[i]);
+
+        avg += in[i].i32Field;
+    }
+
+    avg /= in.size();
+
+    CheckSingleResult<float>("SELECT {fn AVG(floatField)} FROM TestType", avg);
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgFloatDistinct)
+{
+    std::vector<TestType> in(3);
+
+    in[0].floatField = 43.0;
+    in[1].floatField = 311.0;
+    in[2].floatField = 7.0;
+
+    float avg = 0;
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+    {
+        testCache.Put(i, in[i]);
+
+        avg += in[i].i32Field;
+    }
+
+    avg /= in.size();
+
+    testCache.Put(in.size() + 10, in[0]);
+
+    CheckSingleResult<float>("SELECT {fn AVG(DISTINCT floatField)} FROM TestType", avg);
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionCount)
+{
+    std::vector<TestType> in(8);
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+        testCache.Put(i, in[i]);
+
+    CheckSingleResult<int64_t>("SELECT {fn COUNT(*)} FROM TestType", in.size());
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionCountDistinct)
+{
+    std::vector<TestType> in(8);
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+    {
+        in[i].i32Field = i;
+
+        testCache.Put(i, in[i]);
+    }
+
+    testCache.Put(in.size() + 10, in[0]);
+
+    CheckSingleResult<int64_t>("SELECT {fn COUNT(DISTINCT i32Field)} FROM TestType", in.size());
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionMax)
+{
+    std::vector<TestType> in(4);
+
+    in[0].i32Field = 121;
+    in[1].i32Field = 17;
+    in[2].i32Field = 314041;
+    in[3].i32Field = 9410;
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+        testCache.Put(i, in[i]);
+
+    CheckSingleResult<int64_t>("SELECT {fn MAX(i32Field)} FROM TestType", in[2].i32Field);
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionMin)
+{
+    std::vector<TestType> in(4);
+
+    in[0].i32Field = 121;
+    in[1].i32Field = 17;
+    in[2].i32Field = 314041;
+    in[3].i32Field = 9410;
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+        testCache.Put(i, in[i]);
+
+    CheckSingleResult<int64_t>("SELECT {fn MIN(i32Field)} FROM TestType", in[1].i32Field);
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionSum)
+{
+    std::vector<TestType> in(4);
+
+    in[0].i32Field = 121;
+    in[1].i32Field = 17;
+    in[2].i32Field = 314041;
+    in[3].i32Field = 9410;
+
+    int64_t sum = 0;
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+    {
+        testCache.Put(i, in[i]);
+
+        sum += in[i].i32Field;
+    }
+
+    CheckSingleResult<int64_t>("SELECT {fn SUM(i32Field)} FROM TestType", sum);
+}
+
+BOOST_AUTO_TEST_CASE(TestAggregateFunctionSumDistinct)
+{
+    std::vector<TestType> in(4);
+
+    in[0].i32Field = 121;
+    in[1].i32Field = 17;
+    in[2].i32Field = 314041;
+    in[3].i32Field = 9410;
+
+    int64_t sum = 0;
+
+    for (int32_t i = 0; i < static_cast<int32_t>(in.size()); ++i)
+    {
+        testCache.Put(i, in[i]);
+
+        sum += in[i].i32Field;
+    }
+
+    testCache.Put(in.size() + 10, in[0]);
+
+    CheckSingleResult<int64_t>("SELECT {fn SUM(DISTINCT i32Field)} FROM TestType", sum);
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/0e3a6e2d/modules/platforms/cpp/odbc/src/config/connection_info.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
index 1fed7c5..b2480c2 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -225,9 +225,8 @@ namespace ignite
 
 #ifdef SQL_AGGREGATE_FUNCTIONS
                 // Bitmask enumerating support for aggregation functions.
-                intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_ALL | SQL_AF_AVG |
-                    SQL_AF_COUNT | SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN |
-                    SQL_AF_SUM;
+                intParams[SQL_AGGREGATE_FUNCTIONS] = SQL_AF_AVG | SQL_AF_COUNT |
+                    SQL_AF_DISTINCT | SQL_AF_MAX | SQL_AF_MIN | SQL_AF_SUM;
 #endif // SQL_AGGREGATE_FUNCTIONS
 
 #ifdef SQL_NUMERIC_FUNCTIONS


[14/24] ignite git commit: IGNITE-3764: ODBC Added tests for SQL operators. This closes #986.

Posted by sb...@apache.org.
IGNITE-3764: ODBC Added tests for SQL operators. This closes #986.


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

Branch: refs/heads/master
Commit: 99e3e8a2d997aa681264460c2845984712ded90e
Parents: 6fd53ea
Author: Igor Sapego <is...@gridgain.com>
Authored: Fri Aug 26 11:23:49 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Aug 26 11:23:49 2016 +0300

----------------------------------------------------------------------
 .../cpp/common/src/common/big_integer.cpp       |   1 -
 modules/platforms/cpp/odbc-test/Makefile.am     |   1 +
 .../include/sql_function_test_suite_fixture.h   |   3 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   1 +
 .../project/vs/odbc-test.vcxproj.filters        |   3 +
 .../src/sql_function_test_suite_fixture.cpp     |  10 +
 .../cpp/odbc-test/src/sql_operators_test.cpp    | 214 +++++++++++++++++++
 7 files changed, 232 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/99e3e8a2/modules/platforms/cpp/common/src/common/big_integer.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/src/common/big_integer.cpp b/modules/platforms/cpp/common/src/common/big_integer.cpp
index 475ddc6..6aa6614 100644
--- a/modules/platforms/cpp/common/src/common/big_integer.cpp
+++ b/modules/platforms/cpp/common/src/common/big_integer.cpp
@@ -58,7 +58,6 @@ namespace ignite
             assert(val != 0);
             assert(len > 0);
             assert(sign == 1 || sign == 0 || sign == -1);
-            assert(val[0] != 0);
 
             if (bigEndian)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/99e3e8a2/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index 1f641be..a3f954e 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -67,6 +67,7 @@ ignite_odbc_tests_SOURCES = \
     src/sql_numeric_functions_test.cpp \
     src/sql_aggregate_functions_test.cpp \
     src/sql_system_functions_test.cpp \
+    src/sql_operators_test.cpp \
     src/sql_value_expressions.cpp \
     ../odbc/src/cursor.cpp \
     ../odbc/src/config/connection_info.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/99e3e8a2/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
index eef0a9b..57c3f0f 100644
--- a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
+++ b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
@@ -161,6 +161,9 @@ namespace ignite
     void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected);
 
     template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<bool>(const char* request, const bool& expected);
+
+    template<>
     void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request);
 
     template<>

http://git-wip-us.apache.org/repos/asf/ignite/blob/99e3e8a2/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index 4d3a9cc..8b4bee6 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -171,6 +171,7 @@
     <ClCompile Include="..\..\src\sql_aggregate_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_function_test_suite_fixture.cpp" />
     <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp" />
+    <ClCompile Include="..\..\src\sql_operators_test.cpp" />
     <ClCompile Include="..\..\src\sql_string_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_system_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_value_expressions.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/99e3e8a2/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index c4d70b7..4756f83 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -94,6 +94,9 @@
     <ClCompile Include="..\..\src\sql_system_functions_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\sql_operators_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\sql_value_expressions.cpp">
       <Filter>Code</Filter>
     </ClCompile>

http://git-wip-us.apache.org/repos/asf/ignite/blob/99e3e8a2/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
index 4d8b919..d6b175f 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
@@ -192,6 +192,16 @@ namespace ignite
     }
 
     template<>
+    void SqlFunctionTestSuiteFixture::CheckSingleResult<bool>(const char* request, const bool& expected)
+    {
+        SQLCHAR res = 0;
+
+        CheckSingleResult0(request, SQL_C_BIT, &res, 0, 0);
+
+        BOOST_CHECK_EQUAL((res != 0), expected);
+    }
+
+    template<>
     void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request)
     {
         SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };

http://git-wip-us.apache.org/repos/asf/ignite/blob/99e3e8a2/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp
new file mode 100644
index 0000000..db3d3f7
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp
@@ -0,0 +1,214 @@
+/*
+ * 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.
+ */
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <vector>
+#include <string>
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "test_type.h"
+#include "sql_function_test_suite_fixture.h"
+#include <ignite/common/decimal.h>
+
+using namespace ignite;
+
+BOOST_FIXTURE_TEST_SUITE(SqlOperatorTestSuite, ignite::SqlFunctionTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestOperatorAddInt)
+{
+    CheckSingleResult<int32_t>("SELECT 123 + 51", 123 + 51);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorSubInt)
+{
+    CheckSingleResult<int32_t>("SELECT 123 - 51", 123 - 51);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorDivInt)
+{
+    CheckSingleResult<int32_t>("SELECT 123 / 51", 123 / 51);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorModInt)
+{
+    CheckSingleResult<int32_t>("SELECT 123 % 51", 123 % 51);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorMultInt)
+{
+    CheckSingleResult<int32_t>("SELECT 123 * 51", 123 * 51);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorAddDouble)
+{
+    CheckSingleResult<double>("SELECT 123.0 + 51.0", 123.0 + 51.0);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorSubDouble)
+{
+    CheckSingleResult<double>("SELECT 123.0 - 51.0", 123.0 - 51.0);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorDivDouble)
+{
+    CheckSingleResult<double>("SELECT 123.0 / 51.0", 123.0 / 51.0);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorModDouble)
+{
+    CheckSingleResult<double>("SELECT 123.0 % 51.0", 123 % 51);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorMultDouble)
+{
+    CheckSingleResult<double>("SELECT 123.0 * 51.0", 123.0 * 51.0);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorConcatString)
+{
+    CheckSingleResult<std::string>("SELECT 'Hello' || ' ' || 'World' || '!'", "Hello World!");
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorGreaterInt)
+{
+    CheckSingleResult<bool>("SELECT 2 > 3", false);
+    CheckSingleResult<bool>("SELECT 3 > 3", false);
+    CheckSingleResult<bool>("SELECT 34 > 3", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorLessInt)
+{
+    CheckSingleResult<bool>("SELECT 4 < 4", false);
+    CheckSingleResult<bool>("SELECT 4 < 4", false);
+    CheckSingleResult<bool>("SELECT 8 < 42", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorGreaterEquealInt)
+{
+    CheckSingleResult<bool>("SELECT 2 >= 3", false);
+    CheckSingleResult<bool>("SELECT 3 >= 3", true);
+    CheckSingleResult<bool>("SELECT 34 >= 3", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorLessEquealInt)
+{
+    CheckSingleResult<bool>("SELECT 4 <= 3", false);
+    CheckSingleResult<bool>("SELECT 4 <= 4", true);
+    CheckSingleResult<bool>("SELECT 8 <= 42", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorEquealInt)
+{
+    CheckSingleResult<bool>("SELECT 4 = 3", false);
+    CheckSingleResult<bool>("SELECT 4 = 4", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorNotEquealInt)
+{
+    CheckSingleResult<bool>("SELECT 4 != 3", true);
+    CheckSingleResult<bool>("SELECT 4 != 4", false);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorGreaterDouble)
+{
+    CheckSingleResult<bool>("SELECT 2 > 3", false);
+    CheckSingleResult<bool>("SELECT 3 > 3", false);
+    CheckSingleResult<bool>("SELECT 34 > 3", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorLessDouble)
+{
+    CheckSingleResult<bool>("SELECT 4.0 < 4.0", false);
+    CheckSingleResult<bool>("SELECT 4.0 < 4.0", false);
+    CheckSingleResult<bool>("SELECT 8.0 < 42.0", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorGreaterEquealDouble)
+{
+    CheckSingleResult<bool>("SELECT 2.0 >= 3.0", false);
+    CheckSingleResult<bool>("SELECT 3.0 >= 3.0", true);
+    CheckSingleResult<bool>("SELECT 34.0 >= 3.0", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorLessEquealDouble)
+{
+    CheckSingleResult<bool>("SELECT 4.0 <= 3.0", false);
+    CheckSingleResult<bool>("SELECT 4.0 <= 4.0", true);
+    CheckSingleResult<bool>("SELECT 8.0 <= 42.0", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorEquealDouble)
+{
+    CheckSingleResult<bool>("SELECT 4.0 = 3.0", false);
+    CheckSingleResult<bool>("SELECT 4.0 = 4.0", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorNotEquealDouble)
+{
+    CheckSingleResult<bool>("SELECT 4.0 != 3.0", true);
+    CheckSingleResult<bool>("SELECT 4.0 != 4.0", false);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorGreaterString)
+{
+    CheckSingleResult<bool>("SELECT 'abc' > 'bcd'", false);
+    CheckSingleResult<bool>("SELECT 'abc' > 'abc'", false);
+    CheckSingleResult<bool>("SELECT 'bcd' > 'abc'", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorLessString)
+{
+    CheckSingleResult<bool>("SELECT 'bcd' < 'abc'", false);
+    CheckSingleResult<bool>("SELECT 'abc' < 'abc'", false);
+    CheckSingleResult<bool>("SELECT 'abc' < 'bcd'", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorGreaterEquealString)
+{
+    CheckSingleResult<bool>("SELECT 'abc' >= 'bcd'", false);
+    CheckSingleResult<bool>("SELECT 'abc' >= 'abc'", true);
+    CheckSingleResult<bool>("SELECT 'bcd' >= 'abc'", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorLessEquealString)
+{
+    CheckSingleResult<bool>("SELECT 'bcd' <= 'abc'", false);
+    CheckSingleResult<bool>("SELECT 'abc' <= 'bcd'", true);
+    CheckSingleResult<bool>("SELECT 'abc' <= 'abc'", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorEquealString)
+{
+    CheckSingleResult<bool>("SELECT 'bcd' = 'abc'", false);
+    CheckSingleResult<bool>("SELECT 'abc' = 'abc'", true);
+};
+
+BOOST_AUTO_TEST_CASE(TestOperatorNotEquealString)
+{
+    CheckSingleResult<bool>("SELECT 'abc' != 'abc'", false);
+    CheckSingleResult<bool>("SELECT 'bcd' != 'abc'", true);
+};
+
+BOOST_AUTO_TEST_SUITE_END()


[08/24] ignite git commit: IGNITE-3738: ODBC: Fixed escape sequence whitespaces handling. This closes #982.

Posted by sb...@apache.org.
IGNITE-3738: ODBC: Fixed escape sequence whitespaces handling. This closes #982.


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

Branch: refs/heads/master
Commit: e21111f287039011bc9437c94fb574e61e2ac226
Parents: 0e3a6e2
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Thu Aug 25 16:26:02 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Aug 25 16:26:02 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/escape/OdbcEscapeToken.java |  61 +++++++++++
 .../processors/odbc/escape/OdbcEscapeType.java  |  81 ++++++++++++++-
 .../processors/odbc/escape/OdbcEscapeUtils.java | 103 +++++++++++++------
 .../odbc/OdbcEscapeSequenceSelfTest.java        |  15 ++-
 4 files changed, 225 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e21111f2/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java
new file mode 100644
index 0000000..6bb4f81
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeToken.java
@@ -0,0 +1,61 @@
+/*
+ * 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.escape;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * ODBC escape sequence token.
+ */
+public class OdbcEscapeToken {
+    /** Escape sequence type. */
+    private final OdbcEscapeType type;
+
+    /** Token length. */
+    private final int len;
+
+    /**
+     * Constructor.
+     *
+     * @param type Escape sequence type.
+     * @param len Token length.
+     */
+    public OdbcEscapeToken(OdbcEscapeType type, int len) {
+        this.type = type;
+        this.len = len;
+    }
+
+    /**
+     * @return Escape sequence type.
+     */
+    public OdbcEscapeType type() {
+        return type;
+    }
+
+    /**
+     * @return Token length.
+     */
+    public int length() {
+        return len;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(OdbcEscapeToken.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e21111f2/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
index 2df413f..96a2127 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeType.java
@@ -22,5 +22,84 @@ package org.apache.ignite.internal.processors.odbc.escape;
  */
 public enum OdbcEscapeType {
     /** Scalar function. */
-    FN
+    SCALAR_FUNCTION("fn", true, false),
+
+    /** Outer join. */
+    OUTER_JOIN("oj", true, false),
+
+    /** Date. */
+    DATE("d", true, false),
+
+    /** Timestamp. */
+    TIMESTAMP("ts", true, false),
+
+    /** Time. */
+    TIME("t", true, false),
+
+    /** GUID. */
+    GUID("guid", true, false),
+
+    /** LIKE clause. */
+    LIKE("\'", false, true);
+
+    /** Values in convenient order. */
+    private static final OdbcEscapeType[] VALS = new OdbcEscapeType[] {
+        SCALAR_FUNCTION, // Assume that scalar functions are very frequent.
+        DATE, TIMESTAMP, // Date and timestamp are relatively frequent as well; also TS must go before T.
+        OUTER_JOIN,      // Joins are less frequent,
+        LIKE, TIME, GUID // LIKE, TIME and GUID are even less frequent.
+    };
+
+    /**
+     * Get values in convenient order, where the most frequent values goes first, and "startsWith" invocation is
+     * enough to get type (i.e. "ts" goes before "t").
+     *
+     * @return Values.
+     */
+    public static OdbcEscapeType[] sortedValues() {
+        return VALS;
+    }
+
+    /** Escape sequence body. */
+    private final String body;
+
+    /** Whether token must be delimited from the rest of escape sequence. */
+    private final boolean delimited;
+
+    /** Whether empty escape sequence is allowed. */
+    private final boolean allowEmpty;
+
+    /**
+     * Constructor.
+     *
+     * @param body Escape sequence body.
+     * @param delimited Whether token must be delimited from the rest of escape sequence.
+     * @param allowEmpty Whether empty escape sequence is allowed.
+     */
+    OdbcEscapeType(String body, boolean delimited, boolean allowEmpty) {
+        this.body = body;
+        this.delimited = delimited;
+        this.allowEmpty = allowEmpty;
+    }
+
+    /**
+     * @return Escape sequence body.
+     */
+    public String body() {
+        return body;
+    }
+
+    /**
+     * @return Whether token must be delimited from the rest of escape sequence.
+     */
+    public boolean delimited() {
+        return delimited;
+    }
+
+    /**
+     * @return Whether empty escape sequence is allowed.
+     */
+    public boolean allowEmpty() {
+        return allowEmpty;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/e21111f2/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
index 4d8ca69..6299c7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
@@ -90,18 +90,18 @@ public class OdbcEscapeUtils {
 
                     if (nested == null)
                         // Found sequence without nesting, process it.
-                        parseRes = parseExpression(text, openPos, curPos - openPos);
+                        parseRes = parseExpression(text, openPos, curPos + 1 - openPos);
                     else {
                         // Special case to process nesting.
                         String res0 = appendNested(text, openPos, curPos + 1, nested);
 
                         nested = null;
 
-                        parseRes = parseExpression(res0, 0, res0.length()-1);
+                        parseRes = parseExpression(res0, 0, res0.length());
                     }
 
                     if (earlyExit)
-                        return new OdbcEscapeParseResult(startPos, curPos - startPos + 1, parseRes);
+                        return new OdbcEscapeParseResult(startPos, curPos + 1 - startPos, parseRes);
                     else
                         res.append(parseRes);
 
@@ -137,23 +137,21 @@ public class OdbcEscapeUtils {
         char firstChar = text.charAt(startPos);
 
         if (firstChar == '{') {
-            char lastChar = text.charAt(startPos + len);
+            char lastChar = text.charAt(startPos + len - 1);
 
             if (lastChar != '}')
                 throw new IgniteException("Failed to parse escape sequence because it is not enclosed: " +
                     substring(text, startPos, len));
 
-            OdbcEscapeType typ = sequenceType(text, startPos, len);
+            OdbcEscapeToken token = parseToken(text, startPos, len);
 
-            switch (typ) {
-                case FN:
-                    return parseScalarExpression(text, startPos, len);
+            switch (token.type()) {
+                case SCALAR_FUNCTION:
+                    return parseScalarExpression(text, startPos, len, token);
 
-                default: {
-                    assert false : "Unknown expression type: " + typ;
-
-                    return null;
-                }
+                default:
+                    throw new IgniteException("Unsupported escape sequence token [text=" +
+                        substring(text, startPos, len) + ", token=" + token.type().body() + ']');
             }
         }
         else {
@@ -161,8 +159,60 @@ public class OdbcEscapeUtils {
             if (startPos == 0 || text.length() == len)
                 return text;
             else
-                return text.substring(startPos, startPos + len);
+                return substring(text, startPos, len);
+        }
+    }
+
+    /**
+     * Get escape sequence info.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @return Escape sequence info.
+     */
+    private static OdbcEscapeToken parseToken(String text, int startPos, int len) {
+        assert validSubstring(text, startPos, len);
+        assert text.charAt(startPos) == '{';
+
+        int pos = startPos + 1;
+
+        while (Character.isWhitespace(text.charAt(pos)))
+            pos++;
+
+        OdbcEscapeType curTyp = null;
+        boolean empty = false;
+
+        for (OdbcEscapeType typ : OdbcEscapeType.sortedValues()) {
+            if (text.startsWith(typ.body(), pos)) {
+                pos += typ.body().length();
+
+                if (typ == OdbcEscapeType.LIKE)
+                    throw new IgniteException("LIKE escape sequence is not supported yet.");
+                else {
+                    empty = (startPos + len == pos + 1);
+
+                    if (!empty && typ.delimited()) {
+                        char charAfter = text.charAt(pos);
+
+                        if (!Character.isWhitespace(charAfter))
+                            throw new IgniteException("Unexpected escape sequence token: " +
+                                substring(text, startPos, len));
+                    }
+                }
+
+                curTyp = typ;
+
+                break;
+            }
         }
+
+        if (curTyp == null)
+            throw new IgniteException("Unsupported escape sequence: " + substring(text, startPos, len));
+
+        if (empty && !curTyp.allowEmpty())
+            throw new IgniteException("Escape sequence cannot be empty: " + substring(text, startPos, len));
+
+        return new OdbcEscapeToken(curTyp, pos - (startPos + 1));
     }
 
     /**
@@ -171,12 +221,16 @@ public class OdbcEscapeUtils {
      * @param text Text.
      * @param startPos Start position.
      * @param len Length.
+     * @param token Token.
      * @return Parsed expression.
      */
-    private static String parseScalarExpression(String text, int startPos, int len) {
+    private static String parseScalarExpression(String text, int startPos, int len, OdbcEscapeToken token) {
         assert validSubstring(text, startPos, len);
 
-        return substring(text, startPos + 3, len - 3).trim();
+        int startPos0 = startPos + 1 /* open brace */ + token.length() /* token. */;
+        int len0 = len - 1 /* open brace */ - token.length() /* token */ - 1 /* close brace */;
+
+        return substring(text, startPos0, len0).trim();
     }
 
     /**
@@ -212,23 +266,6 @@ public class OdbcEscapeUtils {
     }
 
     /**
-     * Get escape sequence type.
-     *
-     * @param text Text.
-     * @param startPos Start position.
-     * @return Escape sequence type.
-     */
-    private static OdbcEscapeType sequenceType(String text, int startPos, int len) {
-        assert validSubstring(text, startPos, len);
-        assert text.charAt(startPos) == '{';
-
-        if (text.startsWith("fn", startPos + 1))
-            return OdbcEscapeType.FN;
-
-        throw new IgniteException("Unsupported escape sequence: " + text.substring(startPos, startPos + len));
-    }
-
-    /**
      * Perform "substring" using start position and length.
      *
      * @param text Text.

http://git-wip-us.apache.org/repos/asf/ignite/blob/e21111f2/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
index 73fa0f4..d9be6cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/odbc/OdbcEscapeSequenceSelfTest.java
@@ -141,7 +141,7 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
     /**
      * Test non-closed escape sequence.
      */
-    public void testFailedOnInvalidSequence1() {
+    public void testFailedOnNonClosedEscapeSequence() {
         checkFail("select {fn func1(field1, {fn func2(field2), field3)} from SomeTable;");
     }
 
@@ -153,6 +153,19 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test escape sequences with additional whitespace characters
+     */
+    public void testFunctionEscapeSequenceWithWhitespaces() throws Exception {
+        check("func1()", "{ fn func1()}");
+
+        check("func1()", "{    fn  func1()}");
+
+        check("func1()", "{ \n fn  func1()}");
+
+        checkFail("{ \n func1()}");
+    }
+
+    /**
      * Check parsing logic.
      *
      * @param exp Expected result.


[18/24] ignite git commit: IGNITE-3773: ODBC: Added tests for UUID data type. This closes #993.

Posted by sb...@apache.org.
IGNITE-3773: ODBC: Added tests for UUID data type. This closes #993.


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

Branch: refs/heads/master
Commit: 289170346e40a89243b80d3743c1b66543a117ef
Parents: b575764
Author: Igor Sapego <is...@gridgain.com>
Authored: Mon Aug 29 15:00:08 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 29 15:00:08 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/escape/OdbcEscapeUtils.java |   2 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |   5 +-
 .../platforms/cpp/odbc-test/include/Makefile.am |   2 +-
 .../include/sql_function_test_suite_fixture.h   | 188 -------------
 .../odbc-test/include/sql_test_suite_fixture.h  | 191 +++++++++++++
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   7 +-
 .../project/vs/odbc-test.vcxproj.filters        |  13 +-
 .../src/sql_aggregate_functions_test.cpp        |   4 +-
 .../src/sql_function_test_suite_fixture.cpp     | 252 -----------------
 .../src/sql_numeric_functions_test.cpp          |   4 +-
 .../cpp/odbc-test/src/sql_operators_test.cpp    |   4 +-
 .../odbc-test/src/sql_string_functions_test.cpp |   4 +-
 .../odbc-test/src/sql_system_functions_test.cpp |   4 +-
 .../odbc-test/src/sql_test_suite_fixture.cpp    | 271 +++++++++++++++++++
 .../cpp/odbc-test/src/sql_types_test.cpp        |  60 ++++
 .../cpp/odbc-test/src/sql_value_expressions.cpp |  94 -------
 .../src/sql_value_expressions_test.cpp          |  94 +++++++
 17 files changed, 643 insertions(+), 556 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
index 4c1deb6..a4b89c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/escape/OdbcEscapeUtils.java
@@ -291,7 +291,7 @@ public class OdbcEscapeUtils {
         if (!pattern.matcher(val).matches())
             throw new IgniteException("Invalid " + type + " escape sequence: " + substring(text, startPos, len));
 
-        return val;
+        return "CAST(" + val + " AS UUID)";
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/Makefile.am b/modules/platforms/cpp/odbc-test/Makefile.am
index a3f954e..de8fb5d 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -62,13 +62,14 @@ ignite_odbc_tests_SOURCES = \
     src/utility_test.cpp \
     src/queries_test.cpp \
     src/test_utils.cpp \
-    src/sql_function_test_suite_fixture.cpp \
+    src/sql_test_suite_fixture.cpp \
     src/sql_string_functions_test.cpp \
     src/sql_numeric_functions_test.cpp \
     src/sql_aggregate_functions_test.cpp \
     src/sql_system_functions_test.cpp \
     src/sql_operators_test.cpp \
-    src/sql_value_expressions.cpp \
+    src/sql_value_expressions_test.cpp \
+    src/sql_types_test.cpp \
     ../odbc/src/cursor.cpp \
     ../odbc/src/config/connection_info.cpp \
     ../odbc/src/app/application_data_buffer.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/Makefile.am b/modules/platforms/cpp/odbc-test/include/Makefile.am
index baca1b7..832103c 100644
--- a/modules/platforms/cpp/odbc-test/include/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/include/Makefile.am
@@ -21,5 +21,5 @@ noinst_HEADERS = \
     teamcity/teamcity_messages.h \
     test_type.h \
     test_utils.h \
-    sql_function_test_suite_fixture.h
+    sql_test_suite_fixture.h
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
deleted file mode 100644
index 57c3f0f..0000000
--- a/modules/platforms/cpp/odbc-test/include/sql_function_test_suite_fixture.h
+++ /dev/null
@@ -1,188 +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.
- */
-
-#ifndef _IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE
-#define _IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE
-
-#ifdef _WIN32
-#   include <windows.h>
-#endif
-
-#include <sql.h>
-#include <sqlext.h>
-
-#include <string>
-
-#ifndef _MSC_VER
-#   define BOOST_TEST_DYN_LINK
-#endif
-
-#include <boost/test/unit_test.hpp>
-
-#include "ignite/ignite.h"
-#include "ignite/ignition.h"
-
-#include "test_type.h"
-
-namespace ignite
-{
-    /**
-     * Test setup fixture.
-     */
-    struct SqlFunctionTestSuiteFixture
-    {
-        /**
-         * Constructor.
-         */
-        SqlFunctionTestSuiteFixture();
-
-        /**
-         * Destructor.
-         */
-        ~SqlFunctionTestSuiteFixture();
-
-        /**
-         * Run query returning single result and stores it to buffer.
-         *
-         * @param request SQL request.
-         * @param type Result type.
-         * @param column Result buffer.
-         * @param bufSize Result buffer size.
-         * @param resSize Size of received value.
-         */
-        void CheckSingleResult0(const char* request, SQLSMALLINT type,
-            void* column, SQLLEN bufSize, SQLLEN* resSize) const;
-
-        /**
-         * Run query returning single result and check it to be equal to expected.
-         *
-         * @param request SQL request.
-         * @param expected Expected result.
-         */
-        template<typename T>
-        void CheckSingleResult(const char* request, const T& expected)
-        {
-            BOOST_FAIL("Function is not defined for the type.");
-        }
-
-        /**
-         * Run query returning single result and check it to be successful.
-         *
-         * @param request SQL request.
-         */
-        template<typename T>
-        void CheckSingleResult(const char* request)
-        {
-            BOOST_FAIL("Function is not defined for the type.");
-        }
-
-        /**
-         * Run query returning single result and check it to be equal to expected.
-         *
-         * @param request SQL request.
-         * @param expected Expected result.
-         * @param type Result type.
-         */
-        template<typename T>
-        void CheckSingleResultNum0(const char* request, const T& expected, SQLSMALLINT type)
-        {
-            T res = 0;
-
-            CheckSingleResult0(request, type, &res, 0, 0);
-
-            BOOST_CHECK_EQUAL(res, expected);
-        }
-
-        /**
-         * Run query returning single result and check it to be equal to expected.
-         *
-         * @param request SQL request.
-         * @param expected Expected result.
-         * @param type Result type.
-         */
-        template<typename T>
-        void CheckSingleResultNum0(const char* request, SQLSMALLINT type)
-        {
-            T res = 0;
-
-            CheckSingleResult0(request, type, &res, 0, 0);
-        }
-
-
-        /** Node started during the test. */
-        Ignite grid;
-
-        /** Test cache instance. */
-        cache::Cache<int64_t, TestType> testCache;
-
-        /** ODBC Environment. */
-        SQLHENV env;
-
-        /** ODBC Connect. */
-        SQLHDBC dbc;
-
-        /** ODBC Statement. */
-        SQLHSTMT stmt;
-    };
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request, const std::string& expected);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request, const int64_t& expected);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request, const int32_t& expected);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request, const int16_t& expected);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request, const int8_t& expected);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<bool>(const char* request, const bool& expected);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request);
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request);
-}
-
-#endif //_IGNITE_ODBC_TEST_SQL_FUNCTION_TEST_FIXTURE

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
new file mode 100644
index 0000000..9e482da
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
@@ -0,0 +1,191 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE
+#define _IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE
+
+#ifdef _WIN32
+#   include <windows.h>
+#endif
+
+#include <sql.h>
+#include <sqlext.h>
+
+#include <string>
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "ignite/ignite.h"
+#include "ignite/ignition.h"
+
+#include "test_type.h"
+
+namespace ignite
+{
+    /**
+     * Test setup fixture.
+     */
+    struct SqlTestSuiteFixture
+    {
+        /**
+         * Constructor.
+         */
+        SqlTestSuiteFixture();
+
+        /**
+         * Destructor.
+         */
+        ~SqlTestSuiteFixture();
+
+        /**
+         * Run query returning single result and stores it to buffer.
+         *
+         * @param request SQL request.
+         * @param type Result type.
+         * @param column Result buffer.
+         * @param bufSize Result buffer size.
+         * @param resSize Size of received value.
+         */
+        void CheckSingleResult0(const char* request, SQLSMALLINT type,
+            void* column, SQLLEN bufSize, SQLLEN* resSize) const;
+
+        /**
+         * Run query returning single result and check it to be equal to expected.
+         *
+         * @param request SQL request.
+         * @param expected Expected result.
+         */
+        template<typename T>
+        void CheckSingleResult(const char* request, const T& expected)
+        {
+            BOOST_FAIL("Function is not defined for the type.");
+        }
+
+        /**
+         * Run query returning single result and check it to be successful.
+         *
+         * @param request SQL request.
+         */
+        template<typename T>
+        void CheckSingleResult(const char* request)
+        {
+            BOOST_FAIL("Function is not defined for the type.");
+        }
+
+        /**
+         * Run query returning single result and check it to be equal to expected.
+         *
+         * @param request SQL request.
+         * @param expected Expected result.
+         * @param type Result type.
+         */
+        template<typename T>
+        void CheckSingleResultNum0(const char* request, const T& expected, SQLSMALLINT type)
+        {
+            T res = 0;
+
+            CheckSingleResult0(request, type, &res, 0, 0);
+
+            BOOST_CHECK_EQUAL(res, expected);
+        }
+
+        /**
+         * Run query returning single result and check it to be equal to expected.
+         *
+         * @param request SQL request.
+         * @param expected Expected result.
+         * @param type Result type.
+         */
+        template<typename T>
+        void CheckSingleResultNum0(const char* request, SQLSMALLINT type)
+        {
+            T res = 0;
+
+            CheckSingleResult0(request, type, &res, 0, 0);
+        }
+
+
+        /** Node started during the test. */
+        Ignite grid;
+
+        /** Test cache instance. */
+        cache::Cache<int64_t, TestType> testCache;
+
+        /** ODBC Environment. */
+        SQLHENV env;
+
+        /** ODBC Connect. */
+        SQLHDBC dbc;
+
+        /** ODBC Statement. */
+        SQLHSTMT stmt;
+    };
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<std::string>(const char* request, const std::string& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int64_t>(const char* request, const int64_t& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int32_t>(const char* request, const int32_t& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int16_t>(const char* request, const int16_t& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int8_t>(const char* request, const int8_t& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<bool>(const char* request, const bool& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<ignite::Guid>(const char* request, const ignite::Guid& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<std::string>(const char* request);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int64_t>(const char* request);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int32_t>(const char* request);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int16_t>(const char* request);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int8_t>(const char* request);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<float>(const char* request);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<double>(const char* request);
+}
+
+#endif //_IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
index 8b4bee6..a9423ca 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -169,19 +169,20 @@
     <ClCompile Include="..\..\src\parser_test.cpp" />
     <ClCompile Include="..\..\src\row_test.cpp" />
     <ClCompile Include="..\..\src\sql_aggregate_functions_test.cpp" />
-    <ClCompile Include="..\..\src\sql_function_test_suite_fixture.cpp" />
+    <ClCompile Include="..\..\src\sql_test_suite_fixture.cpp" />
     <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_operators_test.cpp" />
     <ClCompile Include="..\..\src\sql_string_functions_test.cpp" />
     <ClCompile Include="..\..\src\sql_system_functions_test.cpp" />
-    <ClCompile Include="..\..\src\sql_value_expressions.cpp" />
+    <ClCompile Include="..\..\src\sql_types_test.cpp" />
+    <ClCompile Include="..\..\src\sql_value_expressions_test.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_boost.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_messages.cpp" />
     <ClCompile Include="..\..\src\test_utils.cpp" />
     <ClCompile Include="..\..\src\utility_test.cpp" />
   </ItemGroup>
   <ItemGroup>
-    <ClInclude Include="..\..\include\sql_function_test_suite_fixture.h" />
+    <ClInclude Include="..\..\include\sql_test_suite_fixture.h" />
     <ClInclude Include="..\..\include\teamcity\teamcity_messages.h" />
     <ClInclude Include="..\..\include\test_type.h" />
     <ClInclude Include="..\..\include\test_utils.h" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
index 4756f83..84f5a29 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj.filters
@@ -85,9 +85,6 @@
     <ClCompile Include="..\..\src\sql_numeric_functions_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
-    <ClCompile Include="..\..\src\sql_function_test_suite_fixture.cpp">
-      <Filter>Code</Filter>
-    </ClCompile>
     <ClCompile Include="..\..\src\sql_aggregate_functions_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
@@ -97,7 +94,13 @@
     <ClCompile Include="..\..\src\sql_operators_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
-    <ClCompile Include="..\..\src\sql_value_expressions.cpp">
+    <ClCompile Include="..\..\src\sql_types_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\sql_test_suite_fixture.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
+    <ClCompile Include="..\..\src\sql_value_expressions_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>
   </ItemGroup>
@@ -111,7 +114,7 @@
     <ClInclude Include="..\..\include\test_utils.h">
       <Filter>Code</Filter>
     </ClInclude>
-    <ClInclude Include="..\..\include\sql_function_test_suite_fixture.h">
+    <ClInclude Include="..\..\include\sql_test_suite_fixture.h">
       <Filter>Code</Filter>
     </ClInclude>
   </ItemGroup>

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp
index 6254913..3fa4d97 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_aggregate_functions_test.cpp
@@ -41,7 +41,7 @@
 
 #include "test_type.h"
 #include "test_utils.h"
-#include "sql_function_test_suite_fixture.h"
+#include "sql_test_suite_fixture.h"
 
 using namespace ignite;
 using namespace ignite::cache;
@@ -52,7 +52,7 @@ using namespace boost::unit_test;
 
 using ignite::impl::binary::BinaryUtils;
 
-BOOST_FIXTURE_TEST_SUITE(SqlAggregateFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
+BOOST_FIXTURE_TEST_SUITE(SqlAggregateFunctionTestSuite, ignite::SqlTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgInt)
 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
deleted file mode 100644
index d6b175f..0000000
--- a/modules/platforms/cpp/odbc-test/src/sql_function_test_suite_fixture.cpp
+++ /dev/null
@@ -1,252 +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.
- */
-
-#include "sql_function_test_suite_fixture.h"
-
-#include "test_utils.h"
-
-namespace ignite
-{
-    SqlFunctionTestSuiteFixture::SqlFunctionTestSuiteFixture():
-        testCache(0),
-        env(NULL),
-        dbc(NULL),
-        stmt(NULL)
-    {
-        IgniteConfiguration cfg;
-
-        cfg.jvmOpts.push_back("-Xdebug");
-        cfg.jvmOpts.push_back("-Xnoagent");
-        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
-        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
-        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
-
-#ifdef IGNITE_TESTS_32
-        cfg.jvmInitMem = 256;
-        cfg.jvmMaxMem = 768;
-#else
-        cfg.jvmInitMem = 1024;
-        cfg.jvmMaxMem = 4096;
-#endif
-
-        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
-
-        BOOST_REQUIRE(cfgPath != 0) ;
-
-        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
-
-        IgniteError err;
-
-        grid = Ignition::Start(cfg, &err);
-
-        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
-        BOOST_FAIL(err.GetText()) ;
-
-        testCache = grid.GetCache<int64_t, TestType>("cache");
-
-        // Allocate an environment handle
-        SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
-
-        BOOST_REQUIRE(env != NULL) ;
-
-        // We want ODBC 3 support
-        SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
-
-        // Allocate a connection handle
-        SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
-
-        BOOST_REQUIRE(dbc != NULL) ;
-
-        // Connect string
-        SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache";
-
-        SQLCHAR outstr[ODBC_BUFFER_SIZE];
-        SQLSMALLINT outstrlen;
-
-        // Connecting to ODBC server.
-        SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast<SQLSMALLINT>(sizeof(connectStr)),
-                                         outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
-
-        if (!SQL_SUCCEEDED(ret))
-        {
-            Ignition::Stop(grid.GetName(), true);
-
-            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)) ;
-        }
-
-        // Allocate a statement handle
-        SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
-
-        BOOST_REQUIRE(stmt != NULL) ;
-    }
-
-    SqlFunctionTestSuiteFixture::~SqlFunctionTestSuiteFixture()
-    {
-        // Releasing statement handle.
-        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
-
-        // Disconneting from the server.
-        SQLDisconnect(dbc);
-
-        // Releasing allocated handles.
-        SQLFreeHandle(SQL_HANDLE_DBC, dbc);
-        SQLFreeHandle(SQL_HANDLE_ENV, env);
-
-        ignite::Ignition::Stop(grid.GetName(), true);
-    }
-
-    void SqlFunctionTestSuiteFixture::CheckSingleResult0(const char* request,
-        SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize) const
-    {
-        SQLRETURN ret;
-
-        ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize);
-
-        if (!SQL_SUCCEEDED(ret))
-        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
-
-        ret = SQLExecDirect(stmt, reinterpret_cast<SQLCHAR*>(const_cast<char*>(request)), SQL_NTS);
-        if (!SQL_SUCCEEDED(ret))
-        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
-
-        ret = SQLFetch(stmt);
-        if (!SQL_SUCCEEDED(ret))
-        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
-
-        ret = SQLFetch(stmt);
-        BOOST_CHECK(ret == SQL_NO_DATA) ;
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request, const std::string& expected)
-    {
-        SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
-        SQLLEN resLen = 0;
-
-        CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
-
-        std::string actual;
-
-        if (resLen > 0)
-            actual.assign(reinterpret_cast<char*>(res), static_cast<size_t>(resLen));
-
-        BOOST_CHECK_EQUAL(actual, expected);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request, const int64_t& expected)
-    {
-        CheckSingleResultNum0<int64_t>(request, expected, SQL_C_SBIGINT);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request, const int32_t& expected)
-    {
-        CheckSingleResultNum0<int32_t>(request, expected, SQL_C_SLONG);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request, const int16_t& expected)
-    {
-        CheckSingleResultNum0<int16_t>(request, expected, SQL_C_SSHORT);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request, const int8_t& expected)
-    {
-        CheckSingleResultNum0<int8_t>(request, expected, SQL_C_STINYINT);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected)
-    {
-        SQLFLOAT res = 0;
-
-        CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
-
-        BOOST_CHECK_CLOSE(static_cast<float>(res), expected, 1E-6f);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected)
-    {
-        SQLDOUBLE res = 0;
-
-        CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0);
-
-        BOOST_CHECK_CLOSE(static_cast<double>(res), expected, 1E-6);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<bool>(const char* request, const bool& expected)
-    {
-        SQLCHAR res = 0;
-
-        CheckSingleResult0(request, SQL_C_BIT, &res, 0, 0);
-
-        BOOST_CHECK_EQUAL((res != 0), expected);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<std::string>(const char* request)
-    {
-        SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
-        SQLLEN resLen = 0;
-
-        CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int64_t>(const char* request)
-    {
-        CheckSingleResultNum0<int64_t>(request, SQL_C_SBIGINT);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int32_t>(const char* request)
-    {
-        CheckSingleResultNum0<int32_t>(request, SQL_C_SLONG);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int16_t>(const char* request)
-    {
-        CheckSingleResultNum0<int16_t>(request, SQL_C_SSHORT);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<int8_t>(const char* request)
-    {
-        CheckSingleResultNum0<int8_t>(request, SQL_C_STINYINT);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<float>(const char* request)
-    {
-        SQLFLOAT res = 0;
-
-        CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
-    }
-
-    template<>
-    void SqlFunctionTestSuiteFixture::CheckSingleResult<double>(const char* request)
-    {
-        SQLDOUBLE res = 0;
-
-        CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp
index 80ffd83..e16d8c1 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_numeric_functions_test.cpp
@@ -41,7 +41,7 @@
 
 #include "test_type.h"
 #include "test_utils.h"
-#include "sql_function_test_suite_fixture.h"
+#include "sql_test_suite_fixture.h"
 
 using namespace ignite;
 using namespace ignite::cache;
@@ -52,7 +52,7 @@ using namespace boost::unit_test;
 
 using ignite::impl::binary::BinaryUtils;
 
-BOOST_FIXTURE_TEST_SUITE(SqlNumericFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
+BOOST_FIXTURE_TEST_SUITE(SqlNumericFunctionTestSuite, ignite::SqlTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestNumericFunctionAbs)
 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp
index db3d3f7..de884ca 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_operators_test.cpp
@@ -29,12 +29,12 @@
 #include <boost/test/unit_test.hpp>
 
 #include "test_type.h"
-#include "sql_function_test_suite_fixture.h"
+#include "sql_test_suite_fixture.h"
 #include <ignite/common/decimal.h>
 
 using namespace ignite;
 
-BOOST_FIXTURE_TEST_SUITE(SqlOperatorTestSuite, ignite::SqlFunctionTestSuiteFixture)
+BOOST_FIXTURE_TEST_SUITE(SqlOperatorTestSuite, ignite::SqlTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestOperatorAddInt)
 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
index d22bb83..d1ce194 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_string_functions_test.cpp
@@ -37,7 +37,7 @@
 
 #include "test_type.h"
 #include "test_utils.h"
-#include "sql_function_test_suite_fixture.h"
+#include "sql_test_suite_fixture.h"
 
 using namespace ignite;
 using namespace ignite::cache;
@@ -48,7 +48,7 @@ using namespace boost::unit_test;
 
 using ignite::impl::binary::BinaryUtils;
 
-BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
+BOOST_FIXTURE_TEST_SUITE(SqlStringFunctionTestSuite, ignite::SqlTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestStringFunctionAscii)
 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
index 752925d..30fa036 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_system_functions_test.cpp
@@ -21,13 +21,13 @@
 
 #include <boost/test/unit_test.hpp>
 
-#include "sql_function_test_suite_fixture.h"
+#include "sql_test_suite_fixture.h"
 
 using namespace ignite;
 
 using namespace boost::unit_test;
 
-BOOST_FIXTURE_TEST_SUITE(SqlSystemFunctionTestSuite, ignite::SqlFunctionTestSuiteFixture)
+BOOST_FIXTURE_TEST_SUITE(SqlSystemFunctionTestSuite, ignite::SqlTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestSystemFunctionDatabase)
 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
new file mode 100644
index 0000000..16e5ea0
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
@@ -0,0 +1,271 @@
+/*
+ * 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.
+ */
+
+#include "sql_test_suite_fixture.h"
+
+#include "test_utils.h"
+
+namespace ignite
+{
+    SqlTestSuiteFixture::SqlTestSuiteFixture():
+        testCache(0),
+        env(NULL),
+        dbc(NULL),
+        stmt(NULL)
+    {
+        IgniteConfiguration cfg;
+
+        cfg.jvmOpts.push_back("-Xdebug");
+        cfg.jvmOpts.push_back("-Xnoagent");
+        cfg.jvmOpts.push_back("-Djava.compiler=NONE");
+        cfg.jvmOpts.push_back("-agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=5005");
+        cfg.jvmOpts.push_back("-XX:+HeapDumpOnOutOfMemoryError");
+
+#ifdef IGNITE_TESTS_32
+        cfg.jvmInitMem = 256;
+        cfg.jvmMaxMem = 768;
+#else
+        cfg.jvmInitMem = 1024;
+        cfg.jvmMaxMem = 4096;
+#endif
+
+        char* cfgPath = getenv("IGNITE_NATIVE_TEST_ODBC_CONFIG_PATH");
+
+        BOOST_REQUIRE(cfgPath != 0) ;
+
+        cfg.springCfgPath.assign(cfgPath).append("/queries-test.xml");
+
+        IgniteError err;
+
+        grid = Ignition::Start(cfg, &err);
+
+        if (err.GetCode() != IgniteError::IGNITE_SUCCESS)
+        BOOST_FAIL(err.GetText()) ;
+
+        testCache = grid.GetCache<int64_t, TestType>("cache");
+
+        // Allocate an environment handle
+        SQLAllocHandle(SQL_HANDLE_ENV, SQL_NULL_HANDLE, &env);
+
+        BOOST_REQUIRE(env != NULL) ;
+
+        // We want ODBC 3 support
+        SQLSetEnvAttr(env, SQL_ATTR_ODBC_VERSION, reinterpret_cast<void*>(SQL_OV_ODBC3), 0);
+
+        // Allocate a connection handle
+        SQLAllocHandle(SQL_HANDLE_DBC, env, &dbc);
+
+        BOOST_REQUIRE(dbc != NULL) ;
+
+        // Connect string
+        SQLCHAR connectStr[] = "DRIVER={Apache Ignite};SERVER=localhost;PORT=10800;CACHE=cache";
+
+        SQLCHAR outstr[ODBC_BUFFER_SIZE];
+        SQLSMALLINT outstrlen;
+
+        // Connecting to ODBC server.
+        SQLRETURN ret = SQLDriverConnect(dbc, NULL, connectStr, static_cast<SQLSMALLINT>(sizeof(connectStr)),
+                                         outstr, sizeof(outstr), &outstrlen, SQL_DRIVER_COMPLETE);
+
+        if (!SQL_SUCCEEDED(ret))
+        {
+            Ignition::Stop(grid.GetName(), true);
+
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_DBC, dbc)) ;
+        }
+
+        // Allocate a statement handle
+        SQLAllocHandle(SQL_HANDLE_STMT, dbc, &stmt);
+
+        BOOST_REQUIRE(stmt != NULL) ;
+    }
+
+    SqlTestSuiteFixture::~SqlTestSuiteFixture()
+    {
+        // Releasing statement handle.
+        SQLFreeHandle(SQL_HANDLE_STMT, stmt);
+
+        // Disconneting from the server.
+        SQLDisconnect(dbc);
+
+        // Releasing allocated handles.
+        SQLFreeHandle(SQL_HANDLE_DBC, dbc);
+        SQLFreeHandle(SQL_HANDLE_ENV, env);
+
+        ignite::Ignition::Stop(grid.GetName(), true);
+    }
+
+    void SqlTestSuiteFixture::CheckSingleResult0(const char* request,
+        SQLSMALLINT type, void* column, SQLLEN bufSize, SQLLEN* resSize) const
+    {
+        SQLRETURN ret;
+
+        ret = SQLBindCol(stmt, 1, type, column, bufSize, resSize);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
+
+        ret = SQLExecDirect(stmt, reinterpret_cast<SQLCHAR*>(const_cast<char*>(request)), SQL_NTS);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
+
+        ret = SQLFetch(stmt);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt)) ;
+
+        ret = SQLFetch(stmt);
+        BOOST_CHECK(ret == SQL_NO_DATA) ;
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<std::string>(const char* request, const std::string& expected)
+    {
+        SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
+        SQLLEN resLen = 0;
+
+        CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
+
+        std::string actual;
+
+        if (resLen > 0)
+            actual.assign(reinterpret_cast<char*>(res), static_cast<size_t>(resLen));
+
+        BOOST_CHECK_EQUAL(actual, expected);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int64_t>(const char* request, const int64_t& expected)
+    {
+        CheckSingleResultNum0<int64_t>(request, expected, SQL_C_SBIGINT);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int32_t>(const char* request, const int32_t& expected)
+    {
+        CheckSingleResultNum0<int32_t>(request, expected, SQL_C_SLONG);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int16_t>(const char* request, const int16_t& expected)
+    {
+        CheckSingleResultNum0<int16_t>(request, expected, SQL_C_SSHORT);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int8_t>(const char* request, const int8_t& expected)
+    {
+        CheckSingleResultNum0<int8_t>(request, expected, SQL_C_STINYINT);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected)
+    {
+        SQLFLOAT res = 0;
+
+        CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
+
+        BOOST_CHECK_CLOSE(static_cast<float>(res), expected, 1E-6f);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<double>(const char* request, const double& expected)
+    {
+        SQLDOUBLE res = 0;
+
+        CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0);
+
+        BOOST_CHECK_CLOSE(static_cast<double>(res), expected, 1E-6);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<bool>(const char* request, const bool& expected)
+    {
+        SQLCHAR res = 0;
+
+        CheckSingleResult0(request, SQL_C_BIT, &res, 0, 0);
+
+        BOOST_CHECK_EQUAL((res != 0), expected);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<ignite::Guid>(const char* request, const ignite::Guid& expected)
+    {
+        SQLGUID res;
+
+        memset(&res, 0, sizeof(res));
+
+        CheckSingleResult0(request, SQL_C_GUID, &res, 0, 0);
+
+        BOOST_CHECK_EQUAL(res.Data1, expected.GetMostSignificantBits() & 0xFFFFFFFF00000000ULL >> 32);
+        BOOST_CHECK_EQUAL(res.Data2, expected.GetMostSignificantBits() & 0x00000000FFFF0000ULL >> 16);
+        BOOST_CHECK_EQUAL(res.Data3, expected.GetMostSignificantBits() & 0x000000000000FFFFULL);
+
+        for (int i = 0; i < sizeof(res.Data4); ++i)
+            BOOST_CHECK_EQUAL(res.Data4[i], (expected.GetLeastSignificantBits() & (0xFFULL << (8 * i))) >> (8 * i));
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<std::string>(const char* request)
+    {
+        SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
+        SQLLEN resLen = 0;
+
+        CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int64_t>(const char* request)
+    {
+        CheckSingleResultNum0<int64_t>(request, SQL_C_SBIGINT);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int32_t>(const char* request)
+    {
+        CheckSingleResultNum0<int32_t>(request, SQL_C_SLONG);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int16_t>(const char* request)
+    {
+        CheckSingleResultNum0<int16_t>(request, SQL_C_SSHORT);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<int8_t>(const char* request)
+    {
+        CheckSingleResultNum0<int8_t>(request, SQL_C_STINYINT);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<float>(const char* request)
+    {
+        SQLFLOAT res = 0;
+
+        CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<double>(const char* request)
+    {
+        SQLDOUBLE res = 0;
+
+        CheckSingleResult0(request, SQL_C_DOUBLE, &res, 0, 0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
new file mode 100644
index 0000000..ccbb917
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
@@ -0,0 +1,60 @@
+/*
+ * 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.
+ */
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "sql_test_suite_fixture.h"
+
+using namespace ignite;
+
+using namespace boost::unit_test;
+
+BOOST_FIXTURE_TEST_SUITE(SqlTypesTestSuite, ignite::SqlTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestGuidTrivial)
+{
+    CheckSingleResult<std::string>("SELECT {guid '04CC382A-0B82-F520-08D0-07A0620C0004'}",
+        "04cc382a-0b82-f520-08d0-07a0620c0004");
+
+    CheckSingleResult<std::string>("SELECT {guid '63802467-9f4a-4f71-8fc8-cf2d99a28ddf'}",
+        "63802467-9f4a-4f71-8fc8-cf2d99a28ddf");
+}
+
+BOOST_AUTO_TEST_CASE(TestGuidEqualsToColumn)
+{
+    TestType in1;
+    TestType in2;
+
+    in1.guidField = Guid(0x638024679f4a4f71, 0x8fc8cf2d99a28ddf);
+    in2.guidField = Guid(0x04cc382a0b82f520, 0x08d007a0620c0004);
+
+    in1.i32Field = 1;
+    in2.i32Field = 2;
+
+    testCache.Put(1, in1);
+    testCache.Put(2, in2);
+
+    CheckSingleResult<int32_t>(
+        "SELECT i32Field FROM TestType WHERE guidField = {guid '04cc382a-0b82-f520-08d0-07a0620c0004'}", in2.i32Field);
+}
+
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp b/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp
deleted file mode 100644
index e7d6598..0000000
--- a/modules/platforms/cpp/odbc-test/src/sql_value_expressions.cpp
+++ /dev/null
@@ -1,94 +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.
- */
-
-#ifndef _MSC_VER
-#   define BOOST_TEST_DYN_LINK
-#endif
-
-#include <boost/test/unit_test.hpp>
-
-#include "sql_function_test_suite_fixture.h"
-
-using namespace ignite;
-
-using namespace boost::unit_test;
-
-BOOST_FIXTURE_TEST_SUITE(SqlValueExpressionTestSuite, ignite::SqlFunctionTestSuiteFixture)
-
-BOOST_AUTO_TEST_CASE(TestCase)
-{
-    TestType in;
-
-    in.i32Field = 82;
-
-    testCache.Put(1, in);
-
-    CheckSingleResult<int32_t>(
-        "SELECT "
-            "CASE i32Field WHEN 82 "
-                "THEN (i32Field / 2) "
-                "ELSE (i32Field / 3) "
-            "END "
-        "FROM TestType", in.i32Field / 2);
-
-    
-    CheckSingleResult<int32_t>(
-        "SELECT "
-            "CASE i32Field WHEN 22 "
-                "THEN (i32Field / 2) "
-                "ELSE (i32Field / 3) "
-            "END "
-        "FROM TestType", in.i32Field / 3);;
-}
-
-BOOST_AUTO_TEST_CASE(TestCast)
-{
-    TestType in;
-
-    in.i32Field = 12345;
-    in.strField = "54321";
-
-    testCache.Put(1, in);
-
-    CheckSingleResult<int32_t>("SELECT CAST(strField AS INT) + i32Field FROM TestType", 
-        common::LexicalCast<int32_t>(in.strField) + in.i32Field);
-
-    CheckSingleResult<std::string>("SELECT CAST(i32Field AS VARCHAR) || strField FROM TestType",
-        common::LexicalCast<std::string>(in.i32Field) + in.strField);
-}
-
-BOOST_AUTO_TEST_CASE(TestCoalesce)
-{
-    CheckSingleResult<std::string>("SELECT COALESCE('One', 'Two', 'Three')", "One");
-    CheckSingleResult<std::string>("SELECT COALESCE(NULL, 'Two', 'Three')", "Two");
-    CheckSingleResult<std::string>("SELECT COALESCE(NULL, 'Two', NULL)", "Two");
-    CheckSingleResult<std::string>("SELECT COALESCE(NULL, NULL, 'Three')", "Three");
-}
-
-BOOST_AUTO_TEST_CASE(TestNullif)
-{
-    TestType in;
-
-    in.strField = "SomeValue";
-
-    testCache.Put(1, in);
-
-    CheckSingleResult<std::string>("SELECT NULLIF(strField, 'blablabla') FROM TestType", in.strField);
-    CheckSingleResult<std::string>("SELECT NULLIF(strField, 'SomeValue') FROM TestType", "");
-}
-
-BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/28917034/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp
new file mode 100644
index 0000000..eb9b6ea
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_value_expressions_test.cpp
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+
+#ifndef _MSC_VER
+#   define BOOST_TEST_DYN_LINK
+#endif
+
+#include <boost/test/unit_test.hpp>
+
+#include "sql_test_suite_fixture.h"
+
+using namespace ignite;
+
+using namespace boost::unit_test;
+
+BOOST_FIXTURE_TEST_SUITE(SqlValueExpressionTestSuite, ignite::SqlTestSuiteFixture)
+
+BOOST_AUTO_TEST_CASE(TestCase)
+{
+    TestType in;
+
+    in.i32Field = 82;
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int32_t>(
+        "SELECT "
+            "CASE i32Field WHEN 82 "
+                "THEN (i32Field / 2) "
+                "ELSE (i32Field / 3) "
+            "END "
+        "FROM TestType", in.i32Field / 2);
+
+    
+    CheckSingleResult<int32_t>(
+        "SELECT "
+            "CASE i32Field WHEN 22 "
+                "THEN (i32Field / 2) "
+                "ELSE (i32Field / 3) "
+            "END "
+        "FROM TestType", in.i32Field / 3);;
+}
+
+BOOST_AUTO_TEST_CASE(TestCast)
+{
+    TestType in;
+
+    in.i32Field = 12345;
+    in.strField = "54321";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<int32_t>("SELECT CAST(strField AS INT) + i32Field FROM TestType", 
+        common::LexicalCast<int32_t>(in.strField) + in.i32Field);
+
+    CheckSingleResult<std::string>("SELECT CAST(i32Field AS VARCHAR) || strField FROM TestType",
+        common::LexicalCast<std::string>(in.i32Field) + in.strField);
+}
+
+BOOST_AUTO_TEST_CASE(TestCoalesce)
+{
+    CheckSingleResult<std::string>("SELECT COALESCE('One', 'Two', 'Three')", "One");
+    CheckSingleResult<std::string>("SELECT COALESCE(NULL, 'Two', 'Three')", "Two");
+    CheckSingleResult<std::string>("SELECT COALESCE(NULL, 'Two', NULL)", "Two");
+    CheckSingleResult<std::string>("SELECT COALESCE(NULL, NULL, 'Three')", "Three");
+}
+
+BOOST_AUTO_TEST_CASE(TestNullif)
+{
+    TestType in;
+
+    in.strField = "SomeValue";
+
+    testCache.Put(1, in);
+
+    CheckSingleResult<std::string>("SELECT NULLIF(strField, 'blablabla') FROM TestType", in.strField);
+    CheckSingleResult<std::string>("SELECT NULLIF(strField, 'SomeValue') FROM TestType", "");
+}
+
+BOOST_AUTO_TEST_SUITE_END()


[05/24] ignite git commit: Fixed issues on node stop: - in service processor need guard depExe access with busyLock - do not error log IO errors in ClientImpl on stop

Posted by sb...@apache.org.
Fixed issues on node stop:
- in service processor need guard depExe access with busyLock
- do not error log IO errors in ClientImpl on stop


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

Branch: refs/heads/master
Commit: d6449ffbc65acda6a2cf4484608188367837dd17
Parents: 13dfcbe
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 24 18:34:02 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 24 18:34:02 2016 +0300

----------------------------------------------------------------------
 .../processors/service/GridServiceProcessor.java  | 18 +++++++++++++-----
 .../ignite/spi/discovery/tcp/ClientImpl.java      |  7 ++++++-
 2 files changed, 19 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d6449ffb/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 01b7302..7b76c48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1355,11 +1355,19 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     private class ServiceEntriesListener implements CacheEntryUpdatedListener<Object, Object> {
         /** {@inheritDoc} */
         @Override public void onUpdated(final Iterable<CacheEntryEvent<?, ?>> deps) {
-            depExe.submit(new BusyRunnable() {
-                @Override public void run0() {
-                    onSystemCacheUpdated(deps);
-                }
-            });
+            if (!busyLock.enterBusy())
+                return;
+
+            try {
+                depExe.submit(new BusyRunnable() {
+                    @Override public void run0() {
+                        onSystemCacheUpdated(deps);
+                    }
+                });
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d6449ffb/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 9821134..1e71888 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -1115,7 +1115,12 @@ class ClientImpl extends TcpDiscoveryImpl {
                     }
                 }
                 catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to send message: " + msg, e);
+                    if (spi.getSpiContext().isStopping()) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to send message, node is stopping [msg=" + msg + ", err=" + e + ']');
+                    }
+                    else
+                        U.error(log, "Failed to send message: " + msg, e);
 
                     msg = null;
                 }


[10/24] ignite git commit: IGNITE-3390: ODBC: Added DSN configuration dialog for Windows. This closes #881.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/src/config/configuration.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/configuration.cpp b/modules/platforms/cpp/odbc/src/config/configuration.cpp
index 24c2bdf..dbe40bd 100644
--- a/modules/platforms/cpp/odbc/src/config/configuration.cpp
+++ b/modules/platforms/cpp/odbc/src/config/configuration.cpp
@@ -122,7 +122,7 @@ namespace ignite
                 return connect_string_buffer.str();
             }
 
-            void Configuration::FillFromConfigAttributes(const char * attributes)
+            void Configuration::FillFromConfigAttributes(const char* attributes)
             {
                 // Initializing map.
                 arguments.clear();
@@ -150,6 +150,11 @@ namespace ignite
                 }
             }
 
+            void Configuration::SetTcpPort(uint16_t port)
+            {
+                arguments[Key::port] = common::LexicalCast<std::string>(port);
+            }
+
             ProtocolVersion Configuration::GetProtocolVersion() const
             {
                 ArgumentMap::const_iterator it = arguments.find(Key::protocolVersion);
@@ -160,6 +165,11 @@ namespace ignite
                 return DefaultValue::protocolVersion;
             }
 
+            void Configuration::SetProtocolVersion(const std::string& version)
+            {
+                arguments[Key::protocolVersion] = version;
+            }
+
             const std::string& Configuration::GetStringValue(const std::string& key, const std::string& dflt) const
             {
                 ArgumentMap::const_iterator it = arguments.find(common::ToLower(key));
@@ -206,6 +216,11 @@ namespace ignite
                 return dflt;
             }
 
+            void Configuration::SetBoolValue(const std::string& key, bool val)
+            {
+                arguments[key] = val ? "true" : "false";
+            }
+
             void Configuration::ParseAttributeList(const char * str, size_t len, char delimeter, ArgumentMap & args)
             {
                 std::string connect_str(str, len);

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index 4315698..0fd9513 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -117,7 +117,7 @@ namespace ignite
                 return SQL_RESULT_ERROR;
             }
 
-            connected = socket.Connect(cfg.GetHost().c_str(), cfg.GetPort());
+            connected = socket.Connect(cfg.GetHost().c_str(), cfg.GetTcpPort());
 
             if (!connected)
             {

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
index 0fdfbc8..8553ee4 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
@@ -171,7 +171,7 @@ namespace ignite
                 return ORIGIN_ISO_9075;
             }
 
-            const std::string& DiagnosticRecord::GetMessage() const
+            const std::string& DiagnosticRecord::GetMessageText() const
             {
                 return message;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp
index 90c0a4f..99ef292 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record_storage.cpp
@@ -190,7 +190,7 @@ namespace ignite
 
                     case IGNITE_SQL_DIAG_STATUS_MESSAGE_TEXT:
                     {
-                        buffer.PutString(record.GetMessage());
+                        buffer.PutString(record.GetMessageText());
 
                         return SQL_RESULT_SUCCESS;
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/src/dsn_config.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/dsn_config.cpp b/modules/platforms/cpp/odbc/src/dsn_config.cpp
new file mode 100644
index 0000000..a304567
--- /dev/null
+++ b/modules/platforms/cpp/odbc/src/dsn_config.cpp
@@ -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.
+ */
+
+#include <set>
+
+#include "ignite/odbc/utility.h"
+#include "ignite/odbc/system/odbc_constants.h"
+
+#include "ignite/odbc/dsn_config.h"
+
+using ignite::odbc::config::Configuration;
+
+#define BUFFER_SIZE 1024
+#define CONFIG_FILE "ODBC.INI"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        void ThrowLastSetupError()
+        {
+            DWORD code;
+            char msg[BUFFER_SIZE];
+
+            SQLInstallerError(1, &code, msg, sizeof(msg), NULL);
+
+            std::stringstream buf;
+
+            buf << "Message: \"" << msg << "\", Code: " << code;
+
+            throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, buf.str().c_str());
+        }
+
+        void WriteDsnString(const char* dsn, const char* key, const char* value)
+        {
+            if (!SQLWritePrivateProfileString(dsn, key, value, CONFIG_FILE))
+                ThrowLastSetupError();
+        }
+
+        std::string ReadDsnString(const char* dsn, const std::string& key, const char* dflt)
+        {
+            char buf[BUFFER_SIZE];
+
+            memset(buf, 0, sizeof(buf));
+
+            SQLGetPrivateProfileString(dsn, key.c_str(), dflt, buf, sizeof(buf), CONFIG_FILE);
+
+            return std::string(buf);
+        }
+
+        int ReadDsnInt(const char* dsn, const std::string& key, int dflt)
+        {
+            char buf[BUFFER_SIZE];
+
+            memset(buf, 0, sizeof(buf));
+
+            std::string dflt0 = common::LexicalCast<std::string>(dflt);
+
+            SQLGetPrivateProfileString(dsn, key.c_str(), dflt0.c_str(), buf, sizeof(buf), CONFIG_FILE);
+
+            return common::LexicalCast<int, std::string>(buf);
+        }
+
+        bool ReadDsnBool(const char* dsn, const std::string& key, bool dflt)
+        {
+            char buf[BUFFER_SIZE];
+
+            memset(buf, 0, sizeof(buf));
+
+            std::string dflt0 = dflt ? "true" : "false";
+
+            SQLGetPrivateProfileString(dsn, key.c_str(), dflt0.c_str(), buf, sizeof(buf), CONFIG_FILE);
+
+            return std::string(buf) == "true";
+        }
+
+        void ReadDsnConfiguration(const char* dsn, Configuration& config)
+        {
+            std::string address = ReadDsnString(dsn, Configuration::Key::address, config.GetAddress().c_str());
+            std::string server = ReadDsnString(dsn, Configuration::Key::server, config.GetHost().c_str());
+            uint16_t port = ReadDsnInt(dsn, Configuration::Key::port, config.GetTcpPort());
+            std::string cache = ReadDsnString(dsn, Configuration::Key::cache, config.GetCache().c_str());
+            bool distributedJoins = ReadDsnBool(dsn, Configuration::Key::distributedJoins, config.IsDistributedJoins());
+            bool enforceJoinOrder = ReadDsnBool(dsn, Configuration::Key::enforceJoinOrder, config.IsEnforceJoinOrder());
+            std::string version = ReadDsnString(dsn, Configuration::Key::protocolVersion,
+                config.GetProtocolVersion().ToString().c_str());
+
+            LOG_MSG("%d\n", __LINE__);
+
+            config.SetAddress(address);
+            config.SetHost(server);
+            config.SetTcpPort(port);
+            config.SetCache(cache);
+            config.SetDistributedJoins(distributedJoins);
+            config.SetEnforceJoinOrder(enforceJoinOrder);
+            config.SetProtocolVersion(version);
+
+            LOG_MSG("%d\n", __LINE__);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/src/entry_points.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/entry_points.cpp b/modules/platforms/cpp/odbc/src/entry_points.cpp
index c8e78a5..f6195e1 100644
--- a/modules/platforms/cpp/odbc/src/entry_points.cpp
+++ b/modules/platforms/cpp/odbc/src/entry_points.cpp
@@ -19,14 +19,6 @@
 
 #include "ignite/odbc/utility.h"
 
-BOOL INSTAPI ConfigDSN(HWND     hwndParent,
-                       WORD     req,
-                       LPCSTR   driver,
-                       LPCSTR   attributes)
-{
-    return ignite::ConfigDSN(hwndParent, req, driver, attributes);
-}
-
 SQLRETURN SQL_API SQLGetInfo(SQLHDBC        conn,
                              SQLUSMALLINT   infoType,
                              SQLPOINTER     infoValue,

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/src/odbc.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp
index 9b4179e..fd35cba 100644
--- a/modules/platforms/cpp/odbc/src/odbc.cpp
+++ b/modules/platforms/cpp/odbc/src/odbc.cpp
@@ -28,70 +28,11 @@
 #include "ignite/odbc/environment.h"
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/statement.h"
+#include "ignite/odbc/dsn_config.h"
 #include "ignite/odbc.h"
 
 namespace ignite
 {
-
-    BOOL ConfigDSN(HWND     hwndParent,
-                   WORD     req,
-                   LPCSTR   driver,
-                   LPCSTR   attributes)
-    {
-        LOG_MSG("ConfigDSN called\n");
-
-        ignite::odbc::config::Configuration config;
-
-        try
-        {
-            config.FillFromConfigAttributes(attributes);
-        }
-        catch (IgniteError& e)
-        {
-            SQLPostInstallerError(e.GetCode(), e.GetText());
-
-            return SQL_FALSE;
-        }
-
-        if (!SQLValidDSN(config.GetDsn().c_str()))
-            return SQL_FALSE;
-
-        LOG_MSG("Driver: %s\n", driver);
-        LOG_MSG("Attributes: %s\n", attributes);
-
-        LOG_MSG("DSN: %s\n", config.GetDsn().c_str());
-
-        switch (req)
-        {
-            case ODBC_ADD_DSN:
-            {
-                LOG_MSG("ODBC_ADD_DSN\n");
-
-                return SQLWriteDSNToIni(config.GetDsn().c_str(), driver);
-            }
-
-            case ODBC_CONFIG_DSN:
-            {
-                LOG_MSG("ODBC_CONFIG_DSN\n");
-                break;
-            }
-
-            case ODBC_REMOVE_DSN:
-            {
-                LOG_MSG("ODBC_REMOVE_DSN\n");
-
-                return SQLRemoveDSNFromIni(config.GetDsn().c_str());
-            }
-
-            default:
-            {
-                return SQL_FALSE;
-            }
-        }
-
-        return SQL_TRUE;
-    }
-
     SQLRETURN SQLGetInfo(SQLHDBC        conn,
                          SQLUSMALLINT   infoType,
                          SQLPOINTER     infoValue,
@@ -315,10 +256,10 @@ namespace ignite
                                SQLSMALLINT* outConnectionStringLen,
                                SQLUSMALLINT driverCompletion)
     {
-        using ignite::odbc::Connection;
-        using ignite::odbc::diagnostic::DiagnosticRecordStorage;
-        using ignite::utility::SqlStringToString;
-        using ignite::utility::CopyStringToBuffer;
+        using odbc::Connection;
+        using odbc::diagnostic::DiagnosticRecordStorage;
+        using utility::SqlStringToString;
+        using utility::CopyStringToBuffer;
 
         UNREFERENCED_PARAMETER(windowHandle);
 
@@ -332,7 +273,16 @@ namespace ignite
 
         std::string connectStr = SqlStringToString(inConnectionString, inConnectionStringLen);
 
-        connection->Establish(connectStr);
+        odbc::config::Configuration config;
+
+        config.FillFromConnectString(connectStr);
+
+        std::string dsn = config.GetDsn();
+
+        if (!dsn.empty())
+            odbc::ReadDsnConfiguration(dsn.c_str(), config);
+
+        connection->Establish(config);
 
         const DiagnosticRecordStorage& diag = connection->GetDiagnosticRecords();
 
@@ -372,9 +322,11 @@ namespace ignite
         if (!connection)
             return SQL_INVALID_HANDLE;
 
-        //std::string server = SqlStringToString(serverName, serverNameLen);
+        odbc::config::Configuration config;
+
+        std::string dsn = SqlStringToString(serverName, serverNameLen);
 
-        Configuration config;
+        odbc::ReadDsnConfiguration(dsn.c_str(), config);
 
         connection->Establish(config);
 
@@ -1175,7 +1127,7 @@ namespace ignite
         SqlLen outResLen;
         ApplicationDataBuffer outBuffer(IGNITE_ODBC_C_TYPE_CHAR, msgBuffer, msgBufferLen, &outResLen);
 
-        outBuffer.PutString(record.GetMessage());
+        outBuffer.PutString(record.GetMessageText());
 
         *msgLen = static_cast<SQLSMALLINT>(outResLen);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/60afa372/modules/platforms/cpp/odbc/src/protocol_version.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/protocol_version.cpp b/modules/platforms/cpp/odbc/src/protocol_version.cpp
index c65099d..ebd3b6a 100644
--- a/modules/platforms/cpp/odbc/src/protocol_version.cpp
+++ b/modules/platforms/cpp/odbc/src/protocol_version.cpp
@@ -14,12 +14,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
-#include "ignite/odbc/protocol_version.h"
 #include <ignite/common/concurrent.h>
 #include <ignite/common/utils.h>
 #include <ignite/ignite_error.h>
 
+#include "ignite/odbc/protocol_version.h"
+#include "ignite/odbc/utility.h"
+
 namespace ignite
 {
     namespace odbc
@@ -50,10 +51,15 @@ namespace ignite
             // No-op.
         }
 
-        int64_t ProtocolVersion::MakeVersion(uint16_t major, uint16_t minor, uint16_t maintenance)
+        int64_t ProtocolVersion::MakeVersion(uint16_t major, uint16_t minor, uint16_t revision)
         {
             const static int64_t MASK = 0x000000000000FFFFLL;
-            return ((major & MASK) << 48) | ((minor & MASK) << 32) | ((maintenance & MASK) << 16);
+            return ((major & MASK) << 48) | ((minor & MASK) << 32) | ((revision & MASK) << 16);
+        }
+
+        const ProtocolVersion::StringToVersionMap& ProtocolVersion::GetMap()
+        {
+            return stringToVersionMap;
         }
 
         const ProtocolVersion& ProtocolVersion::GetCurrent()
@@ -68,8 +74,8 @@ namespace ignite
             if (it == stringToVersionMap.end())
             {
                 throw IgniteError(IgniteError::IGNITE_ERR_GENERIC,
-                    "Invalid version format. Valid format is X.Y.Z, where X, Y and Z are major, "
-                    "minor and maintenance versions of Ignite since which protocol is introduced.");
+                    "Invalid version format. Valid format is X.Y.Z, where X, Y and Z are major "
+                    "and minor versions and revision of Ignite since which protocol is introduced.");
             }
 
             return it->second;
@@ -100,6 +106,11 @@ namespace ignite
             return *this == VERSION_UNKNOWN;
         }
 
+        bool ProtocolVersion::IsDistributedJoinsSupported() const
+        {
+            return *this >= VERSION_1_8_0;
+        }
+
         bool operator==(const ProtocolVersion& val1, const ProtocolVersion& val2)
         {
             return val1.val == val2.val;