You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2017/01/19 10:03:22 UTC

ignite git commit: IGNITE-3837: ODBC: Support for CONVERT function escape sequence

Repository: ignite
Updated Branches:
  refs/heads/master 80bcf2739 -> 664dc88e9


IGNITE-3837: ODBC: Support for CONVERT function escape sequence

This closes #1422


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

Branch: refs/heads/master
Commit: 664dc88e94a8ac0506c2762b7e964f51c72d7459
Parents: 80bcf27
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Thu Jan 19 13:03:10 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Jan 19 13:03:10 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/odbc/IgniteTypes.java   |  69 +++++
 .../internal/processors/odbc/OdbcTypes.java     | 131 ++++++++++
 .../internal/processors/odbc/OdbcUtils.java     |  85 ++++++
 .../processors/odbc/escape/OdbcEscapeUtils.java |  52 +++-
 .../odbc/OdbcEscapeSequenceSelfTest.java        | 131 ++++++++++
 .../cpp/binary/src/impl/binary/binary_utils.cpp |   6 +-
 .../cpp/common/include/ignite/common/utils.h    |   8 +
 .../cpp/common/os/linux/src/common/utils.cpp    |  22 +-
 .../cpp/common/os/win/src/common/utils.cpp      |  14 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |   1 +
 .../odbc-test/include/sql_test_suite_fixture.h  |  13 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   1 +
 .../project/vs/odbc-test.vcxproj.filters        |   3 +
 .../cpp/odbc-test/src/api_robustness_test.cpp   |   2 +-
 .../src/sql_aggregate_functions_test.cpp        |   4 +-
 .../src/sql_esc_convert_function_test.cpp       | 160 ++++++++++++
 .../odbc-test/src/sql_test_suite_fixture.cpp    |  52 +++-
 .../cpp/odbc-test/src/sql_types_test.cpp        | 131 +++++++++-
 .../odbc/src/app/application_data_buffer.cpp    |  58 ++++-
 .../platforms/cpp/odbc/src/app/parameter.cpp    |   4 +-
 .../cpp/odbc/src/config/connection_info.cpp     | 260 +++++++++++++++++--
 21 files changed, 1168 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/IgniteTypes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/IgniteTypes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/IgniteTypes.java
new file mode 100644
index 0000000..1eea4e2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/IgniteTypes.java
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+/**
+ * Data type names usable in SQL queries
+ * after escape sequence transformation
+ */
+public class IgniteTypes {
+    /** Type name for 64-bit integer */
+    static final String BIGINT = "BIGINT";
+
+    /** Type name for byte array */
+    static final String BINARY = "BINARY";
+
+    /** Type name for boolean flag */
+    static final String BIT = "BIT";
+
+    /** Type name for unicode string */
+    static final String CHAR = "CHAR";
+
+    /** Type name for decimal number */
+    static final String DECIMAL = "DECIMAL";
+
+    /** Type name for unicode string */
+    static final String VARCHAR = "VARCHAR";
+
+    /** Type name for floating point number */
+    static final String DOUBLE = "DOUBLE";
+
+    /** Type name for single precision floating point number */
+    static final String REAL = "REAL";
+
+    /** Type name for universally unique identifier */
+    static final String UUID = "UUID";
+
+    /** Type name for 16-bit integer */
+    static final String SMALLINT = "SMALLINT";
+
+    /** Type name for 32-bit integer */
+    static final String INTEGER = "INTEGER";
+
+    /** Type name for 8-bit integer */
+    static final String TINYINT = "TINYINT";
+
+    /** Type name for date */
+    static final String DATE = "DATE";
+
+    /** Type name for time */
+    static final String TIME = "TIME";
+
+    /** Type name for timestamp */
+    static final String TIMESTAMP = "TIMESTAMP";
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTypes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTypes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTypes.java
new file mode 100644
index 0000000..70ac92b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTypes.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.odbc;
+
+/**
+ * Data type names that can occur within ODBC escape sequence
+ */
+public class OdbcTypes {
+    /** Type name for BIGINT */
+    static final String SQL_BIGINT = "SQL_BIGINT";
+
+    /** Type name for BINARY */
+    static final String SQL_BINARY = "SQL_BINARY";
+
+    /** Type name for LONGVARBINARY */
+    static final String SQL_LONGVARBINARY = "SQL_LONGVARBINARY";
+
+    /** Type name for VARBINARY */
+    static final String SQL_VARBINARY = "SQL_VARBINARY";
+
+    /** Type name for BIT */
+    static final String SQL_BIT = "SQL_BIT";
+
+    /** Type name for CHAR */
+    static final String SQL_CHAR = "SQL_CHAR";
+
+    /** Type name for DECIMAL */
+    static final String SQL_DECIMAL = "SQL_DECIMAL";
+
+    /** Type name for NUMERIC */
+    static final String SQL_NUMERIC = "SQL_NUMERIC";
+
+    /** Type name for LONGVARCHAR */
+    static final String SQL_LONGVARCHAR = "SQL_LONGVARCHAR";
+
+    /** Type name for VARCHAR */
+    static final String SQL_VARCHAR = "SQL_VARCHAR";
+
+    /** Type name for WCHAR */
+    static final String SQL_WCHAR = "SQL_WCHAR";
+
+    /** Type name for WLONGVARCHAR */
+    static final String SQL_WLONGVARCHAR = "SQL_WLONGVARCHAR";
+
+    /** Type name for WVARCHAR */
+    static final String SQL_WVARCHAR = "SQL_WVARCHAR";
+
+    /** Type name for DOUBLE */
+    static final String SQL_DOUBLE = "SQL_DOUBLE";
+
+    /** Type name for FLOAT */
+    static final String SQL_FLOAT = "SQL_FLOAT";
+
+    /** Type name for REAL */
+    static final String SQL_REAL = "SQL_REAL";
+
+    /** Type name for GUID */
+    static final String SQL_GUID = "SQL_GUID";
+
+    /** Type name for SMALLINT */
+    static final String SQL_SMALLINT = "SQL_SMALLINT";
+
+    /** Type name for INTEGER */
+    static final String SQL_INTEGER = "SQL_INTEGER";
+
+    /** Type name for DATE */
+    static final String SQL_DATE = "SQL_DATE";
+
+    /** Type name for TIME */
+    static final String SQL_TIME = "SQL_TIME";
+
+    /** Type name for TIMESTAMP */
+    static final String SQL_TIMESTAMP = "SQL_TIMESTAMP";
+
+    /** Type name for TINYINT */
+    static final String SQL_TINYINT = "SQL_TINYINT";
+
+    /** Type name for INTERVAL_SECOND */
+    static final String SQL_INTERVAL_SECOND = "SQL_INTERVAL_SECOND";
+
+    /** Type name for INTERVAL_MINUTE */
+    static final String SQL_INTERVAL_MINUTE = "SQL_INTERVAL_MINUTE";
+
+    /** Type name for INTERVAL_HOUR */
+    static final String SQL_INTERVAL_HOUR = "SQL_INTERVAL_HOUR";
+
+    /** Type name for INTERVAL_DAY */
+    static final String SQL_INTERVAL_DAY = "SQL_INTERVAL_DAY";
+
+    /** Type name for INTERVAL_MONTH */
+    static final String SQL_INTERVAL_MONTH = "SQL_INTERVAL_MONTH";
+
+    /** Type name for INTERVAL_YEAR */
+    static final String SQL_INTERVAL_YEAR = "SQL_INTERVAL_YEAR";
+
+    /** Type name for INTERVAL_YEAR_TO_MONTH */
+    static final String SQL_INTERVAL_YEAR_TO_MONTH = "SQL_INTERVAL_YEAR_TO_MONTH";
+
+    /** Type name for INTERVAL_HOUR_TO_MINUTE */
+    static final String SQL_INTERVAL_HOUR_TO_MINUTE = "SQL_INTERVAL_HOUR_TO_MINUTE";
+
+    /** Type name for INTERVAL_HOUR_TO_SECOND */
+    static final String SQL_INTERVAL_HOUR_TO_SECOND = "SQL_INTERVAL_HOUR_TO_SECOND";
+
+    /** Type name for INTERVAL_MINUTE_TO_SECOND */
+    static final String SQL_INTERVAL_MINUTE_TO_SECOND = "SQL_INTERVAL_MINUTE_TO_SECOND";
+
+    /** Type name for INTERVAL_DAY_TO_HOUR */
+    static final String SQL_INTERVAL_DAY_TO_HOUR = "SQL_INTERVAL_DAY_TO_HOUR";
+
+    /** Type name for INTERVAL_DAY_TO_MINUTE */
+    static final String SQL_INTERVAL_DAY_TO_MINUTE = "SQL_INTERVAL_DAY_TO_MINUTE";
+
+    /** Type name for INTERVAL_DAY_TO_SECOND */
+    static final String SQL_INTERVAL_DAY_TO_SECOND  = "SQL_INTERVAL_DAY_TO_SECOND";
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
index 3903562..d851d13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcUtils.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.odbc;
 
+import org.apache.ignite.IgniteException;
+
 /**
  * Various ODBC utility methods.
  */
@@ -53,4 +55,87 @@ public class OdbcUtils {
     private OdbcUtils() {
         // No-op.
     }
+
+    /**
+     * Lookup Ignite data type corresponding to specific ODBC data type
+     *
+     * @param odbcDataType ODBC data type identifier
+     * @return Ignite data type name
+     */
+    public static String getIgniteTypeFromOdbcType(String odbcDataType) {
+        assert odbcDataType != null;
+        switch (odbcDataType.toUpperCase()) {
+            case OdbcTypes.SQL_BIGINT:
+                return IgniteTypes.BIGINT;
+
+            case OdbcTypes.SQL_BINARY:
+            case OdbcTypes.SQL_LONGVARBINARY:
+            case OdbcTypes.SQL_VARBINARY:
+                return IgniteTypes.BINARY;
+
+            case OdbcTypes.SQL_BIT:
+                return IgniteTypes.BIT;
+
+            case OdbcTypes.SQL_CHAR:
+                return IgniteTypes.CHAR;
+
+            case OdbcTypes.SQL_DECIMAL:
+            case OdbcTypes.SQL_NUMERIC:
+                return IgniteTypes.DECIMAL;
+
+            case OdbcTypes.SQL_LONGVARCHAR:
+            case OdbcTypes.SQL_VARCHAR:
+            case OdbcTypes.SQL_WCHAR:
+            case OdbcTypes.SQL_WLONGVARCHAR:
+            case OdbcTypes.SQL_WVARCHAR:
+                return IgniteTypes.VARCHAR;
+
+            case OdbcTypes.SQL_DOUBLE:
+            case OdbcTypes.SQL_FLOAT:
+                return IgniteTypes.DOUBLE;
+
+            case OdbcTypes.SQL_REAL:
+                return IgniteTypes.REAL;
+
+            case OdbcTypes.SQL_GUID:
+                return IgniteTypes.UUID;
+
+            case OdbcTypes.SQL_SMALLINT:
+                return IgniteTypes.SMALLINT;
+
+            case OdbcTypes.SQL_INTEGER:
+                return IgniteTypes.INTEGER;
+
+            case OdbcTypes.SQL_DATE:
+                return IgniteTypes.DATE;
+
+            case OdbcTypes.SQL_TIME:
+                return IgniteTypes.TIME;
+
+            case OdbcTypes.SQL_TIMESTAMP:
+                return IgniteTypes.TIMESTAMP;
+
+            case OdbcTypes.SQL_TINYINT:
+                return IgniteTypes.TINYINT;
+
+            //No support for interval types
+            case OdbcTypes.SQL_INTERVAL_SECOND:
+            case OdbcTypes.SQL_INTERVAL_MINUTE:
+            case OdbcTypes.SQL_INTERVAL_HOUR:
+            case OdbcTypes.SQL_INTERVAL_DAY:
+            case OdbcTypes.SQL_INTERVAL_MONTH:
+            case OdbcTypes.SQL_INTERVAL_YEAR:
+            case OdbcTypes.SQL_INTERVAL_YEAR_TO_MONTH:
+            case OdbcTypes.SQL_INTERVAL_HOUR_TO_MINUTE:
+            case OdbcTypes.SQL_INTERVAL_HOUR_TO_SECOND:
+            case OdbcTypes.SQL_INTERVAL_MINUTE_TO_SECOND:
+            case OdbcTypes.SQL_INTERVAL_DAY_TO_HOUR:
+            case OdbcTypes.SQL_INTERVAL_DAY_TO_MINUTE:
+            case OdbcTypes.SQL_INTERVAL_DAY_TO_SECOND:
+                throw new IgniteException("Unsupported ODBC data type '" + odbcDataType + "'");
+
+            default:
+                throw new IgniteException("Invalid ODBC data type '" + odbcDataType + "'");
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 bbf19c7..01f32d1 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
@@ -18,8 +18,9 @@
 package org.apache.ignite.internal.processors.odbc.escape;
 
 import org.apache.ignite.IgniteException;
-
+import org.apache.ignite.internal.processors.odbc.OdbcUtils;
 import java.util.LinkedList;
+import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 /**
@@ -40,6 +41,11 @@ public class OdbcEscapeUtils {
     private static final Pattern GUID_PATTERN =
         Pattern.compile("^'\\p{XDigit}{8}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{4}-\\p{XDigit}{12}'$");
 
+    /** CONVERT function data type parameter pattern: last parameter, after comma */
+    private static final Pattern CONVERT_TYPE_PATTERN =
+        Pattern.compile(",\\s*(SQL_[\\w_]+)\\s*(?:\\(\\s*\\d+\\s*(?:,\\s*\\d+\\s*)?\\))?\\s*\\)\\s*$",
+                        Pattern.CASE_INSENSITIVE | Pattern.UNICODE_CASE);
+
     /**
      * Parse escape sequence.
      *
@@ -248,7 +254,7 @@ public class OdbcEscapeUtils {
 
         switch (token.type()) {
             case SCALAR_FUNCTION:
-                return parseExpression(text, startPos0, len0);
+                return parseScalarFunctionExpression(text, startPos0, len0);
 
             case GUID: {
                 String res = parseExpression(text, startPos0, len0, token.type(), GUID_PATTERN);
@@ -326,6 +332,48 @@ public class OdbcEscapeUtils {
     }
 
     /**
+     * Parse scalar function expression.
+     *
+     * @param text Text.
+     * @param startPos Start position.
+     * @param len Length.
+     * @return Parsed expression.
+     */
+    private static String parseScalarFunctionExpression(String text, int startPos, int len) {
+        int pos = startPos;
+        int endPos = startPos + len;
+        final String errPrefix = "Malformed scalar function escape sequence.";
+
+        while ((++pos < endPos) && Character.isWhitespace(text.charAt(pos)));
+        if (pos == endPos)
+            throw new IgniteException(errPrefix + " Expected function name.");
+
+        int funcNamePos = pos;
+        while ((++pos < endPos) && Character.isAlphabetic(text.charAt(pos)));
+        if (pos == endPos)
+            throw new IgniteException(errPrefix + " Expected function parameter list: " +
+                                      substring(text, startPos, len));
+
+        String funcName = text.substring(funcNamePos, pos);
+
+        switch (funcName.toUpperCase()) {
+            case "CONVERT": {
+                Matcher matcher = CONVERT_TYPE_PATTERN.matcher(text.substring(startPos, endPos));
+
+                if (!matcher.find())
+                    throw new IgniteException(errPrefix + " Invalid arguments :" +
+                                              substring(text, startPos, len));
+
+                return (text.substring(startPos, startPos + matcher.start(1)) +
+                        OdbcUtils.getIgniteTypeFromOdbcType(matcher.group(1)) +
+                        text.substring(startPos + matcher.end(1), startPos + len)).trim();
+            }
+            default:
+                return substring(text, startPos, len).trim();
+        }
+    }
+
+    /**
      * Append nested results.
      *
      * @param text Original text.

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 5303c6e..ecb6c2d 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
@@ -74,6 +74,137 @@ public class OdbcEscapeSequenceSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test escape sequence for explicit data type conversion
+     */
+    public void testConvertFunction() throws Exception {
+        check(
+         "CONVERT ( CURDATE(), CHAR )",
+         "{ fn CONVERT ( { fn CURDATE() }, SQL_CHAR ) }"
+        );
+
+        check(
+         "conVerT ( some_expression('one', 'two') , DECIMAL ( 5 , 2 ) )",
+         "{ fn conVerT ( some_expression('one', 'two') , SQL_DECIMAL ( 5 , 2 ) ) }"
+        );
+
+        check(
+         "convert(field,CHAR)",
+         "{fn convert(field,sql_char)}"
+        );
+
+        check(
+         "convert(field,BIGINT)",
+         "{fn convert(field,sql_bigint)}"
+        );
+
+        check(
+         "convert(field,BINARY)",
+         "{fn convert(field,sql_binary)}" // also sql_varbinary,sql_longvarbinary
+        );
+
+        check(
+         "convert(field,BIT)",
+         "{fn convert(field,sql_bit)}"
+        );
+
+        check(
+         "convert(field,CHAR(100))",
+         "{fn convert(field,sql_char(100))}"
+        );
+
+        check(
+         "convert(field,DECIMAL(5,2))",
+         "{fn convert(field,sql_decimal(5,2))}" // also sql_numeric
+        );
+
+        check(
+         "convert(field,VARCHAR(100))",
+         "{fn convert(field,sql_varchar(100))}" // also sql_longvarchar,sql_wchar,sql_wlongvarchar,sql_wvarchar
+        );
+
+        check(
+         "convert(field,DOUBLE)",
+         "{fn convert(field,sql_double)}" // also sql_float
+        );
+
+        check(
+         "convert(field,REAL)",
+         "{fn convert(field,sql_real)}"
+        );
+
+        check(
+         "convert(field,UUID)",
+         "{fn convert(field,sql_guid)}"
+        );
+
+        check(
+         "convert(field,SMALLINT)",
+         "{fn convert(field,sql_smallint)}"
+        );
+
+        check(
+         "convert(field,INTEGER)",
+         "{fn convert(field,sql_integer)}"
+        );
+
+        check(
+         "convert(field,DATE)",
+         "{fn convert(field,sql_date)}"
+        );
+
+        check(
+         "convert(field,TIME)",
+         "{fn convert(field,sql_time)}"
+        );
+
+        check(
+         "convert(field,TIMESTAMP)",
+         "{fn convert(field,sql_timestamp)}"
+        );
+
+        check(
+         "convert(field,TINYINT)",
+         "{fn convert(field,sql_tinyint)}"
+        );
+
+        //invalid odbc type
+        checkFail("{fn convert(field,char)}");
+
+        //no support for interval types
+        checkFail("{fn convert(field,sql_interval_second)}");
+        checkFail("{fn convert(field,sql_interval_minute)}");
+        checkFail("{fn convert(field,sql_interval_hour)}");
+        checkFail("{fn convert(field,sql_interval_day)}");
+        checkFail("{fn convert(field,sql_interval_month)}");
+        checkFail("{fn convert(field,sql_interval_year)}");
+        checkFail("{fn convert(field,sql_interval_year_to_month)}");
+        checkFail("{fn convert(field,sql_interval_hour_to_minute)}");
+        checkFail("{fn convert(field,sql_interval_hour_to_second)}");
+        checkFail("{fn convert(field,sql_interval_minute_to_second)}");
+        checkFail("{fn convert(field,sql_interval_day_to_hour)}");
+        checkFail("{fn convert(field,sql_interval_day_to_minute)}");
+        checkFail("{fn convert(field,sql_interval_day_to_second)}");
+
+        //failure: expected function name
+        checkFail("{fn    }");
+
+        //failure: expected function parameter list
+        checkFail("{fn convert}");
+
+        //failure: expected data type parameter for convert function
+        checkFail("{fn convert ( justoneparam ) }");
+
+        //failure: empty precision/scale
+        checkFail("{fn convert ( justoneparam, sql_decimal( ) }");
+
+        //failure: empty precision/scale
+        checkFail("{fn convert ( justoneparam, sql_decimal(not_a_number) }");
+
+        //failure: missing scale after comma
+        checkFail("{fn convert ( justoneparam, sql_decimal(10,) }");
+    }
+
+    /**
      * Test simple nested escape sequences. Depth = 2.
      */
     public void testNestedFunction() throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp
index 1a1946c..22738ef 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_utils.cpp
@@ -220,13 +220,13 @@ namespace ignite
                 int64_t milliseconds = stream->ReadInt64();
                 int32_t nanoseconds = stream->ReadInt32();
 
-                return Timestamp(milliseconds / 1000, nanoseconds);
+                return Timestamp(milliseconds / 1000, (milliseconds % 1000) * 1000000 + nanoseconds);
             }
 
             void BinaryUtils::WriteTimestamp(interop::InteropOutputStream* stream, const Timestamp val)
             {
-                stream->WriteInt64(val.GetSeconds() * 1000);
-                stream->WriteInt32(val.GetSecondFraction());
+                stream->WriteInt64(val.GetSeconds() * 1000 + val.GetSecondFraction() / 1000000);
+                stream->WriteInt32(val.GetSecondFraction() % 1000000);
             }
 
             void BinaryUtils::WriteString(interop::InteropOutputStream* stream, const char* val, const int32_t len)

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 f4d2a9f..6ac3c17 100644
--- a/modules/platforms/cpp/common/include/ignite/common/utils.h
+++ b/modules/platforms/cpp/common/include/ignite/common/utils.h
@@ -204,6 +204,14 @@ namespace ignite
 
             return i == end;
         }
+
+        /**
+        * Converts 32-bit integer to big endian format
+        *
+        * @param value Input value
+        * @return Resulting value
+        */
+        IGNITE_IMPORT_EXPORT uint32_t ToBigEndian(uint32_t value);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/platforms/cpp/common/os/linux/src/common/utils.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/os/linux/src/common/utils.cpp b/modules/platforms/cpp/common/os/linux/src/common/utils.cpp
index e37a91c..0fa9231 100644
--- a/modules/platforms/cpp/common/os/linux/src/common/utils.cpp
+++ b/modules/platforms/cpp/common/os/linux/src/common/utils.cpp
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
- 
+
 #include <time.h>
 
 #include <sys/stat.h>
@@ -109,17 +109,17 @@ namespace ignite
         std::string GetEnv(const std::string& name, bool& found)
         {
             char* val = std::getenv(name.c_str());
-            
+
             if (val)
             {
                 found = true;
-                
+
                 return std::string(val);
             }
             else
             {
                 found = false;
-                
+
                 return std::string();
             }
         }
@@ -127,10 +127,22 @@ namespace ignite
         bool FileExists(const std::string& path)
         {
             struct stat s;
-            
+
             int res = stat(path.c_str(), &s);
 
             return res != -1;
         }
+
+        uint32_t ToBigEndian(uint32_t value)
+        {
+            // The answer is 42
+            static const int num = 42;
+            static const bool isLittleEndian = (*reinterpret_cast<const char*>(&num) == num);
+
+            if (isLittleEndian)
+                return ((value & 0xFF) << 24) | (((value >> 8) & 0xFF) << 16) | (((value >> 16) & 0xFF) << 8) | ((value >> 24) & 0xFF);
+
+            return value;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 77c90b8..c6afce2 100644
--- a/modules/platforms/cpp/common/os/win/src/common/utils.cpp
+++ b/modules/platforms/cpp/common/os/win/src/common/utils.cpp
@@ -115,7 +115,7 @@ namespace ignite
 
                 return std::string(res0);
             }
-            else 
+            else
             {
                 found = false;
 
@@ -138,5 +138,17 @@ namespace ignite
                 return true;
             }
         }
+
+        uint32_t ToBigEndian(uint32_t value)
+        {
+            // The answer is 42
+            static const int num = 42;
+            static const bool isLittleEndian = (*reinterpret_cast<const char*>(&num) == num);
+
+            if (isLittleEndian)
+                return ((value & 0xFF) << 24) | (((value >> 8) & 0xFF) << 16) | (((value >> 16) & 0xFF) << 8) | ((value >> 24) & 0xFF);
+
+            return value;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 c2f3fa2..56ae56a 100644
--- a/modules/platforms/cpp/odbc-test/Makefile.am
+++ b/modules/platforms/cpp/odbc-test/Makefile.am
@@ -69,6 +69,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_esc_convert_function_test.cpp \
     src/sql_operators_test.cpp \
     src/sql_value_expressions_test.cpp \
     src/sql_types_test.cpp \

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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
index 55353e5..0fa6ec9 100644
--- a/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
+++ b/modules/platforms/cpp/odbc-test/include/sql_test_suite_fixture.h
@@ -35,6 +35,7 @@
 
 #include "ignite/ignite.h"
 #include "ignite/ignition.h"
+#include "ignite/common/decimal.h"
 
 #include "test_type.h"
 
@@ -195,6 +196,18 @@ namespace ignite
 
     template<>
     void SqlTestSuiteFixture::CheckSingleResult<std::vector<int8_t> >(const char* request, const std::vector<int8_t>& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<ignite::common::Decimal>(const char* request, const ignite::common::Decimal& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<Date>(const char* request, const Date& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<SQL_TIME_STRUCT>(const char* request, const SQL_TIME_STRUCT& expected);
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<Timestamp>(const char* request, const Timestamp& expected);
 }
 
 #endif //_IGNITE_ODBC_TEST_SQL_TEST_SUIT_FIXTURE

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 6f85896..8740d5a 100644
--- a/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
+++ b/modules/platforms/cpp/odbc-test/project/vs/odbc-test.vcxproj
@@ -178,6 +178,7 @@
     <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_esc_convert_function_test.cpp" />
     <ClCompile Include="..\..\src\sql_types_test.cpp" />
     <ClCompile Include="..\..\src\sql_value_expressions_test.cpp" />
     <ClCompile Include="..\..\src\teamcity\teamcity_boost.cpp" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 bedceaa..a53cc47 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_esc_convert_function_test.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\sql_operators_test.cpp">
       <Filter>Code</Filter>
     </ClCompile>

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
index 0b6df93..5247129 100644
--- a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
@@ -354,7 +354,7 @@ BOOST_AUTO_TEST_CASE(TestSQLConnect)
     // Everyting is ok.
     SQLRETURN ret = SQLGetInfo(dbc, SQL_DRIVER_NAME, buffer, ODBC_BUFFER_SIZE, &resLen);
 
-    ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt);
+    ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc);
 
     // Resulting length is null.
     SQLGetInfo(dbc, SQL_DRIVER_NAME, buffer, ODBC_BUFFER_SIZE, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 3fa4d97..de1f5f8 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
@@ -114,7 +114,7 @@ BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgFloat)
     {
         testCache.Put(i, in[i]);
 
-        avg += in[i].i32Field;
+        avg += in[i].floatField;
     }
 
     avg /= in.size();
@@ -136,7 +136,7 @@ BOOST_AUTO_TEST_CASE(TestAggregateFunctionAvgFloatDistinct)
     {
         testCache.Put(i, in[i]);
 
-        avg += in[i].i32Field;
+        avg += in[i].floatField;
     }
 
     avg /= in.size();

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/platforms/cpp/odbc-test/src/sql_esc_convert_function_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/sql_esc_convert_function_test.cpp b/modules/platforms/cpp/odbc-test/src/sql_esc_convert_function_test.cpp
new file mode 100644
index 0000000..d9a14a9
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/src/sql_esc_convert_function_test.cpp
@@ -0,0 +1,160 @@
+/*
+ * 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 "ignite/common/decimal.h"
+#include "ignite/common/utils.h"
+#include "sql_test_suite_fixture.h"
+#include "test_utils.h"
+
+using namespace ignite;
+using namespace ignite::common;
+using namespace boost::unit_test;
+
+BOOST_FIXTURE_TEST_SUITE(SqlEscConvertFunctionTestSuite, ignite::SqlTestSuiteFixture)
+
+int CheckConnectionInfo(HDBC dbc, int infoType)
+{
+    SQLUINTEGER mask = 0;
+    SQLRETURN ret = SQLGetInfo(dbc, infoType, &mask, sizeof(mask), 0);
+    ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_DBC, dbc);
+    return mask;
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionGetInfo)
+{
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_FUNCTIONS) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_BIGINT) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_BINARY) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_BIT) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_CHAR) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_DATE) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_DECIMAL) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_DOUBLE) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_FLOAT) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_INTEGER) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_LONGVARCHAR) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_NUMERIC) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_REAL) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_SMALLINT) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_TIME) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_TIMESTAMP) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_TINYINT) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_VARBINARY) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_VARCHAR) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_LONGVARBINARY) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_WCHAR) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_WLONGVARCHAR) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_WVARCHAR) != 0);
+    BOOST_REQUIRE(CheckConnectionInfo(dbc, SQL_CONVERT_GUID) != 0);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionInt64)
+{
+    CheckSingleResult<int64_t>("SELECT {fn CONVERT(72623859790382856, SQL_BIGINT)}", 72623859790382856);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionInt32)
+{
+    CheckSingleResult<int32_t>("SELECT {fn CONVERT(1234567890, SQL_INTEGER)}", 1234567890);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionInt16)
+{
+    CheckSingleResult<int16_t>("SELECT {fn CONVERT(12345, SQL_SMALLINT)}", 12345);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionInt8)
+{
+    CheckSingleResult<int8_t>("SELECT {fn CONVERT(123, SQL_TINYINT)}", 123);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionByteArray)
+{
+    int32_t value = ToBigEndian(123456);
+
+    std::vector<int8_t> val;
+    val.assign((const int8_t*)&value, (const int8_t*)&value+sizeof(value));
+
+    CheckSingleResult<std::vector<int8_t> >("SELECT {fn CONVERT(123456, SQL_BINARY(4))}", val);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionBool)
+{
+    CheckSingleResult<bool>("SELECT {fn CONVERT(1, SQL_BIT)}", true);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionString)
+{
+    CheckSingleResult<std::string>("SELECT {fn CONVERT(123, SQL_VARCHAR(10))}", "123");
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionDecimal)
+{
+    CheckSingleResult<Decimal>("SELECT {fn CONVERT(-1.25, SQL_DECIMAL(5,2))}", Decimal("-1.25"));
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionDouble)
+{
+    CheckSingleResult<double>("SELECT CAST(-1.25 AS DOUBLE)", -1.25);
+    CheckSingleResult<double>("SELECT CONVERT(-1.25, DOUBLE)", -1.25);
+    CheckSingleResult<double>("SELECT {fn CONVERT(-1.25, SQL_DOUBLE)}", -1.25);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionFloat)
+{
+    CheckSingleResult<float>("SELECT CAST(-1.25 AS REAL)", -1.25);
+    CheckSingleResult<float>("SELECT CONVERT(-1.25, REAL)", -1.25);
+    CheckSingleResult<float>("SELECT CAST(-1.25 AS FLOAT4)", -1.25);
+    CheckSingleResult<float>("SELECT CONVERT(-1.25, FLOAT4)", -1.25);
+    CheckSingleResult<float>("SELECT {fn CONVERT(-1.25, SQL_REAL)}", -1.25);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionGuid)
+{
+    //no support for binding as GUID because we report v3.0 to DM, thus fallback to string binding for now
+    CheckSingleResult<std::string>("SELECT {fn CONVERT({guid '04cc382a-0b82-f520-08d0-07a0620c0004'}, SQL_GUID)}", "04cc382a-0b82-f520-08d0-07a0620c0004");
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionDate)
+{
+    using ignite::impl::binary::BinaryUtils;
+    Date date = BinaryUtils::MakeDateGmt(1983, 3, 14);
+    CheckSingleResult<Date>("SELECT {fn CONVERT('1983-03-14', SQL_DATE)}", date);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionTime)
+{
+    SQL_TIME_STRUCT exp;
+    exp.hour = 13;
+    exp.minute = 20;
+    exp.second = 15;
+    CheckSingleResult<SQL_TIME_STRUCT>("SELECT {fn CONVERT('13:20:15', SQL_TIME)}", exp);
+}
+
+BOOST_AUTO_TEST_CASE(TestEscConvertFunctionTimestamp)
+{
+    using ignite::impl::binary::BinaryUtils;
+    Timestamp ts = BinaryUtils::MakeTimestampGmt(1983, 3, 14, 13, 20, 15, 999999999);
+    CheckSingleResult<Timestamp>("SELECT {fn CONVERT('1983-03-14 13:20:15.999999999', SQL_TIMESTAMP)}", ts);
+}
+
+BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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
index b118ff8..1ecd26a 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_test_suite_fixture.cpp
@@ -177,7 +177,7 @@ namespace ignite
     template<>
     void SqlTestSuiteFixture::CheckSingleResult<float>(const char* request, const float& expected)
     {
-        SQLFLOAT res = 0;
+        SQLREAL res = 0;
 
         CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
 
@@ -257,7 +257,7 @@ namespace ignite
     template<>
     void SqlTestSuiteFixture::CheckSingleResult<float>(const char* request)
     {
-        SQLFLOAT res = 0;
+        SQLREAL res = 0;
 
         CheckSingleResult0(request, SQL_C_FLOAT, &res, 0, 0);
     }
@@ -302,4 +302,52 @@ namespace ignite
             BOOST_REQUIRE_EQUAL_COLLECTIONS(expected.begin(), expected.end(), actual.begin(), actual.end());
         }
     }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<ignite::common::Decimal>(const char* request, const ignite::common::Decimal& expected)
+    {
+        SQLCHAR res[ODBC_BUFFER_SIZE] = { 0 };
+        SQLLEN resLen = 0;
+
+        CheckSingleResult0(request, SQL_C_CHAR, res, ODBC_BUFFER_SIZE, &resLen);
+        ignite::common::Decimal actual(std::string(res, res + resLen));
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<Date>(const char* request, const Date& expected)
+    {
+        SQL_DATE_STRUCT res;
+
+        CheckSingleResult0(request, SQL_C_DATE, &res, 0, 0);
+
+        using ignite::impl::binary::BinaryUtils;
+        Date actual = BinaryUtils::MakeDateGmt(res.year, res.month, res.day);
+        BOOST_REQUIRE_EQUAL(actual.GetSeconds(), expected.GetSeconds());
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<SQL_TIME_STRUCT>(const char* request, const SQL_TIME_STRUCT& expected)
+    {
+        SQL_TIME_STRUCT res;
+
+        CheckSingleResult0(request, SQL_C_TIME, &res, 0, 0);
+
+        BOOST_REQUIRE_EQUAL(res.hour, expected.hour);
+        BOOST_REQUIRE_EQUAL(res.minute, expected.minute);
+        BOOST_REQUIRE_EQUAL(res.second, expected.second);
+    }
+
+    template<>
+    void SqlTestSuiteFixture::CheckSingleResult<Timestamp>(const char* request, const Timestamp& expected)
+    {
+        SQL_TIMESTAMP_STRUCT res;
+
+        CheckSingleResult0(request, SQL_C_TIMESTAMP, &res, 0, 0);
+
+        using ignite::impl::binary::BinaryUtils;
+        Timestamp actual = BinaryUtils::MakeTimestampGmt(res.year, res.month, res.day, res.hour, res.minute, res.second, res.fraction);
+
+        BOOST_REQUIRE_EQUAL(actual.GetSeconds(), expected.GetSeconds());
+        BOOST_REQUIRE_EQUAL(actual.GetSecondFraction(), expected.GetSecondFraction());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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
index 33797b0..bba806c 100644
--- a/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/sql_types_test.cpp
@@ -76,14 +76,14 @@ BOOST_AUTO_TEST_CASE(TestByteArraySelect)
 BOOST_AUTO_TEST_CASE(TestByteArrayParam)
 {
     SQLRETURN ret;
-    
+
     TestType in;
     in.i8Field = 101;
 
     const int8_t data[] = { 'A','B','C','D','E','F','G','H','I','J' };
     in.i8ArrayField.assign(data, data + sizeof(data) / sizeof(data[0]));
 
-    testCache.Put(1, in);   
+    testCache.Put(1, in);
 
     SQLLEN colLen = 0;
     SQLCHAR colData = 0;
@@ -139,7 +139,7 @@ BOOST_AUTO_TEST_CASE(TestByteArrayParamInsert)
 
     if (!SQL_SUCCEEDED(ret))
         BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
-    
+
     SQLLEN paramLen = paramData.size();
 
     ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_BINARY, SQL_VARBINARY, paramData.size(), 0, &paramData[0], paramData.size(), &paramLen);
@@ -188,4 +188,129 @@ BOOST_AUTO_TEST_CASE(TestByteParamInsert)
     BOOST_REQUIRE_EQUAL(out.i8Field, data);
 }
 
+BOOST_AUTO_TEST_CASE(TestTimestampSelect)
+{
+    TestType in1;
+    in1.i32Field = 1;
+    in1.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2017, 1, 13, 19, 54, 01, 987654321);
+
+    testCache.Put(1, in1);
+
+    CheckSingleResult<int32_t>(
+        "SELECT i32Field FROM TestType WHERE timestampField = '2017-01-13 19:54:01.987654321'", in1.i32Field);
+
+    CheckSingleResult<Timestamp>(
+        "SELECT timestampField FROM TestType WHERE i32Field = 1", in1.timestampField);
+}
+
+BOOST_AUTO_TEST_CASE(TestTimestampInsert)
+{
+    SQLRETURN ret;
+
+    SQLCHAR request[] = "INSERT INTO TestType(_key, timestampField) VALUES(?, ?)";
+
+    ret = SQLPrepare(stmt, request, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int64_t key = 1;
+    ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    SQL_TIMESTAMP_STRUCT data;
+    data.year = 2017;
+    data.month = 1;
+    data.day = 13;
+    data.hour = 19;
+    data.minute = 54;
+    data.second = 1;
+    data.fraction = 987654321;
+
+    using ignite::impl::binary::BinaryUtils;
+    Timestamp expected = BinaryUtils::MakeTimestampGmt(data.year, data.month, data.day, data.hour,
+        data.minute, data.second, data.fraction);
+
+    SQLLEN lenInd = sizeof(data);
+    ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_TIMESTAMP, SQL_TIMESTAMP, sizeof(data), 0, &data, sizeof(data), &lenInd);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLExecute(stmt);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    TestType out = testCache.Get(key);
+
+    BOOST_REQUIRE_EQUAL(out.timestampField.GetSeconds(), expected.GetSeconds());
+    BOOST_REQUIRE_EQUAL(out.timestampField.GetSecondFraction(), expected.GetSecondFraction());
+}
+
+BOOST_AUTO_TEST_CASE(TestTimeSelect)
+{
+    SQL_TIME_STRUCT ts;
+    ts.hour = 19;
+    ts.minute = 54;
+    ts.second = 1;
+
+    TestType in1;
+    in1.i32Field = 1;
+    in1.timestampField = impl::binary::BinaryUtils::MakeTimestampGmt(2017, 1, 13, ts.hour, ts.minute, ts.second);
+
+    testCache.Put(1, in1);
+
+    CheckSingleResult<SQL_TIME_STRUCT>(
+        "SELECT CAST(timestampField AS TIME) FROM TestType WHERE i32Field = 1", ts);
+
+    CheckSingleResult<int32_t>(
+        "SELECT i32Field FROM TestType WHERE CAST(timestampField AS TIME) = '19:54:01'", in1.i32Field);
+}
+
+BOOST_AUTO_TEST_CASE(TestTimeInsertToTimestamp)
+{
+    SQLRETURN ret;
+
+    SQLCHAR request[] = "INSERT INTO TestType(_key, timestampField) VALUES(?, ?)";
+
+    ret = SQLPrepare(stmt, request, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    int64_t key = 1;
+    ret = SQLBindParameter(stmt, 1, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_BIGINT, 0, 0, &key, 0, 0);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    SQL_TIME_STRUCT data = { 0 };
+    data.hour = 19;
+    data.minute = 54;
+    data.second = 1;
+
+    using ignite::impl::binary::BinaryUtils;
+    Timestamp expected = BinaryUtils::MakeTimestampGmt(1970, 1, 1, data.hour,
+        data.minute, data.second, 0);
+
+    SQLLEN lenInd = sizeof(data);
+    ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_TIME, SQL_TIMESTAMP, sizeof(data), 0, &data, sizeof(data), &lenInd);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLExecute(stmt);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    TestType out = testCache.Get(key);
+
+    BOOST_REQUIRE_EQUAL(out.timestampField.GetSeconds(), expected.GetSeconds());
+    BOOST_REQUIRE_EQUAL(out.timestampField.GetSecondFraction(), expected.GetSecondFraction());
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp
index 71c5f39..85979c8 100644
--- a/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp
+++ b/modules/platforms/cpp/odbc/src/app/application_data_buffer.cpp
@@ -45,7 +45,7 @@ namespace ignite
                 // No-op.
             }
 
-            ApplicationDataBuffer::ApplicationDataBuffer(type_traits::IgniteSqlType type, 
+            ApplicationDataBuffer::ApplicationDataBuffer(type_traits::IgniteSqlType type,
                 void* buffer, SqlLen buflen, SqlLen* reslen, int** offset) :
                 type(type),
                 buffer(buffer),
@@ -736,6 +736,20 @@ namespace ignite
                         break;
                     }
 
+                    case IGNITE_ODBC_C_TYPE_TTIME:
+                    {
+                        SQL_TIME_STRUCT* buffer = reinterpret_cast<SQL_TIME_STRUCT*>(dataPtr);
+
+                        buffer->hour = tmTime.tm_hour;
+                        buffer->minute = tmTime.tm_min;
+                        buffer->second = tmTime.tm_sec;
+
+                        if (resLenPtr)
+                            *resLenPtr = static_cast<SqlLen>(sizeof(SQL_TIME_STRUCT));
+
+                        break;
+                    }
+
                     case IGNITE_ODBC_C_TYPE_TTIMESTAMP:
                     {
                         SQL_TIMESTAMP_STRUCT* buffer = reinterpret_cast<SQL_TIMESTAMP_STRUCT*>(dataPtr);
@@ -856,6 +870,20 @@ namespace ignite
                         break;
                     }
 
+                    case IGNITE_ODBC_C_TYPE_TTIME:
+                    {
+                        SQL_TIME_STRUCT* buffer = reinterpret_cast<SQL_TIME_STRUCT*>(dataPtr);
+
+                        buffer->hour = tmTime.tm_hour;
+                        buffer->minute = tmTime.tm_min;
+                        buffer->second = tmTime.tm_sec;
+
+                        if (resLenPtr)
+                            *resLenPtr = static_cast<SqlLen>(sizeof(SQL_TIME_STRUCT));
+
+                        break;
+                    }
+
                     case IGNITE_ODBC_C_TYPE_TTIMESTAMP:
                     {
                         SQL_TIMESTAMP_STRUCT* buffer = reinterpret_cast<SQL_TIMESTAMP_STRUCT*>(dataPtr);
@@ -1080,7 +1108,7 @@ namespace ignite
                 return ApplyOffset(reslen);
             }
 
-            void* ApplicationDataBuffer::GetData() 
+            void* ApplicationDataBuffer::GetData()
             {
                 return ApplyOffset(buffer);
             }
@@ -1228,6 +1256,19 @@ namespace ignite
                         break;
                     }
 
+                    case IGNITE_ODBC_C_TYPE_TTIME:
+                    {
+                        const SQL_TIME_STRUCT* buffer = reinterpret_cast<const SQL_TIME_STRUCT*>(GetData());
+
+                        tmTime.tm_year = 70;
+                        tmTime.tm_mday = 1;
+                        tmTime.tm_hour = buffer->hour;
+                        tmTime.tm_min = buffer->minute;
+                        tmTime.tm_sec = buffer->second;
+
+                        break;
+                    }
+
                     case IGNITE_ODBC_C_TYPE_TTIMESTAMP:
                     {
                         const SQL_TIMESTAMP_STRUCT* buffer = reinterpret_cast<const SQL_TIMESTAMP_STRUCT*>(GetData());
@@ -1289,6 +1330,19 @@ namespace ignite
                         break;
                     }
 
+                    case IGNITE_ODBC_C_TYPE_TTIME:
+                    {
+                        const SQL_TIME_STRUCT* buffer = reinterpret_cast<const SQL_TIME_STRUCT*>(GetData());
+
+                        tmTime.tm_year = 70;
+                        tmTime.tm_mday = 1;
+                        tmTime.tm_hour = buffer->hour;
+                        tmTime.tm_min = buffer->minute;
+                        tmTime.tm_sec = buffer->second;
+
+                        break;
+                    }
+
                     case IGNITE_ODBC_C_TYPE_TTIMESTAMP:
                     {
                         const SQL_TIMESTAMP_STRUCT* buffer = reinterpret_cast<const SQL_TIMESTAMP_STRUCT*>(GetData());

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/modules/platforms/cpp/odbc/src/app/parameter.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/app/parameter.cpp b/modules/platforms/cpp/odbc/src/app/parameter.cpp
index 937ef58..ded2e4b 100644
--- a/modules/platforms/cpp/odbc/src/app/parameter.cpp
+++ b/modules/platforms/cpp/odbc/src/app/parameter.cpp
@@ -40,7 +40,7 @@ namespace ignite
                 // No-op.
             }
 
-            Parameter::Parameter(const ApplicationDataBuffer& buffer, int16_t sqlType, 
+            Parameter::Parameter(const ApplicationDataBuffer& buffer, int16_t sqlType,
                 size_t columnSize, int16_t decDigits) :
                 buffer(buffer),
                 sqlType(sqlType),
@@ -150,12 +150,14 @@ namespace ignite
                         break;
                     }
 
+                    case SQL_TYPE_DATE:
                     case SQL_DATE:
                     {
                         writer.WriteDate(buf.GetDate());
                         break;
                     }
 
+                    case SQL_TYPE_TIMESTAMP:
                     case SQL_TIMESTAMP:
                     {
                         writer.WriteTimestamp(buf.GetTimestamp());

http://git-wip-us.apache.org/repos/asf/ignite/blob/664dc88e/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 ed76aab..a34d434 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -29,11 +29,11 @@
 
 #ifndef SQL_ASYNC_NOTIFICATION_NOT_CAPABLE
 #define SQL_ASYNC_NOTIFICATION_NOT_CAPABLE      0x00000000L
-#endif 
+#endif
 
 #ifndef SQL_ASYNC_NOTIFICATION_CAPABLE
 #define SQL_ASYNC_NOTIFICATION_CAPABLE          0x00000001L
-#endif 
+#endif
 
 namespace ignite
 {
@@ -93,7 +93,32 @@ namespace ignite
                     DBG_STR_CASE(SQL_SQL92_VALUE_EXPRESSIONS);
                     DBG_STR_CASE(SQL_STATIC_CURSOR_ATTRIBUTES1);
                     DBG_STR_CASE(SQL_STATIC_CURSOR_ATTRIBUTES2);
-                default: 
+                    DBG_STR_CASE(SQL_CONVERT_BIGINT);
+                    DBG_STR_CASE(SQL_CONVERT_BINARY);
+                    DBG_STR_CASE(SQL_CONVERT_BIT);
+                    DBG_STR_CASE(SQL_CONVERT_CHAR);
+                    DBG_STR_CASE(SQL_CONVERT_DATE);
+                    DBG_STR_CASE(SQL_CONVERT_DECIMAL);
+                    DBG_STR_CASE(SQL_CONVERT_DOUBLE);
+                    DBG_STR_CASE(SQL_CONVERT_FLOAT);
+                    DBG_STR_CASE(SQL_CONVERT_INTEGER);
+                    DBG_STR_CASE(SQL_CONVERT_LONGVARCHAR);
+                    DBG_STR_CASE(SQL_CONVERT_NUMERIC);
+                    DBG_STR_CASE(SQL_CONVERT_REAL);
+                    DBG_STR_CASE(SQL_CONVERT_SMALLINT);
+                    DBG_STR_CASE(SQL_CONVERT_TIME);
+                    DBG_STR_CASE(SQL_CONVERT_TIMESTAMP);
+                    DBG_STR_CASE(SQL_CONVERT_TINYINT);
+                    DBG_STR_CASE(SQL_CONVERT_VARBINARY);
+                    DBG_STR_CASE(SQL_CONVERT_VARCHAR);
+                    DBG_STR_CASE(SQL_CONVERT_LONGVARBINARY);
+                    DBG_STR_CASE(SQL_CONVERT_WCHAR);
+                    DBG_STR_CASE(SQL_CONVERT_INTERVAL_DAY_TIME);
+                    DBG_STR_CASE(SQL_CONVERT_INTERVAL_YEAR_MONTH);
+                    DBG_STR_CASE(SQL_CONVERT_WLONGVARCHAR);
+                    DBG_STR_CASE(SQL_CONVERT_WVARCHAR);
+                    DBG_STR_CASE(SQL_CONVERT_GUID);
+                default:
                     break;
                 }
                 return "<< UNKNOWN TYPE >>";
@@ -114,7 +139,7 @@ namespace ignite
                 strParams[SQL_DBMS_VER]        = "03.00";
 
 #ifdef SQL_DRIVER_VER
-                // Driver version. At a minimum, the version is of the form 
+                // Driver version. At a minimum, the version is of the form
                 // ##.##.####, where the first two digits are the major version,
                 // the next two digits are the minor version, and the last four
                 // digits are the release version.
@@ -122,7 +147,7 @@ namespace ignite
 #endif // SQL_DRIVER_VER
 
 #ifdef SQL_COLUMN_ALIAS
-                // A character string: "Y" if the data source supports column 
+                // A character string: "Y" if the data source supports column
                 // aliases; otherwise, "N".
                 strParams[SQL_COLUMN_ALIAS] = "Y";
 #endif // SQL_COLUMN_ALIAS
@@ -165,7 +190,7 @@ namespace ignite
 #endif // SQL_TABLE_TERM
 
 #ifdef SQL_SCHEMA_TERM
-                // A character string with the data source vendor's name for 
+                // A character string with the data source vendor's name for
                 // a schema; for example, "owner", "Authorization ID", or "Schema".
                 strParams[SQL_SCHEMA_TERM] = "schema";
 #endif // SQL_SCHEMA_TERM
@@ -191,9 +216,9 @@ namespace ignite
 
 #ifdef SQL_ASYNC_NOTIFICATION
                 // Indicates if the driver supports asynchronous notification.
-                // SQL_ASYNC_NOTIFICATION_CAPABLE  = Asynchronous execution 
+                // SQL_ASYNC_NOTIFICATION_CAPABLE  = Asynchronous execution
                 // notification is supported by the driver.
-                // SQL_ASYNC_NOTIFICATION_NOT_CAPABLE Asynchronous execution 
+                // SQL_ASYNC_NOTIFICATION_NOT_CAPABLE Asynchronous execution
                 // notification is not supported by the driver.
                 intParams[SQL_ASYNC_NOTIFICATION] = SQL_ASYNC_NOTIFICATION_NOT_CAPABLE;
 #endif // SQL_ASYNC_NOTIFICATION
@@ -204,7 +229,7 @@ namespace ignite
 #endif // SQL_GETDATA_EXTENSIONS
 
 #ifdef SQL_ODBC_INTERFACE_CONFORMANCE
-                // Indicates the level of the ODBC 3.x interface that the driver 
+                // Indicates the level of the ODBC 3.x interface that the driver
                 // complies with.
                 intParams[SQL_ODBC_INTERFACE_CONFORMANCE] = SQL_OIC_CORE;
 #endif // SQL_ODBC_INTERFACE_CONFORMANCE
@@ -226,7 +251,7 @@ namespace ignite
 #endif // SQL_SCHEMA_USAGE
 
 #ifdef SQL_MAX_IDENTIFIER_LEN
-                // Indicates the maximum size in characters that the data source 
+                // Indicates the maximum size in characters that the data source
                 // supports for user-defined names.
                 intParams[SQL_MAX_IDENTIFIER_LEN] = 128;
 #endif // SQL_MAX_IDENTIFIER_LEN
@@ -240,7 +265,7 @@ 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 | SQL_FN_NUM_ACOS | SQL_FN_NUM_ASIN | 
+                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 |
@@ -270,7 +295,7 @@ namespace ignite
 #endif // SQL_TIMEDATE_FUNCTIONS
 
 #ifdef SQL_TIMEDATE_ADD_INTERVALS
-                // Bitmask enumerating timestamp intervals supported by the driver 
+                // Bitmask enumerating timestamp intervals supported by the driver
                 // and associated data source for the TIMESTAMPADD scalar function.
                 intParams[SQL_TIMEDATE_ADD_INTERVALS] = 0;
 #endif // SQL_TIMEDATE_ADD_INTERVALS
@@ -296,11 +321,11 @@ namespace ignite
 #ifdef SQL_CONVERT_FUNCTIONS
                 // Bitmask enumerating the scalar conversion functions supported
                 // by the driver and associated data source.
-                intParams[SQL_CONVERT_FUNCTIONS] = 0;
+                intParams[SQL_CONVERT_FUNCTIONS] = SQL_FN_CVT_CONVERT | SQL_FN_CVT_CAST;
 #endif // SQL_CONVERT_FUNCTIONS
 
 #ifdef SQL_OJ_CAPABILITIES
-                // Bitmask enumerating the types of outer joins supported by the 
+                // Bitmask enumerating the types of outer joins supported by the
                 // driver and data source.
                 intParams[SQL_OJ_CAPABILITIES] = SQL_OJ_LEFT | SQL_OJ_NOT_ORDERED | SQL_OJ_ALL_COMPARISON_OPS;
 #endif // SQL_OJ_CAPABILITIES
@@ -330,7 +355,7 @@ namespace ignite
 #ifdef SQL_SQL92_VALUE_EXPRESSIONS
                 // Bitmask enumerating the value expressions supported,
                 // as defined in SQL-92.
-                intParams[SQL_SQL92_VALUE_EXPRESSIONS] = SQL_SVE_CASE | 
+                intParams[SQL_SQL92_VALUE_EXPRESSIONS] = SQL_SVE_CASE |
                     SQL_SVE_CAST | SQL_SVE_COALESCE | SQL_SVE_NULLIF;
 #endif // SQL_SQL92_VALUE_EXPRESSIONS
 
@@ -366,6 +391,207 @@ namespace ignite
                 intParams[SQL_STATIC_CURSOR_ATTRIBUTES2] = 0;
 #endif //SQL_STATIC_CURSOR_ATTRIBUTES2
 
+#ifdef SQL_CONVERT_BIGINT
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type BIGINT
+                intParams[SQL_CONVERT_BIGINT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR |
+                    SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_BIT |
+                    SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT |
+                    SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_BIGINT
+
+#ifdef SQL_CONVERT_BINARY
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type BINARY
+                intParams[SQL_CONVERT_BINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC | SQL_CVT_DECIMAL |
+                    SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
+                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
+                    SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_BINARY
+
+#ifdef SQL_CONVERT_BIT
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type BIT
+                intParams[SQL_CONVERT_BIT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR | SQL_CVT_NUMERIC |
+                    SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT;
+#endif //SQL_CONVERT_BIT
+
+#ifdef SQL_CONVERT_CHAR
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type CHAR
+                intParams[SQL_CONVERT_CHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
+                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_CHAR
+
+#ifdef SQL_CONVERT_VARCHAR
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type VARCHAR
+                intParams[SQL_CONVERT_VARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
+                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_VARCHAR
+
+#ifdef SQL_CONVERT_LONGVARCHAR
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type LONGVARCHAR
+                intParams[SQL_CONVERT_LONGVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
+                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_LONGVARCHAR
+
+#ifdef SQL_CONVERT_WCHAR
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type WCHAR
+                intParams[SQL_CONVERT_WCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
+                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_WCHAR
+
+#ifdef SQL_CONVERT_WVARCHAR
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type WVARCHAR
+                intParams[SQL_CONVERT_WVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
+                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_WVARCHAR
+
+#ifdef SQL_CONVERT_WLONGVARCHAR
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type WLONGVARCHAR
+                intParams[SQL_CONVERT_WLONGVARCHAR] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
+                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_WLONGVARCHAR
+
+#ifdef SQL_CONVERT_DATE
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type DATE
+                intParams[SQL_CONVERT_DATE] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
+                    SQL_CVT_DATE | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_DATE
+
+#ifdef SQL_CONVERT_DECIMAL
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type DECIMAL
+                intParams[SQL_CONVERT_DECIMAL] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT |
+                    SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_DECIMAL
+
+#ifdef SQL_CONVERT_DOUBLE
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type DOUBLE
+                intParams[SQL_CONVERT_DOUBLE] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR | SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
+                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_DOUBLE
+
+#ifdef SQL_CONVERT_FLOAT
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type FLOAT
+                intParams[SQL_CONVERT_FLOAT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT | SQL_CVT_INTEGER |
+                    SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_FLOAT
+
+#ifdef SQL_CONVERT_REAL
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type REAL
+                intParams[SQL_CONVERT_REAL] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_REAL
+
+#ifdef SQL_CONVERT_INTEGER
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type INTEGER
+                intParams[SQL_CONVERT_INTEGER] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY |
+                    SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_INTEGER
+
+#ifdef SQL_CONVERT_NUMERIC
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type NUMERIC
+                intParams[SQL_CONVERT_NUMERIC] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT |
+                    SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
+                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_NUMERIC
+
+#ifdef SQL_CONVERT_SMALLINT
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type SMALLINT
+                intParams[SQL_CONVERT_SMALLINT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT |
+                    SQL_CVT_SMALLINT | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL |
+                    SQL_CVT_FLOAT | SQL_CVT_DOUBLE | SQL_CVT_BINARY | SQL_CVT_VARBINARY |
+                    SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_SMALLINT
+
+#ifdef SQL_CONVERT_TINYINT
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type TINYINT
+                intParams[SQL_CONVERT_TINYINT] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_TINYINT
+
+#ifdef SQL_CONVERT_TIME
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type TIME
+                intParams[SQL_CONVERT_TIME] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY |
+                    SQL_CVT_TIME | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_TIME
+
+#ifdef SQL_CONVERT_TIMESTAMP
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type TIMESTAMP
+                intParams[SQL_CONVERT_TIMESTAMP] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_BINARY |
+                    SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_DATE | SQL_CVT_TIME | SQL_CVT_TIMESTAMP;
+#endif //SQL_CONVERT_TIMESTAMP
+
+#ifdef SQL_CONVERT_VARBINARY
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type VARBINARY
+                intParams[SQL_CONVERT_VARBINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_DATE |
+                    SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_VARBINARY
+
+#ifdef SQL_CONVERT_LONGVARBINARY
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type LONGVARBINARY
+                intParams[SQL_CONVERT_LONGVARBINARY] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_NUMERIC | SQL_CVT_DECIMAL | SQL_CVT_BIT | SQL_CVT_TINYINT | SQL_CVT_SMALLINT |
+                    SQL_CVT_INTEGER | SQL_CVT_BIGINT | SQL_CVT_REAL | SQL_CVT_FLOAT | SQL_CVT_DOUBLE |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_DATE |
+                    SQL_CVT_TIME | SQL_CVT_TIMESTAMP | SQL_CVT_GUID;
+#endif //SQL_CONVERT_LONGVARBINARY
+
+#ifdef SQL_CONVERT_GUID
+                // Bitmask indicates the conversions supported by the CONVERT scalar function for target type GUID
+                intParams[SQL_CONVERT_GUID] = SQL_CVT_CHAR | SQL_CVT_VARCHAR | SQL_CVT_LONGVARCHAR |
+                    SQL_CVT_WCHAR | SQL_CVT_WLONGVARCHAR | SQL_CVT_WVARCHAR |
+                    SQL_CVT_BINARY | SQL_CVT_VARBINARY | SQL_CVT_LONGVARBINARY | SQL_CVT_GUID;
+#endif //SQL_CONVERT_GUID
+
                 //======================= Short Params ========================
 #ifdef SQL_MAX_CONCURRENT_ACTIVITIES
                 // The maximum number of active statements that the driver can
@@ -409,10 +635,10 @@ namespace ignite
 
                 StringInfoMap::const_iterator itStr = strParams.find(type);
 
-                if (itStr != strParams.end()) 
+                if (itStr != strParams.end())
                 {
                     unsigned short strlen = static_cast<short>(
-                        utility::CopyStringToBuffer(itStr->second, 
+                        utility::CopyStringToBuffer(itStr->second,
                             reinterpret_cast<char*>(buf), buflen));
 
                     if (reslen)